From 3b6c647b1b86f43625f55a4b9b50a9f9d86fcba7 Mon Sep 17 00:00:00 2001 From: Tugdual Saunier Date: Wed, 28 May 2014 15:54:53 +0100 Subject: [PATCH 001/270] Fixed maven build when druid is integrated as a submodule --- services/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/services/pom.xml b/services/pom.xml index 80c4b4add81..16b02c6bb8c 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -101,10 +101,6 @@ distro-assembly - package - - assembly - src/assembly/assembly.xml From 77ec4df79718a218acd78a7d45cdfe87935eba83 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 3 Jun 2014 13:43:38 -0700 Subject: [PATCH 002/270] update guava, java-util, and druid-api --- .../collections/OrderedMergeSequence.java | 3 ++- .../druid/common/utils/SerializerUtils.java | 7 +---- .../storage/hdfs/HdfsDataSegmentPuller.java | 27 +++++++++---------- .../storage/hdfs/HdfsDataSegmentPusher.java | 12 +++------ .../io/druid/indexer/IndexGeneratorJob.java | 5 ++-- .../common/task/RealtimeIndexTask.java | 9 +++---- .../indexing/overlord/RemoteTaskRunner.java | 2 +- .../io/druid/indexing/common/TestUtils.java | 2 +- .../kafka/KafkaEightFirehoseFactory.java | 10 +++---- .../kafka/KafkaSevenFirehoseFactory.java | 12 ++++----- pom.xml | 8 +++--- .../data/input/ProtoBufInputRowParser.java | 4 +-- .../ReferenceCountingSegmentQueryRunner.java | 4 +-- .../cardinality/CardinalityAggregator.java | 6 ++--- .../query/groupby/GroupByQueryEngine.java | 6 ++--- .../druid/query/topn/PooledTopNAlgorithm.java | 4 +-- .../ColumnSelectorBitmapIndexSelector.java | 4 +-- .../main/java/io/druid/segment/IndexIO.java | 11 +++----- .../java/io/druid/segment/IndexMerger.java | 7 ++--- .../io/druid/segment/MMappedIndexAdapter.java | 6 ++--- .../java/io/druid/segment/MetricHolder.java | 6 ++--- .../QueryableIndexIndexableAdapter.java | 6 ++--- .../segment/QueryableIndexStorageAdapter.java | 22 +++++++-------- .../io/druid/segment/column/SimpleColumn.java | 4 +-- .../data/CompressedFloatsIndexedSupplier.java | 3 ++- .../CompressedFloatsSupplierSerializer.java | 8 +----- .../data/CompressedLongsIndexedSupplier.java | 3 ++- .../CompressedLongsSupplierSerializer.java | 8 +----- .../data/CompressedObjectStrategy.java | 6 ++--- .../io/druid/segment/data/GenericIndexed.java | 8 +++--- .../data/InMemoryCompressedFloats.java | 6 ++--- .../segment/data/InMemoryCompressedLongs.java | 3 ++- .../test/java/io/druid/segment/TestIndex.java | 1 + .../CompressedLongsIndexedSupplierTest.java | 8 +++--- .../druid/storage/s3/S3DataSegmentPuller.java | 11 +++----- .../io/druid/client/DirectDruidClient.java | 6 ++--- .../druid/curator/announcement/Announcer.java | 6 ++--- .../segment/realtime/RealtimeManager.java | 15 ++++++----- .../realtime/firehose/IrcFirehoseFactory.java | 5 ---- .../segment/realtime/firehose/IrcParser.java | 5 +--- .../java/io/druid/server/QueryResource.java | 4 +-- .../server/bridge/DruidClusterBridge.java | 4 +-- .../server/coordinator/DruidCoordinator.java | 4 +-- .../initialization/PropertiesModule.java | 4 +-- .../druid/server/log/FileRequestLogger.java | 6 ++--- .../java/io/druid/server/sql/SQLRunner.java | 4 +-- .../client/BatchServerInventoryViewTest.java | 2 +- .../io/druid/curator/CuratorTestBase.java | 6 ++--- .../segment/realtime/RealtimeManagerTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 5 ++-- .../DruidCoordinatorBalancerProfiler.java | 4 +-- .../druid/cli/convert/ConvertProperties.java | 4 +-- 52 files changed, 147 insertions(+), 191 deletions(-) diff --git a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java index 1b076c4c299..cfc29682d0d 100644 --- a/common/src/main/java/io/druid/collections/OrderedMergeSequence.java +++ b/common/src/main/java/io/druid/collections/OrderedMergeSequence.java @@ -24,6 +24,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.Ordering; import com.google.common.io.Closeables; import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.Yielders; @@ -70,7 +71,7 @@ public class OrderedMergeSequence implements Sequence return yielder.get(); } finally { - Closeables.closeQuietly(yielder); + CloseQuietly.close(yielder); } } diff --git a/common/src/main/java/io/druid/common/utils/SerializerUtils.java b/common/src/main/java/io/druid/common/utils/SerializerUtils.java index 28cdf7629a3..b5dc650de5c 100644 --- a/common/src/main/java/io/druid/common/utils/SerializerUtils.java +++ b/common/src/main/java/io/druid/common/utils/SerializerUtils.java @@ -46,14 +46,9 @@ public class SerializerUtils public void writeString(OutputSupplier supplier, String name) throws IOException { - OutputStream out = null; - try { - out = supplier.getOutput(); + try (OutputStream out = supplier.getOutput()) { writeString(out, name); } - finally { - Closeables.closeQuietly(out); - } } public void writeString(WritableByteChannel out, String name) throws IOException diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java index 6af99afcbcd..6b6f002f774 100644 --- a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPuller.java @@ -21,6 +21,7 @@ package io.druid.storage.hdfs; import com.google.common.io.Closeables; import com.google.inject.Inject; +import com.metamx.common.guava.CloseQuietly; import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; @@ -52,22 +53,17 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller final FileSystem fs = checkPathAndGetFilesystem(path); - FSDataInputStream in = null; - try { - if (path.getName().endsWith(".zip")) { - in = fs.open(path); - CompressionUtils.unzip(in, dir); - in.close(); + if (path.getName().endsWith(".zip")) { + try { + try (FSDataInputStream in = fs.open(path)) { + CompressionUtils.unzip(in, dir); + } } - else { - throw new SegmentLoadingException("Unknown file type[%s]", path); + catch (IOException e) { + throw new SegmentLoadingException(e, "Some IOException"); } - } - catch (IOException e) { - throw new SegmentLoadingException(e, "Some IOException"); - } - finally { - Closeables.closeQuietly(in); + } else { + throw new SegmentLoadingException("Unknown file type[%s]", path); } } @@ -85,7 +81,8 @@ public class HdfsDataSegmentPuller implements DataSegmentPuller } } - private Path getPath(DataSegment segment) { + private Path getPath(DataSegment segment) + { return new Path(String.valueOf(segment.getLoadSpec().get("path"))); } diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java index 2bb36abb2a0..239cfc01d04 100644 --- a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentPusher.java @@ -26,6 +26,7 @@ import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.common.io.OutputSupplier; import com.google.inject.Inject; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.druid.segment.SegmentUtils; import io.druid.segment.loading.DataSegmentPusher; @@ -78,17 +79,10 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher fs.mkdirs(outFile.getParent()); log.info("Compressing files from[%s] to [%s]", inDir, outFile); - FSDataOutputStream out = null; + long size; - try { - out = fs.create(outFile); - + try (FSDataOutputStream out = fs.create(outFile)) { size = CompressionUtils.zip(inDir, out); - - out.close(); - } - finally { - Closeables.closeQuietly(out); } return createDescriptorFile( diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index b2f1dfda129..9341aae7024 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -31,6 +31,7 @@ import com.google.common.io.Closeables; import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; import io.druid.data.input.impl.StringInputRowParser; @@ -420,7 +421,7 @@ public class IndexGeneratorJob implements Jobby if (caughtException == null) { Closeables.close(out, false); } else { - Closeables.closeQuietly(out); + CloseQuietly.close(out); throw Throwables.propagate(caughtException); } } @@ -600,7 +601,7 @@ public class IndexGeneratorJob implements Jobby } } finally { - Closeables.closeQuietly(in); + CloseQuietly.close(in); } out.closeEntry(); context.progress(); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index e0fc20d1216..bd04596f047 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -24,9 +24,8 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.io.Closeables; import com.metamx.common.Granularity; -import com.metamx.common.exception.FormattedException; +import com.metamx.common.guava.CloseQuietly; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -44,8 +43,8 @@ import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryToolChest; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.RealtimeIOConfig; +import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.FireDepartment; import io.druid.segment.realtime.FireDepartmentConfig; @@ -353,7 +352,7 @@ public class RealtimeIndexTask extends AbstractTask nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); } } - catch (FormattedException e) { + catch (Exception e) { log.warn(e, "unparseable line"); fireDepartment.getMetrics().incrementUnparseable(); } @@ -375,7 +374,7 @@ public class RealtimeIndexTask extends AbstractTask log.makeAlert(e, "Failed to finish realtime task").emit(); } finally { - Closeables.closeQuietly(firehose); + CloseQuietly.close(firehose); toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index a20d2a42dbc..c546fbcce06 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -585,7 +585,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer // Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running // on a worker - this avoids overflowing a worker with tasks - Stopwatch timeoutStopwatch = new Stopwatch(); + Stopwatch timeoutStopwatch = Stopwatch.createUnstarted(); timeoutStopwatch.start(); synchronized (statusLock) { while (!isWorkerRunningTask(theWorker, task.getId())) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index 2ec6c474e6c..99a9e65b815 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -31,7 +31,7 @@ public class TestUtils public static boolean conditionValid(IndexingServiceCondition condition) { try { - Stopwatch stopwatch = new Stopwatch(); + Stopwatch stopwatch = Stopwatch.createUnstarted(); stopwatch.start(); while (!condition.isValid()) { Thread.sleep(100); diff --git a/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java index ab7987f93ee..70d373d7319 100644 --- a/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java +++ b/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java @@ -22,9 +22,9 @@ package io.druid.firehose.kafka; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; -import com.metamx.common.exception.FormattedException; import com.metamx.common.logger.Logger; import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.Firehose; @@ -115,7 +115,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactoryscm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.117-SNAPSHOT @@ -39,9 +39,9 @@ UTF-8 - 0.25.6 + 0.26.0-SNAPSHOT 2.4.0 - 0.2.3 + 0.2.4-SNAPSHOT @@ -198,7 +198,7 @@ com.google.guava guava - 14.0.1 + 17.0 com.google.inject diff --git a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java index 63620ffd498..df8bdb5cef8 100644 --- a/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java +++ b/processing/src/main/java/io/druid/data/input/ProtoBufInputRowParser.java @@ -28,10 +28,8 @@ import com.google.protobuf.ByteString; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.InvalidProtocolBufferException; -import com.metamx.common.exception.FormattedException; import com.metamx.common.logger.Logger; import io.druid.data.input.impl.DimensionsSpec; -import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.ParseSpec; @@ -94,7 +92,7 @@ public class ProtoBufInputRowParser implements ByteBufferInputRowParser } @Override - public InputRow parse(ByteBuffer input) throws FormattedException + public InputRow parse(ByteBuffer input) { // We should really create a ProtoBufBasedInputRow that does not need an intermediate map but accesses // the DynamicMessage directly... diff --git a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java index c366c287ac5..e801e5516ac 100644 --- a/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -19,7 +19,7 @@ package io.druid.query; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.ResourceClosingSequence; import com.metamx.common.guava.Sequence; import io.druid.segment.ReferenceCountingSegment; @@ -52,7 +52,7 @@ public class ReferenceCountingSegmentQueryRunner implements QueryRunner return new ResourceClosingSequence(baseSequence, closeable); } catch (RuntimeException e) { - Closeables.closeQuietly(closeable); + CloseQuietly.close(closeable); throw e; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java index 85a07fd0715..afd893afc3f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregator.java @@ -54,7 +54,7 @@ public class CardinalityAggregator implements Aggregator // nothing to add to hasher if size == 0, only handle size == 1 and size != 0 cases. if (size == 1) { final String value = selector.lookupName(row.get(0)); - hasher.putString(value != null ? value : NULL_STRING); + hasher.putUnencodedChars(value != null ? value : NULL_STRING); } else if (size != 0) { final String[] values = new String[size]; for (int i = 0; i < size; ++i) { @@ -67,7 +67,7 @@ public class CardinalityAggregator implements Aggregator if (i != 0) { hasher.putChar(SEPARATOR); } - hasher.putString(values[i]); + hasher.putUnencodedChars(values[i]); } } } @@ -79,7 +79,7 @@ public class CardinalityAggregator implements Aggregator for (final DimensionSelector selector : selectors) { for (final Integer index : selector.getRow()) { final String value = selector.lookupName(index); - collector.add(hashFn.hashString(value == null ? NULL_STRING : value).asBytes()); + collector.add(hashFn.hashUnencodedChars(value == null ? NULL_STRING : value).asBytes()); } } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index d9e52014f86..8599c6bdd16 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -25,12 +25,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.FunctionalIterator; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; @@ -123,7 +123,7 @@ public class GroupByQueryEngine @Override public void cleanup(RowIterator iterFromMake) { - Closeables.closeQuietly(iterFromMake); + CloseQuietly.close(iterFromMake); } } ); @@ -135,7 +135,7 @@ public class GroupByQueryEngine @Override public void close() throws IOException { - Closeables.closeQuietly(bufferHolder); + CloseQuietly.close(bufferHolder); } } ) diff --git a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java index ea4fd37547d..caf892f292d 100644 --- a/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java +++ b/processing/src/main/java/io/druid/query/topn/PooledTopNAlgorithm.java @@ -19,8 +19,8 @@ package io.druid.query.topn; -import com.google.common.io.Closeables; import com.metamx.common.Pair; +import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidPool; import io.druid.query.aggregation.BufferAggregator; @@ -233,7 +233,7 @@ public class PooledTopNAlgorithm if (resultsBufHolder != null) { resultsBufHolder.get().clear(); } - Closeables.closeQuietly(resultsBufHolder); + CloseQuietly.close(resultsBufHolder); } public static class PooledTopNParams extends TopNParams diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index fc124621cb8..6aea75e2532 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -19,8 +19,8 @@ package io.druid.segment; -import com.google.common.io.Closeables; import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.common.guava.CloseQuietly; import io.druid.query.filter.BitmapIndexSelector; import io.druid.segment.column.Column; import io.druid.segment.column.DictionaryEncodedColumn; @@ -95,7 +95,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector return column.length(); } finally { - Closeables.closeQuietly(column); + CloseQuietly.close(column); } } diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 403789190ea..a78adad37ac 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -165,15 +165,10 @@ public class IndexIO } final File indexFile = new File(inDir, "index.drd"); - InputStream in = null; int version; - try { - in = new FileInputStream(indexFile); + try (InputStream in = new FileInputStream(indexFile)) { version = in.read(); } - finally { - Closeables.closeQuietly(in); - } return version; } @@ -194,8 +189,8 @@ public class IndexIO case 2: case 3: log.makeAlert("Attempt to load segment of version <= 3.") - .addData("version", version) - .emit(); + .addData("version", version) + .emit(); return false; case 4: case 5: diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 672cec724b5..cee858283ad 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -38,6 +38,7 @@ import com.metamx.collections.spatial.RTree; import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; import com.metamx.common.IAE; import com.metamx.common.ISE; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.MergeIterable; import com.metamx.common.guava.nary.BinaryFn; @@ -438,9 +439,9 @@ public class IndexMerger serializerUtils.writeString(channel, String.format("%s/%s", minTime, maxTime)); } finally { - Closeables.closeQuietly(channel); + CloseQuietly.close(channel); channel = null; - Closeables.closeQuietly(fileOutputStream); + CloseQuietly.close(fileOutputStream); fileOutputStream = null; } IndexIO.checkFileSize(indexFile); @@ -881,7 +882,7 @@ public class IndexMerger ); } finally { - Closeables.closeQuietly(channel); + CloseQuietly.close(channel); channel = null; } IndexIO.checkFileSize(indexFile); diff --git a/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java b/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java index 0db39fb9261..0af0a82bbc8 100644 --- a/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/MMappedIndexAdapter.java @@ -20,7 +20,7 @@ package io.druid.segment; import com.google.common.collect.Maps; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import io.druid.segment.data.ConciseCompressedIndexedInts; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedFloats; @@ -118,9 +118,9 @@ public class MMappedIndexAdapter implements IndexableAdapter { final boolean hasNext = currRow < numRows; if (!hasNext && !done) { - Closeables.closeQuietly(timestamps); + CloseQuietly.close(timestamps); for (IndexedFloats floatMetric : floatMetrics) { - Closeables.closeQuietly(floatMetric); + CloseQuietly.close(floatMetric); } done = true; } diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 648f2fbe70c..2627444e758 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -20,11 +20,11 @@ package io.druid.segment; import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; import com.google.common.io.InputSupplier; import com.google.common.io.OutputSupplier; import com.metamx.common.IAE; import com.metamx.common.ISE; +import com.metamx.common.guava.CloseQuietly; import io.druid.common.utils.SerializerUtils; import io.druid.segment.data.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedFloatsSupplierSerializer; @@ -84,8 +84,8 @@ public class MetricHolder ByteStreams.copy(in, out); } finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(in); + CloseQuietly.close(out); + CloseQuietly.close(in); } } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index efe09b51f55..ef6dc2f6adc 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -22,8 +22,8 @@ package io.druid.segment; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.io.Closeables; import com.metamx.common.ISE; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; @@ -208,10 +208,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter { final boolean hasNext = currRow < numRows; if (!hasNext && !done) { - Closeables.closeQuietly(timestamps); + CloseQuietly.close(timestamps); for (Object metric : metrics) { if (metric instanceof Closeable) { - Closeables.closeQuietly((Closeable) metric); + CloseQuietly.close((Closeable) metric); } } done = true; diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index a71297b3b89..d43bd4d8a66 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -23,7 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Predicates; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; @@ -108,7 +108,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter return new DateTime(column.getLongSingleValueRow(0)); } finally { - Closeables.closeQuietly(column); + CloseQuietly.close(column); } } @@ -121,7 +121,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter return new DateTime(column.getLongSingleValueRow(column.length() - 1)); } finally { - Closeables.closeQuietly(column); + CloseQuietly.close(column); } } @@ -531,16 +531,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public void close() throws IOException { - Closeables.closeQuietly(timestamps); + CloseQuietly.close(timestamps); for (GenericColumn column : genericColumnCache.values()) { - Closeables.closeQuietly(column); + CloseQuietly.close(column); } for (ComplexColumn complexColumn : complexColumnCache.values()) { - Closeables.closeQuietly(complexColumn); + CloseQuietly.close(complexColumn); } for (Object column : objectColumnCache.values()) { if(column instanceof Closeable) { - Closeables.closeQuietly((Closeable) column); + CloseQuietly.close((Closeable) column); } } } @@ -955,16 +955,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public void close() throws IOException { - Closeables.closeQuietly(timestamps); + CloseQuietly.close(timestamps); for (GenericColumn column : genericColumnCache.values()) { - Closeables.closeQuietly(column); + CloseQuietly.close(column); } for (ComplexColumn complexColumn : complexColumnCache.values()) { - Closeables.closeQuietly(complexColumn); + CloseQuietly.close(complexColumn); } for (Object column : objectColumnCache.values()) { if (column instanceof Closeable) { - Closeables.closeQuietly((Closeable) column); + CloseQuietly.close((Closeable) column); } } } diff --git a/processing/src/main/java/io/druid/segment/column/SimpleColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleColumn.java index 597069a3b93..6ef558901f7 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleColumn.java @@ -20,7 +20,7 @@ package io.druid.segment.column; import com.google.common.base.Supplier; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; /** */ @@ -68,7 +68,7 @@ class SimpleColumn implements Column return column.length(); } finally { - Closeables.closeQuietly(column); + CloseQuietly.close(column); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java index c969d479068..1beccc5426f 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -25,6 +25,7 @@ import com.google.common.io.Closeables; import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.metamx.common.IAE; +import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; @@ -123,7 +124,7 @@ public class CompressedFloatsIndexedSupplier implements Supplier private void loadBuffer(int bufferNum) { - Closeables.closeQuietly(holder); + CloseQuietly.close(holder); holder = baseFloatBuffers.get(bufferNum); buffer = holder.get(); currIndex = bufferNum; diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java index d0bf42ee451..da967898dba 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java @@ -106,17 +106,11 @@ public class CompressedFloatsSupplierSerializer flattener.close(); - OutputStream out = null; - try { - out = consolidatedOut.getOutput(); - + try (OutputStream out = consolidatedOut.getOutput()) { out.write(CompressedFloatsIndexedSupplier.version); out.write(Ints.toByteArray(numInserted)); out.write(Ints.toByteArray(sizePer)); ByteStreams.copy(flattener.combineStreams(), out); } - finally { - Closeables.closeQuietly(out); - } } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index c9d433b7740..8ad267168a5 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -25,6 +25,7 @@ import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.IAE; +import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; @@ -122,7 +123,7 @@ public class CompressedLongsIndexedSupplier implements Supplier private void loadBuffer(int bufferNum) { - Closeables.closeQuietly(holder); + CloseQuietly.close(holder); holder = baseLongBuffers.get(bufferNum); buffer = holder.get(); currIndex = bufferNum; diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java index 558c81afa47..b0e63a8e391 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java @@ -100,17 +100,11 @@ public class CompressedLongsSupplierSerializer flattener.close(); - OutputStream out = null; - try { - out = consolidatedOut.getOutput(); - + try (OutputStream out = consolidatedOut.getOutput()) { out.write(CompressedLongsIndexedSupplier.version); out.write(Ints.toByteArray(numInserted)); out.write(Ints.toByteArray(sizePer)); ByteStreams.copy(flattener.combineStreams(), out); } - finally { - Closeables.closeQuietly(out); - } } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index 2c42ca1327f..4a0b5723b41 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -20,7 +20,7 @@ package io.druid.segment.data; import com.google.common.base.Throwables; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import com.ning.compress.lzf.ChunkEncoder; import com.ning.compress.lzf.LZFChunk; import com.ning.compress.lzf.LZFDecoder; @@ -74,7 +74,7 @@ public class CompressedObjectStrategy implements ObjectStrateg buf.put(outputBytes, 0, numDecompressedBytes); buf.flip(); - Closeables.closeQuietly(outputBytesHolder); + CloseQuietly.close(outputBytesHolder); return new ResourceHolder() { @@ -105,7 +105,7 @@ public class CompressedObjectStrategy implements ObjectStrateg final ResourceHolder encoder = CompressedPools.getChunkEncoder(); LZFChunk chunk = encoder.get().encodeChunk(buf.array(), 0, buf.array().length); - Closeables.closeQuietly(encoder); + CloseQuietly.close(encoder); return chunk.getData(); } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 1ae8ec8ea01..4e56d86da27 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -21,9 +21,9 @@ package io.druid.segment.data; import com.google.common.base.Charsets; import com.google.common.collect.Ordering; -import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import com.metamx.common.IAE; +import com.metamx.common.guava.CloseQuietly; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -73,14 +73,14 @@ public class GenericIndexed implements Indexed allowReverseLookup = false; } if (prevVal instanceof Closeable) { - Closeables.closeQuietly((Closeable) prevVal); + CloseQuietly.close((Closeable) prevVal); } prevVal = next; ++count; } if (prevVal instanceof Closeable) { - Closeables.closeQuietly((Closeable) prevVal); + CloseQuietly.close((Closeable) prevVal); } ByteArrayOutputStream headerBytes = new ByteArrayOutputStream(4 + (count * 4)); @@ -98,7 +98,7 @@ public class GenericIndexed implements Indexed valueBytes.write(bytes); if (object instanceof Closeable) { - Closeables.closeQuietly((Closeable) object); + CloseQuietly.close((Closeable) object); } } } diff --git a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java index 4e6aaac55fb..d056158fa7d 100644 --- a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java +++ b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java @@ -22,7 +22,7 @@ package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; @@ -153,7 +153,7 @@ public class InMemoryCompressedFloats implements IndexedFloats private void loadBuffer(int bufferNum) { loadBuffer = null; - Closeables.closeQuietly(holder); + CloseQuietly.close(holder); final byte[] compressedBytes = compressedBuffers.get(bufferNum); holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length); loadBuffer = holder.get(); @@ -191,6 +191,6 @@ public class InMemoryCompressedFloats implements IndexedFloats @Override public void close() throws IOException { - Closeables.closeQuietly(holder); + CloseQuietly.close(holder); } } diff --git a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java index 67a05b17508..e0ef6fac375 100644 --- a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java +++ b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; @@ -163,7 +164,7 @@ public class InMemoryCompressedLongs implements IndexedLongs private void loadBuffer(int bufferNum) { loadBuffer = null; - Closeables.closeQuietly(holder); + CloseQuietly.close(holder); final byte[] compressedBytes = compressedBuffers.get(bufferNum); holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length); loadBuffer = holder.get(); diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 95823210549..d3073bcd6ac 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -180,6 +180,7 @@ public class TestIndex new TimestampSpec("ts", "iso"), new DimensionsSpec(Arrays.asList(DIMENSIONS), null, null), "\t", + "\u0001", Arrays.asList(COLUMNS) ), null, null, null, null diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java index 873d3b3d423..768100fd559 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java @@ -19,8 +19,8 @@ package io.druid.segment.data; -import com.google.common.io.Closeables; import com.google.common.primitives.Longs; +import com.metamx.common.guava.CloseQuietly; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -47,7 +47,7 @@ public class CompressedLongsIndexedSupplierTest @Before public void setUp() throws Exception { - Closeables.closeQuietly(indexed); + CloseQuietly.close(indexed); indexed = null; supplier = null; vals = null; @@ -56,7 +56,7 @@ public class CompressedLongsIndexedSupplierTest @After public void tearDown() throws Exception { - Closeables.closeQuietly(indexed); + CloseQuietly.close(indexed); } private void setupSimple() @@ -247,7 +247,7 @@ public class CompressedLongsIndexedSupplierTest stopLatch.await(); } finally { - Closeables.closeQuietly(indexed2); + CloseQuietly.close(indexed2); } if (failureHappened.get()) { diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java index 1e310a9b90f..a40e524e94f 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java @@ -21,7 +21,6 @@ package io.druid.storage.s3; import com.google.common.base.Throwables; import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; import com.google.common.io.Files; import com.google.inject.Inject; import com.metamx.common.ISE; @@ -95,9 +94,7 @@ public class S3DataSegmentPuller implements DataSegmentPuller try { s3Obj = s3Client.getObject(s3Coords.bucket, s3Coords.path); - InputStream in = null; - try { - in = s3Obj.getDataInputStream(); + try (InputStream in = s3Obj.getDataInputStream()) { final String key = s3Obj.getKey(); if (key.endsWith(".zip")) { CompressionUtils.unzip(in, outDir); @@ -113,9 +110,6 @@ public class S3DataSegmentPuller implements DataSegmentPuller catch (IOException e) { throw new IOException(String.format("Problem decompressing object[%s]", s3Obj), e); } - finally { - Closeables.closeQuietly(in); - } } finally { S3Utils.closeStreamsQuietly(s3Obj); @@ -127,7 +121,8 @@ public class S3DataSegmentPuller implements DataSegmentPuller catch (Exception e) { try { FileUtils.deleteDirectory(outDir); - } catch (IOException ioe) { + } + catch (IOException ioe) { log.warn( ioe, "Failed to remove output directory for segment[%s] after exception: %s", diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index ae994d1cda7..f113bf794a0 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -28,7 +28,6 @@ import com.fasterxml.jackson.databind.type.TypeFactory; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Throwables; import com.google.common.collect.Maps; -import com.google.common.io.Closeables; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -36,6 +35,7 @@ import com.metamx.common.IAE; import com.metamx.common.Pair; import com.metamx.common.RE; import com.metamx.common.guava.BaseSequence; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; @@ -208,7 +208,7 @@ public class DirectDruidClient implements QueryRunner @Override public void cleanup(JsonParserIterator iterFromMake) { - Closeables.closeQuietly(iterFromMake); + CloseQuietly.close(iterFromMake); } } ); @@ -251,7 +251,7 @@ public class DirectDruidClient implements QueryRunner return false; } if (jp.getCurrentToken() == JsonToken.END_ARRAY) { - Closeables.closeQuietly(jp); + CloseQuietly.close(jp); return false; } diff --git a/server/src/main/java/io/druid/curator/announcement/Announcer.java b/server/src/main/java/io/druid/curator/announcement/Announcer.java index 99393a7af34..a0f396d4396 100644 --- a/server/src/main/java/io/druid/curator/announcement/Announcer.java +++ b/server/src/main/java/io/druid/curator/announcement/Announcer.java @@ -23,10 +23,10 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.MapMaker; import com.google.common.collect.Sets; -import com.google.common.io.Closeables; import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.Pair; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; @@ -112,7 +112,7 @@ public class Announcer started = false; for (Map.Entry entry : listeners.entrySet()) { - Closeables.closeQuietly(entry.getValue()); + CloseQuietly.close(entry.getValue()); } for (Map.Entry> entry : announcements.entrySet()) { @@ -353,7 +353,7 @@ public class Announcer cache.start(); } catch (Exception e) { - Closeables.closeQuietly(cache); + CloseQuietly.close(cache); throw Throwables.propagate(e); } } diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index f83a1e3911f..f249d5e3570 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -23,9 +23,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; -import com.google.common.io.Closeables; import com.google.inject.Inject; -import com.metamx.common.exception.FormattedException; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; @@ -95,7 +94,7 @@ public class RealtimeManager implements QuerySegmentWalker public void stop() { for (FireChief chief : chiefs.values()) { - Closeables.closeQuietly(chief); + CloseQuietly.close(chief); } } @@ -185,7 +184,7 @@ public class RealtimeManager implements QuerySegmentWalker long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); while (firehose.hasMore()) { - final InputRow inputRow; + InputRow inputRow = null; try { try { inputRow = firehose.nextRow(); @@ -214,8 +213,10 @@ public class RealtimeManager implements QuerySegmentWalker } metrics.incrementProcessed(); } - catch (FormattedException e) { - log.info(e, "unparseable line: %s", e.getDetails()); + catch (Exception e) { + if (inputRow != null) { + log.error(e, "unparseable line: %s", inputRow); + } metrics.incrementUnparseable(); continue; } @@ -237,7 +238,7 @@ public class RealtimeManager implements QuerySegmentWalker throw e; } finally { - Closeables.closeQuietly(firehose); + CloseQuietly.close(firehose); if (normalExit) { plumber.finishJob(); plumber = null; diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java index c495f293e34..f4fd3e6fc41 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java @@ -31,18 +31,13 @@ import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg; import com.ircclouds.irc.api.listeners.VariousMessageListenerAdapter; import com.ircclouds.irc.api.state.IIRCState; import com.metamx.common.Pair; -import com.metamx.common.exception.FormattedException; import com.metamx.common.logger.Logger; -import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.impl.InputRowParser; -import io.druid.data.input.impl.ParseSpec; import org.joda.time.DateTime; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.List; import java.util.UUID; import java.util.concurrent.LinkedBlockingQueue; diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IrcParser.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcParser.java index 68a93e061b1..57ae95a3e53 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IrcParser.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IrcParser.java @@ -24,12 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.ircclouds.irc.api.domain.messages.ChannelPrivMsg; import com.metamx.common.Pair; -import com.metamx.common.exception.FormattedException; import io.druid.data.input.InputRow; -import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.ParseSpec; -import io.druid.data.input.impl.TimestampSpec; import org.joda.time.DateTime; /** @@ -52,7 +49,7 @@ public class IrcParser implements InputRowParser> } @Override - public InputRow parse(Pair msg) throws FormattedException + public InputRow parse(Pair msg) { return decoder.decodeMessage(msg.lhs, msg.rhs.getChannelName(), msg.rhs.getText()); } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 6319a0d4633..1c238945489 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -25,8 +25,8 @@ import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; import com.google.inject.Inject; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.EmittingLogger; @@ -200,7 +200,7 @@ public class QueryResource } finally { resp.flushBuffer(); - Closeables.closeQuietly(out); + CloseQuietly.close(out); } } } diff --git a/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java b/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java index f0e7355a1c6..585b4b69639 100644 --- a/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java +++ b/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; import com.google.common.base.Throwables; import com.google.common.collect.Maps; -import com.google.common.io.Closeables; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -338,7 +338,7 @@ public class DruidClusterBridge log.makeAlert(e, "Exception becoming leader") .emit(); final LeaderLatch oldLatch = createNewLeaderLatch(); - Closeables.closeQuietly(oldLatch); + CloseQuietly.close(oldLatch); try { leaderLatch.get().start(); } diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index f57bd87ebd5..1223434577f 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -27,12 +27,12 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.common.io.Closeables; import com.google.inject.Inject; import com.metamx.common.IAE; import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; @@ -591,7 +591,7 @@ public class DruidCoordinator log.makeAlert(e, "Unable to become leader") .emit(); final LeaderLatch oldLatch = createNewLeaderLatch(); - Closeables.closeQuietly(oldLatch); + CloseQuietly.close(oldLatch); try { leaderLatch.get().start(); } diff --git a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java index 3aad005dd68..56773ee5a7d 100644 --- a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java +++ b/server/src/main/java/io/druid/server/initialization/PropertiesModule.java @@ -21,9 +21,9 @@ package io.druid.server.initialization; import com.google.common.base.Charsets; import com.google.common.base.Throwables; -import com.google.common.io.Closeables; import com.google.inject.Binder; import com.google.inject.Module; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import java.io.BufferedInputStream; @@ -80,7 +80,7 @@ public class PropertiesModule implements Module log.wtf(e, "This can only happen if the .exists() call lied. That's f'd up."); } finally { - Closeables.closeQuietly(stream); + CloseQuietly.close(stream); } binder.bind(Properties.class).toInstance(props); diff --git a/server/src/main/java/io/druid/server/log/FileRequestLogger.java b/server/src/main/java/io/druid/server/log/FileRequestLogger.java index ba450ce9c5f..51c8ea4dcf6 100644 --- a/server/src/main/java/io/druid/server/log/FileRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/FileRequestLogger.java @@ -21,8 +21,8 @@ package io.druid.server.log; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; -import com.google.common.io.Closeables; import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import io.druid.server.RequestLogLine; @@ -83,7 +83,7 @@ public class FileRequestLogger implements RequestLogger try { synchronized (lock) { - Closeables.closeQuietly(fileWriter); + CloseQuietly.close(fileWriter); fileWriter = new FileWriter(new File(baseDir, currentDay.toString()), true); } } @@ -105,7 +105,7 @@ public class FileRequestLogger implements RequestLogger public void stop() { synchronized (lock) { - Closeables.closeQuietly(fileWriter); + CloseQuietly.close(fileWriter); } } diff --git a/server/src/main/java/io/druid/server/sql/SQLRunner.java b/server/src/main/java/io/druid/server/sql/SQLRunner.java index dc6cb8dfa12..68a00bccc30 100644 --- a/server/src/main/java/io/druid/server/sql/SQLRunner.java +++ b/server/src/main/java/io/druid/server/sql/SQLRunner.java @@ -27,7 +27,7 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; @@ -222,6 +222,6 @@ public class SQLRunner } } - Closeables.closeQuietly(stdInput); + CloseQuietly.close(stdInput); } } diff --git a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java index 720ee14bef8..c2ed92d170c 100644 --- a/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/io/druid/client/client/BatchServerInventoryViewTest.java @@ -335,7 +335,7 @@ public class BatchServerInventoryViewTest private static void waitForSync(BatchServerInventoryView batchServerInventoryView, Set testSegments) throws Exception { - Stopwatch stopwatch = new Stopwatch().start(); + Stopwatch stopwatch = Stopwatch.createStarted(); while (Iterables.isEmpty(batchServerInventoryView.getInventory()) || Iterables.get(batchServerInventoryView.getInventory(), 0).getSegments().size() != testSegments.size()) { Thread.sleep(500); diff --git a/server/src/test/java/io/druid/curator/CuratorTestBase.java b/server/src/test/java/io/druid/curator/CuratorTestBase.java index a850d756ea3..a45f6e77643 100644 --- a/server/src/test/java/io/druid/curator/CuratorTestBase.java +++ b/server/src/test/java/io/druid/curator/CuratorTestBase.java @@ -19,7 +19,7 @@ package io.druid.curator; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.RetryOneTime; @@ -51,7 +51,7 @@ public class CuratorTestBase protected void tearDownServerAndCurator() { - Closeables.closeQuietly(curator); - Closeables.closeQuietly(server); + CloseQuietly.close(curator); + CloseQuietly.close(server); } } diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index e29dd187b4b..d1d6e1e6f67 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -137,7 +137,7 @@ public class RealtimeManagerTest { realtimeManager.start(); - Stopwatch stopwatch = new Stopwatch().start(); + Stopwatch stopwatch = Stopwatch.createStarted(); while (realtimeManager.getMetrics("test").processed() != 1) { Thread.sleep(100); if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index ddcb503af58..6b0a9e45f64 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -26,7 +26,6 @@ import com.google.common.io.Files; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.Granularity; import com.metamx.common.ISE; -import com.metamx.common.exception.FormattedException; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.FilteredServerView; import io.druid.client.ServerView; @@ -85,7 +84,7 @@ public class RealtimePlumberSchoolTest new InputRowParser() { @Override - public InputRow parse(Object input) throws FormattedException + public InputRow parse(Object input) { return null; } @@ -177,7 +176,7 @@ public class RealtimePlumberSchoolTest } ); - Stopwatch stopwatch = new Stopwatch().start(); + Stopwatch stopwatch = Stopwatch.createStarted(); while (!committed.booleanValue()) { Thread.sleep(100); if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index c31a58aa1fd..2f8f18a28ae 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -71,7 +71,7 @@ public class DruidCoordinatorBalancerProfiler public void bigProfiler() { - Stopwatch watch = new Stopwatch(); + Stopwatch watch = Stopwatch.createUnstarted(); int numSegments = 55000; int numServers = 50; EasyMock.expect(manager.getAllRules()).andReturn(ImmutableMap.>of("test", rules)).anyTimes(); @@ -184,7 +184,7 @@ public class DruidCoordinatorBalancerProfiler public void profileRun() { - Stopwatch watch = new Stopwatch(); + Stopwatch watch = Stopwatch.createUnstarted(); LoadQueuePeonTester fromPeon = new LoadQueuePeonTester(); LoadQueuePeonTester toPeon = new LoadQueuePeonTester(); diff --git a/services/src/main/java/io/druid/cli/convert/ConvertProperties.java b/services/src/main/java/io/druid/cli/convert/ConvertProperties.java index fdbf2c9f2f0..0175712ea37 100644 --- a/services/src/main/java/io/druid/cli/convert/ConvertProperties.java +++ b/services/src/main/java/io/druid/cli/convert/ConvertProperties.java @@ -22,7 +22,7 @@ package io.druid.cli.convert; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.airlift.command.Option; @@ -196,7 +196,7 @@ public class ConvertProperties implements Runnable } finally { if (out != null) { - Closeables.closeQuietly(out); + CloseQuietly.close(out); } } From 55f1e2e2a75ae8561b649548d014d5ec51600d5e Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 6 Jun 2014 14:19:55 -0700 Subject: [PATCH 003/270] add dependencies for validator error --- pom.xml | 19 +++++++++++++++---- server/pom.xml | 19 ++++++++++++++----- 2 files changed, 29 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 2116d7f617f..30d6fdd1989 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid @@ -203,17 +204,17 @@ com.google.inject guice - 4.0-beta + 4.0-beta4 com.google.inject.extensions guice-servlet - 4.0-beta + 4.0-beta4 com.google.inject.extensions guice-multibindings - 4.0-beta + 4.0-beta4 com.ibm.icu @@ -280,6 +281,16 @@ javax.inject 1 + + javax.el + javax.el-api + 2.2.4 + + + org.glassfish.web + javax.el + 2.2.4 + com.jamesmurty.utils java-xmlbuilder diff --git a/server/pom.xml b/server/pom.xml index 83635cdc29f..ee1974b000c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -57,6 +58,14 @@ javax.inject javax.inject + + javax.el + javax.el-api + + + org.glassfish.web + javax.el + com.amazonaws aws-java-sdk @@ -130,12 +139,12 @@ jetty-servlets - com.ircclouds.irc - irc-api + com.ircclouds.irc + irc-api - com.maxmind.geoip2 - geoip2 + com.maxmind.geoip2 + geoip2 From d3f2faef92c30a2131782f348678dffeb236c081 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 15:32:22 -0700 Subject: [PATCH 004/270] address code review --- pom.xml | 5 ----- server/pom.xml | 4 ---- 2 files changed, 9 deletions(-) diff --git a/pom.xml b/pom.xml index 30d6fdd1989..447fa65c084 100644 --- a/pom.xml +++ b/pom.xml @@ -281,11 +281,6 @@ javax.inject 1 - - javax.el - javax.el-api - 2.2.4 - org.glassfish.web javax.el diff --git a/server/pom.xml b/server/pom.xml index ee1974b000c..f01e920cb77 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -58,10 +58,6 @@ javax.inject javax.inject - - javax.el - javax.el-api - org.glassfish.web javax.el From eb5034ce7f0665f8eb1c9cf2e7430572c317bc73 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 15:33:24 -0700 Subject: [PATCH 005/270] update dep version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 447fa65c084..c513b9bd387 100644 --- a/pom.xml +++ b/pom.xml @@ -284,7 +284,7 @@ org.glassfish.web javax.el - 2.2.4 + 3.0.0 com.jamesmurty.utils From 5e48f3907ba198678e47173d75138affe98cc3cc Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 16:24:12 -0700 Subject: [PATCH 006/270] fix index task new schema defaults --- .../druid/indexing/common/task/IndexTask.java | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 32565f1d51a..a71d16c54f6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -35,7 +34,6 @@ import com.google.common.primitives.Ints; import com.metamx.common.ISE; import com.metamx.common.guava.Comparators; import com.metamx.common.logger.Logger; -import com.metamx.common.parsers.TimestampParser; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; @@ -46,15 +44,14 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.index.YeOldePlumberSchool; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.IngestionSpec; -import io.druid.segment.indexing.TuningConfig; import io.druid.segment.indexing.IOConfig; +import io.druid.segment.indexing.IngestionSpec; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.TuningConfig; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.plumber.Plumber; -import io.druid.segment.realtime.plumber.Sink; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; @@ -528,6 +525,9 @@ public class IndexTask extends AbstractFixedIntervalTask @JsonTypeName("index") public static class IndexTuningConfig implements TuningConfig { + private static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000; + private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 500000; + private final int targetPartitionSize; private final int rowFlushBoundary; @@ -537,8 +537,8 @@ public class IndexTask extends AbstractFixedIntervalTask @JsonProperty("rowFlushBoundary") int rowFlushBoundary ) { - this.targetPartitionSize = targetPartitionSize; - this.rowFlushBoundary = rowFlushBoundary; + this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; + this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary; } @JsonProperty @@ -553,12 +553,4 @@ public class IndexTask extends AbstractFixedIntervalTask return rowFlushBoundary; } } - - - public static void main(String[] args) - { - Function parser = TimestampParser.createTimestampParser("millis"); - parser.apply("1401266370985"); - - } } From ad7db018d5c87a0a4d5b7cbe4cc31dd0b2037526 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 16:28:47 -0700 Subject: [PATCH 007/270] add default tuning config for index task --- .../src/main/java/io/druid/indexing/common/task/IndexTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index a71d16c54f6..182a1569903 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -477,7 +477,7 @@ public class IndexTask extends AbstractFixedIntervalTask this.dataSchema = dataSchema; this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig; + this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0) : tuningConfig; } @Override From 6adc3011fad93d3b1aa7e313dc257873f62b04d0 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 18:25:10 -0700 Subject: [PATCH 008/270] fix post aggs with group by in #472 --- .../io/druid/query/groupby/GroupByQuery.java | 77 +++++++++++----- .../query/groupby/GroupByQueryEngine.java | 69 ++++++++------- .../groupby/GroupByQueryQueryToolChest.java | 19 ++-- .../timeseries/TimeseriesQueryEngine.java | 60 +++++++------ .../query/groupby/GroupByQueryRunnerTest.java | 87 +++++++++++++++++++ 5 files changed, 220 insertions(+), 92 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 0a23b5704b7..d9431849ea2 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -292,11 +292,25 @@ public class GroupByQuery extends BaseQuery private List orderByColumnSpecs = Lists.newArrayList(); private int limit = Integer.MAX_VALUE; - private Builder() + public Builder() { } - private Builder(Builder builder) + public Builder(GroupByQuery query) + { + dataSource = query.getDataSource(); + querySegmentSpec = query.getQuerySegmentSpec(); + limitSpec = query.getOrderBy(); + dimFilter = query.getDimFilter(); + granularity = query.getGranularity(); + dimensions = query.getDimensions(); + aggregatorSpecs = query.getAggregatorSpecs(); + postAggregatorSpecs = query.getPostAggregatorSpecs(); + havingSpec = query.getHavingSpec(); + context = query.getContext(); + } + + public Builder(Builder builder) { dataSource = builder.dataSource; querySegmentSpec = builder.querySegmentSpec; @@ -515,36 +529,57 @@ public class GroupByQuery extends BaseQuery public String toString() { return "GroupByQuery{" + - "limitSpec=" + limitSpec + - ", dimFilter=" + dimFilter + - ", granularity=" + granularity + - ", dimensions=" + dimensions + - ", aggregatorSpecs=" + aggregatorSpecs + - ", postAggregatorSpecs=" + postAggregatorSpecs + - ", orderByLimitFn=" + orderByLimitFn + - '}'; + "limitSpec=" + limitSpec + + ", dimFilter=" + dimFilter + + ", granularity=" + granularity + + ", dimensions=" + dimensions + + ", aggregatorSpecs=" + aggregatorSpecs + + ", postAggregatorSpecs=" + postAggregatorSpecs + + ", orderByLimitFn=" + orderByLimitFn + + '}'; } @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } GroupByQuery that = (GroupByQuery) o; - if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) + if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) { return false; - if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false; - if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false; - if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false; - if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) return false; - if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) return false; - if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null) + } + if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) { return false; - if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null) + } + if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) { return false; + } + if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) { + return false; + } + if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) { + return false; + } + if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) { + return false; + } + if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null) { + return false; + } + if (postAggregatorSpecs != null + ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) + : that.postAggregatorSpecs != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index d9e52014f86..8a42a4e10fe 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -42,7 +42,6 @@ import io.druid.data.input.Row; import io.druid.guice.annotations.Global; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; -import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.DimExtractionFn; import io.druid.segment.Cursor; @@ -104,41 +103,41 @@ public class GroupByQueryEngine return Sequences.concat( Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(@Nullable final Cursor cursor) - { - return new BaseSequence( - new BaseSequence.IteratorMaker() + Sequences.map( + cursors, + new Function>() + { + @Override + public Sequence apply(@Nullable final Cursor cursor) { - @Override - public RowIterator make() - { - return new RowIterator(query, cursor, bufferHolder.get(), config.get()); - } + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public RowIterator make() + { + return new RowIterator(query, cursor, bufferHolder.get(), config.get()); + } - @Override - public void cleanup(RowIterator iterFromMake) - { - Closeables.closeQuietly(iterFromMake); - } + @Override + public void cleanup(RowIterator iterFromMake) + { + Closeables.closeQuietly(iterFromMake); + } + } + ); } - ); + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + Closeables.closeQuietly(bufferHolder); + } } - } - ), - new Closeable() - { - @Override - public void close() throws IOException - { - Closeables.closeQuietly(bufferHolder); - } - } - ) + ) ); } @@ -414,9 +413,9 @@ public class GroupByQueryEngine position += increments[i]; } - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); - } + //for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + // theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); + //} return new MapBasedRow(timestamp, theEvent); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 0e00ceae46d..75190a66cda 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -43,7 +43,9 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.aggregation.PostAggregator; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; @@ -59,7 +61,10 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); + private static final Map NO_MERGE_CONTEXT = ImmutableMap.of( + GROUP_BY_MERGE_KEY, + "false" + ); private final Supplier configSupplier; private GroupByQueryEngine engine; // For running the outer query around a subquery @@ -92,7 +97,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { - Sequence result; // If there's a subquery, merge subquery results and then apply the aggregator @@ -101,12 +105,17 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); + final GroupByQuery.Builder builder = new GroupByQuery.Builder(subquery); + for (PostAggregator postAggregator : subquery.getPostAggregatorSpecs()) { + builder.addAggregator(new DoubleSumAggregatorFactory(postAggregator.getName(), postAggregator.getName())); + } IncrementalIndexStorageAdapter adapter - = new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult)); + = new IncrementalIndexStorageAdapter(makeIncrementalIndex(builder.build(), subqueryResult)); result = engine.process(query, adapter); } else { result = runner.run(query); @@ -128,7 +137,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest>() - { - private final List aggregatorSpecs = query.getAggregatorSpecs(); - private final List postAggregatorSpecs = query.getPostAggregatorSpecs(); + adapter, + query.getQuerySegmentSpec().getIntervals(), + Filters.convertDimensionFilters(query.getDimensionsFilter()), + query.getGranularity(), + new Function>() + { + private final List aggregatorSpecs = query.getAggregatorSpecs(); - @Override - public Result apply(Cursor cursor) - { - Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); + @Override + public Result apply(Cursor cursor) + { + Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); - while (!cursor.isDone()) { - for (Aggregator aggregator : aggregators) { - aggregator.aggregate(); - } - cursor.advance(); - } + while (!cursor.isDone()) { + for (Aggregator aggregator : aggregators) { + aggregator.aggregate(); + } + cursor.advance(); + } - TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); + TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); - for (Aggregator aggregator : aggregators) { - bob.addMetric(aggregator); - } + for (Aggregator aggregator : aggregators) { + bob.addMetric(aggregator); + } - Result retVal = bob.build(); + Result retVal = bob.build(); - // cleanup - for (Aggregator agg : aggregators) { - agg.close(); - } + // cleanup + for (Aggregator agg : aggregators) { + agg.close(); + } - return retVal; - } - } + return retVal; + } + } ); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 97e64a0ec0c..39cd21f81e1 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -44,6 +44,10 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.ExtractionDimensionSpec; @@ -1028,6 +1032,89 @@ public class GroupByQueryRunnerTest Assert.assertFalse(results.iterator().hasNext()); } + @Test + public void testSubqueryWithPostAggregators() + { + + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }")) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx_subagg", "index") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx_subpostagg", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"), + new ConstantPostAggregator("thousand", 1000, 1000) + ) + ) + + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_agg", "idx"), + new ConstantPostAggregator("ten_thousand", 10000, 10000) + ) + ) + + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + private Iterable runQuery(GroupByQuery query) { From 32b65d9c5b4fd07e8a96ed8663e44369ef96ccce Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 18:26:37 -0700 Subject: [PATCH 009/270] remove commented out code --- .../main/java/io/druid/query/groupby/GroupByQueryEngine.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index 8a42a4e10fe..e158fe078be 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -413,10 +413,6 @@ public class GroupByQueryEngine position += increments[i]; } - //for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - // theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); - //} - return new MapBasedRow(timestamp, theEvent); } } From 187b2beb47ce960f6cb01712a760218267168e68 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 22:27:33 -0700 Subject: [PATCH 010/270] cleaner way of implementing things --- .../query/GroupByParallelQueryRunner.java | 1 - .../query/aggregation/AggregatorFactory.java | 19 ++++- .../aggregation/CountAggregatorFactory.java | 19 ++++- .../DoubleSumAggregatorFactory.java | 22 ++++-- .../HistogramAggregatorFactory.java | 32 +++++++-- .../JavaScriptAggregatorFactory.java | 16 +++++ .../aggregation/LongSumAggregatorFactory.java | 26 +++++-- .../aggregation/MaxAggregatorFactory.java | 6 ++ .../aggregation/MinAggregatorFactory.java | 6 ++ .../ToLowerCaseAggregatorFactory.java | 6 ++ .../CardinalityAggregatorFactory.java | 17 +++++ .../HyperUniquesAggregatorFactory.java | 6 ++ .../query/groupby/GroupByQueryEngine.java | 71 ++++++++++--------- .../query/groupby/GroupByQueryHelper.java | 6 +- .../groupby/GroupByQueryQueryToolChest.java | 30 +++++--- .../query/groupby/GroupByQueryRunnerTest.java | 7 +- 16 files changed, 216 insertions(+), 74 deletions(-) diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 51c663c6a2e..65925689125 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -77,7 +77,6 @@ public class GroupByParallelQueryRunner implements QueryRunner @Override public Sequence run(final Query queryParam) { - final GroupByQuery query = (GroupByQuery) queryParam; final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index 87077e45b31..e236e683d08 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -26,9 +26,9 @@ import java.util.List; /** * Processing related interface - * + *

* An AggregatorFactory is an object that knows how to generate an Aggregator using a ColumnSelectorFactory. - * + *

* This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects * without making any assumptions about how they are pulling values out of the base data. That is, the data is * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how @@ -37,7 +37,9 @@ import java.util.List; public interface AggregatorFactory { public Aggregator factorize(ColumnSelectorFactory metricFactory); + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory); + public Comparator getComparator(); /** @@ -48,6 +50,7 @@ public interface AggregatorFactory * * @param lhs The left hand side of the combine * @param rhs The right hand side of the combine + * * @return an object representing the combination of lhs and rhs, this can be a new object or a mutation of the inputs */ public Object combine(Object lhs, Object rhs); @@ -61,11 +64,19 @@ public interface AggregatorFactory */ public AggregatorFactory getCombiningFactory(); + /** + * Gets a list of aggregator factories using the actual column names of the data + * + * @return Base AggregatorFactories for different fields of this AggregatorFactory + */ + public List getBaseFactories(); + /** * A method that knows how to "deserialize" the object from whatever form it might have been put into * in order to transfer via JSON. * * @param object the object to deserialize + * * @return the deserialized object */ public Object deserialize(Object object); @@ -75,13 +86,17 @@ public interface AggregatorFactory * intermediate format than their final resultant output. * * @param object the object to be finalized + * * @return the finalized value that should be returned for the initial query */ public Object finalizeComputation(Object object); public String getName(); + public List requiredFields(); + public byte[] getCacheKey(); + public String getTypeName(); /** diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index e47999e8719..1d1e2e149d7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; import io.druid.segment.ColumnSelectorFactory; +import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -76,6 +77,12 @@ public class CountAggregatorFactory implements AggregatorFactory return new LongSumAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new CountAggregatorFactory(name)); + } + @Override public Object deserialize(Object object) { @@ -136,12 +143,18 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } CountAggregatorFactory that = (CountAggregatorFactory) o; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index c7f3eba75f4..7e395c3c3ca 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -85,6 +85,12 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory return new DoubleSumAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new DoubleSumAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { @@ -158,13 +164,21 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } DoubleSumAggregatorFactory that = (DoubleSumAggregatorFactory) o; - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java index 060d40d2798..76809eb39a4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -56,7 +56,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory this.name = name; this.fieldName = fieldName; - this.breaksList = (breaksList == null) ? Lists.newArrayList() :breaksList; + this.breaksList = (breaksList == null) ? Lists.newArrayList() : breaksList; this.breaks = new float[this.breaksList.size()]; for (int i = 0; i < this.breaksList.size(); ++i) { this.breaks[i] = this.breaksList.get(i); @@ -100,6 +100,12 @@ public class HistogramAggregatorFactory implements AggregatorFactory return new HistogramAggregatorFactory(name, name, breaksList); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new HistogramAggregatorFactory(fieldName, fieldName, breaksList)); + } + @Override public Object deserialize(Object object) { @@ -183,15 +189,27 @@ public class HistogramAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } HistogramAggregatorFactory that = (HistogramAggregatorFactory) o; - if (!Arrays.equals(breaks, that.breaks)) return false; - if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) return false; - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (!Arrays.equals(breaks, that.breaks)) { + return false; + } + if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) { + return false; + } + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 0c1e4d2ad9d..ac85b75b8dc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -140,6 +140,22 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory return new JavaScriptAggregatorFactory(name, Lists.newArrayList(name), fnCombine, fnReset, fnCombine); } + @Override + public List getBaseFactories() + { + return Lists.transform( + fieldNames, + new com.google.common.base.Function() + { + @Override + public AggregatorFactory apply(String input) + { + return new JavaScriptAggregatorFactory(name, Arrays.asList(input), fnAggregate, fnReset, fnCombine); + } + } + ); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 50ef5130756..0eaae184e9f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -31,11 +31,11 @@ import java.util.Comparator; import java.util.List; /** -*/ + */ public class LongSumAggregatorFactory implements AggregatorFactory { private static final byte CACHE_TYPE_ID = 0x1; - + private final String fieldName; private final String name; @@ -85,6 +85,12 @@ public class LongSumAggregatorFactory implements AggregatorFactory return new LongSumAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { @@ -154,13 +160,21 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } LongSumAggregatorFactory that = (LongSumAggregatorFactory) o; - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index b731c4319e7..78abafa3d76 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -82,6 +82,12 @@ public class MaxAggregatorFactory implements AggregatorFactory return new MaxAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new MaxAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index d3956c94b52..6c373eded4b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -82,6 +82,12 @@ public class MinAggregatorFactory implements AggregatorFactory return new MinAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new MinAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java index 6c559ba8ec6..b649f1fa5b3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java @@ -65,6 +65,12 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory return baseAggregatorFactory.getCombiningFactory(); } + @Override + public List getBaseFactories() + { + return baseAggregatorFactory.getBaseFactories(); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 2978dba9d9f..eacfede07ea 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -38,6 +38,7 @@ import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -145,6 +146,22 @@ public class CardinalityAggregatorFactory implements AggregatorFactory return new CardinalityAggregatorFactory(name, fieldNames, byRow); } + @Override + public List getBaseFactories() + { + return Lists.transform( + fieldNames, + new Function() + { + @Override + public AggregatorFactory apply(String input) + { + return new CardinalityAggregatorFactory(name, Arrays.asList(input), byRow); + } + } + ); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index fd4e36fb781..2193be3419e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -131,6 +131,12 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory return new HyperUniquesAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new HyperUniquesAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index e158fe078be..d9e52014f86 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -42,6 +42,7 @@ import io.druid.data.input.Row; import io.druid.guice.annotations.Global; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.DimExtractionFn; import io.druid.segment.Cursor; @@ -103,41 +104,41 @@ public class GroupByQueryEngine return Sequences.concat( Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(@Nullable final Cursor cursor) - { - return new BaseSequence( - new BaseSequence.IteratorMaker() - { - @Override - public RowIterator make() - { - return new RowIterator(query, cursor, bufferHolder.get(), config.get()); - } - - @Override - public void cleanup(RowIterator iterFromMake) - { - Closeables.closeQuietly(iterFromMake); - } - } - ); - } - } - ), - new Closeable() + Sequences.map( + cursors, + new Function>() + { + @Override + public Sequence apply(@Nullable final Cursor cursor) { - @Override - public void close() throws IOException - { - Closeables.closeQuietly(bufferHolder); - } + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public RowIterator make() + { + return new RowIterator(query, cursor, bufferHolder.get(), config.get()); + } + + @Override + public void cleanup(RowIterator iterFromMake) + { + Closeables.closeQuietly(iterFromMake); + } + } + ); } - ) + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + Closeables.closeQuietly(bufferHolder); + } + } + ) ); } @@ -413,6 +414,10 @@ public class GroupByQueryEngine position += increments[i]; } + for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); + } + return new MapBasedRow(timestamp, theEvent); } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 00298f18ba0..44fb82b7c6e 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -53,7 +53,7 @@ public class GroupByQueryHelper new Function() { @Override - public AggregatorFactory apply(@Nullable AggregatorFactory input) + public AggregatorFactory apply(AggregatorFactory input) { return input.getCombiningFactory(); } @@ -64,7 +64,7 @@ public class GroupByQueryHelper new Function() { @Override - public String apply(@Nullable DimensionSpec input) + public String apply(DimensionSpec input) { return input.getOutputName(); } @@ -90,7 +90,7 @@ public class GroupByQueryHelper return accumulated; } }; - return new Pair>(index, accumulator); + return new Pair<>(index, accumulator); } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 75190a66cda..681871cfd93 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -24,6 +24,7 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.Pair; @@ -51,6 +52,8 @@ import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; +import javax.annotation.Nullable; +import java.util.List; import java.util.Map; /** @@ -100,7 +103,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest result; // If there's a subquery, merge subquery results and then apply the aggregator - DataSource dataSource = query.getDataSource(); + final DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { GroupByQuery subquery; try { @@ -109,19 +112,24 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); - final GroupByQuery.Builder builder = new GroupByQuery.Builder(subquery); - for (PostAggregator postAggregator : subquery.getPostAggregatorSpecs()) { - builder.addAggregator(new DoubleSumAggregatorFactory(postAggregator.getName(), postAggregator.getName())); + final Sequence subqueryResult = mergeGroupByResults(subquery, runner); + final List aggs = Lists.newArrayList(); + for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { + aggs.addAll(aggregatorFactory.getBaseFactories()); } - IncrementalIndexStorageAdapter adapter - = new IncrementalIndexStorageAdapter(makeIncrementalIndex(builder.build(), subqueryResult)); - result = engine.process(query, adapter); + final GroupByQuery innerQuery = new GroupByQuery.Builder(query).setAggregatorSpecs(aggs) + .setInterval(subquery.getIntervals()) + .setPostAggregatorSpecs(Lists.newArrayList()) + .build(); + + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter( + makeIncrementalIndex(innerQuery, subqueryResult) + ); + return engine.process(query, adapter); } else { result = runner.run(query); + return postAggregate(query, makeIncrementalIndex(query, result)); } - - return postAggregate(query, makeIncrementalIndex(query, result)); } @@ -161,7 +169,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { - return new ConcatSequence(seqOfSequences); + return new ConcatSequence<>(seqOfSequences); } @Override diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 39cd21f81e1..6b44645ac81 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -1035,8 +1035,7 @@ public class GroupByQueryRunnerTest @Test public void testSubqueryWithPostAggregators() { - - GroupByQuery subquery = GroupByQuery + final GroupByQuery subquery = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -1062,7 +1061,7 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - GroupByQuery query = GroupByQuery + final GroupByQuery query = GroupByQuery .builder() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -1076,7 +1075,7 @@ public class GroupByQueryRunnerTest .setPostAggregatorSpecs( Arrays.asList( new ArithmeticPostAggregator( - "idx", "+", Arrays.asList( + "idx", "+", Arrays.asList( new FieldAccessPostAggregator("the_idx_agg", "idx"), new ConstantPostAggregator("ten_thousand", 10000, 10000) ) From 5955ecf032d678aa27095bd5193e24ed865b32b4 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 22:31:30 -0700 Subject: [PATCH 011/270] fix bugs wtih js and card aggs --- .../io/druid/query/aggregation/JavaScriptAggregatorFactory.java | 2 +- .../aggregation/cardinality/CardinalityAggregatorFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index ac85b75b8dc..90175c54b88 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -150,7 +150,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new JavaScriptAggregatorFactory(name, Arrays.asList(input), fnAggregate, fnReset, fnCombine); + return new JavaScriptAggregatorFactory(input, Arrays.asList(input), fnAggregate, fnReset, fnCombine); } } ); diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index eacfede07ea..4c79b076533 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -156,7 +156,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new CardinalityAggregatorFactory(name, Arrays.asList(input), byRow); + return new CardinalityAggregatorFactory(input, Arrays.asList(input), byRow); } } ); From 21908b507784a02576fd64932dc6ac92ec5d7b4c Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 10 Jun 2014 17:42:30 +0530 Subject: [PATCH 012/270] hdfs data segment killer implement hdfs data segment killer --- .../storage/hdfs/HdfsDataSegmentKiller.java | 76 +++++++++++++++++++ .../storage/hdfs/HdfsStorageDruidModule.java | 1 + 2 files changed, 77 insertions(+) create mode 100644 hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java new file mode 100644 index 00000000000..4d4ddf70766 --- /dev/null +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java @@ -0,0 +1,76 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.hdfs; + +import com.google.inject.Inject; +import io.druid.segment.loading.DataSegmentKiller; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.IOException; + +public class HdfsDataSegmentKiller implements DataSegmentKiller +{ + private final Configuration config; + + @Inject + public HdfsDataSegmentKiller(final Configuration config) + { + this.config = config; + } + + @Override + public void kill(DataSegment segment) throws SegmentLoadingException + { + final Path path = getPath(segment); + final FileSystem fs = checkPathAndGetFilesystem(path); + try { + fs.delete(path, true); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to kill segment"); + } + } + + private Path getPath(DataSegment segment) + { + return new Path(String.valueOf(segment.getLoadSpec().get("path"))); + } + + private FileSystem checkPathAndGetFilesystem(Path path) throws SegmentLoadingException + { + FileSystem fs; + try { + fs = path.getFileSystem(config); + + if (!fs.exists(path)) { + throw new SegmentLoadingException("Path[%s] doesn't exist.", path); + } + + return fs; + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Problems interacting with filesystem[%s].", path); + } + } +} diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java index e2fb1475742..0ebb477a593 100644 --- a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java @@ -55,6 +55,7 @@ public class HdfsStorageDruidModule implements DruidModule { Binders.dataSegmentPullerBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPuller.class).in(LazySingleton.class); Binders.dataSegmentPusherBinder(binder).addBinding("hdfs").to(HdfsDataSegmentPusher.class).in(LazySingleton.class); + Binders.dataSegmentKillerBinder(binder).addBinding("hdfs").to(HdfsDataSegmentKiller.class).in(LazySingleton.class); final Configuration conf = new Configuration(); if (props != null) { From f3e60795d0ac9e8b736a2b6a465d1ca76d0c2e65 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 10 Jun 2014 19:25:08 +0530 Subject: [PATCH 013/270] delete the parent directory --- .../java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java index 4d4ddf70766..bc62fbaa46c 100644 --- a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsDataSegmentKiller.java @@ -45,7 +45,12 @@ public class HdfsDataSegmentKiller implements DataSegmentKiller final Path path = getPath(segment); final FileSystem fs = checkPathAndGetFilesystem(path); try { - fs.delete(path, true); + if (path.getName().endsWith(".zip")) { + // delete the parent directory containing the zip file and the descriptor + fs.delete(path.getParent(), true); + } else { + throw new SegmentLoadingException("Unknown file type[%s]", path); + } } catch (IOException e) { throw new SegmentLoadingException(e, "Unable to kill segment"); From d6f38827db453f31dd1e082e31dbcaade23b085a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 8 Apr 2014 17:13:22 -0700 Subject: [PATCH 014/270] initial query cancellation commit --- .../DruidDefaultSerializersModule.java | 1 + .../query/ChainedExecutionQueryRunner.java | 131 ++++++++-------- .../query/QueryInterruptedException.java | 37 +++++ .../java/io/druid/query/QueryWatcher.java | 27 ++++ .../search/SearchQueryRunnerFactory.java | 8 +- .../select/SelectQueryRunnerFactory.java | 18 ++- .../TimeBoundaryQueryRunnerFactory.java | 11 +- .../TimeseriesQueryRunnerFactory.java | 22 ++- .../query/topn/TopNQueryRunnerFactory.java | 8 +- .../ChainedExecutionQueryRunnerTest.java | 148 ++++++++++++++++++ .../java/io/druid/query/TestQueryRunners.java | 30 +++- .../query/search/SearchQueryRunnerTest.java | 12 +- .../TimeBoundaryQueryRunnerTest.java | 12 +- .../druid/query/topn/TopNQueryRunnerTest.java | 23 ++- .../druid/guice/QueryRunnerFactoryModule.java | 8 + .../java/io/druid/server/QueryManager.java | 48 ++++++ .../java/io/druid/server/QueryResource.java | 21 ++- 17 files changed, 481 insertions(+), 84 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/QueryInterruptedException.java create mode 100644 processing/src/main/java/io/druid/query/QueryWatcher.java create mode 100644 processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java create mode 100644 server/src/main/java/io/druid/server/QueryManager.java diff --git a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java index 38c3bc135ef..068bfecd963 100644 --- a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java @@ -32,6 +32,7 @@ import com.google.common.base.Throwables; import com.metamx.common.Granularity; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Yielder; import org.joda.time.DateTimeZone; import java.io.IOException; diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 31eac12702e..6c12d37669b 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -25,6 +25,10 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.MergeIterable; @@ -35,11 +39,8 @@ import com.metamx.common.logger.Logger; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; /** * A QueryRunner that combines a list of other QueryRunners and executes them in parallel on an executor. @@ -59,27 +60,33 @@ public class ChainedExecutionQueryRunner implements QueryRunner private static final Logger log = new Logger(ChainedExecutionQueryRunner.class); private final Iterable> queryables; - private final ExecutorService exec; + private final ListeningExecutorService exec; private final Ordering ordering; + private final QueryWatcher queryWatcher; public ChainedExecutionQueryRunner( ExecutorService exec, Ordering ordering, + QueryWatcher queryWatcher, QueryRunner... queryables ) { - this(exec, ordering, Arrays.asList(queryables)); + this(exec, ordering, queryWatcher, Arrays.asList(queryables)); } public ChainedExecutionQueryRunner( ExecutorService exec, Ordering ordering, + QueryWatcher queryWatcher, Iterable> queryables ) { - this.exec = exec; + // listeningDecorator will leave PrioritizedExecutorService unchanged, + // since it already implements ListeningExecutorService + this.exec = MoreExecutors.listeningDecorator(exec); this.ordering = ordering; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); + this.queryWatcher = queryWatcher; } @Override @@ -94,71 +101,67 @@ public class ChainedExecutionQueryRunner implements QueryRunner public Iterator make() { // Make it a List<> to materialize all of the values (so that it will submit everything to the executor) - List>> futures = Lists.newArrayList( - Iterables.transform( - queryables, - new Function, Future>>() - { - @Override - public Future> apply(final QueryRunner input) - { - return exec.submit( - new AbstractPrioritizedCallable>(priority) - { - @Override - public List call() throws Exception - { - try { - if (input == null) { - throw new ISE("Input is null?! How is this possible?!"); - } + ListenableFuture>> futures = Futures.allAsList( + Lists.newArrayList( + Iterables.transform( + queryables, + new Function, ListenableFuture>>() + { + @Override + public ListenableFuture> apply(final QueryRunner input) + { + return exec.submit( + new AbstractPrioritizedCallable>(priority) + { + @Override + public Iterable call() throws Exception + { + try { + if (input == null) { + throw new ISE("Input is null?! How is this possible?!"); + } - Sequence result = input.run(query); - if (result == null) { - throw new ISE("Got a null result! Segments are missing!"); - } + Sequence result = input.run(query); + if (result == null) { + throw new ISE("Got a null result! Segments are missing!"); + } - List retVal = Sequences.toList(result, Lists.newArrayList()); - if (retVal == null) { - throw new ISE("Got a null list of results! WTF?!"); - } + List retVal = Sequences.toList(result, Lists.newArrayList()); + if (retVal == null) { + throw new ISE("Got a null list of results! WTF?!"); + } - return retVal; + return retVal; + } + catch (Exception e) { + log.error(e, "Exception with one of the sequences!"); + throw Throwables.propagate(e); + } + } } - catch (Exception e) { - log.error(e, "Exception with one of the sequences!"); - throw Throwables.propagate(e); - } - } - } - ); - } - } + ); + } + } + ) ) ); - return new MergeIterable( - ordering.nullsFirst(), - Iterables.transform( - futures, - new Function>, Iterable>() - { - @Override - public Iterable apply(Future> input) - { - try { - return input.get(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - catch (ExecutionException e) { - throw new RuntimeException(e); - } - } - } - ) - ).iterator(); + queryWatcher.registerQuery(query, futures); + + try { + return new MergeIterable<>( + ordering.nullsFirst(), + futures.get() + ).iterator(); + } + catch (InterruptedException e) { + log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); + futures.cancel(true); + throw new QueryInterruptedException(e); + } + catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } } @Override diff --git a/processing/src/main/java/io/druid/query/QueryInterruptedException.java b/processing/src/main/java/io/druid/query/QueryInterruptedException.java new file mode 100644 index 00000000000..7b889ef5f2d --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryInterruptedException.java @@ -0,0 +1,37 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.query; + +public class QueryInterruptedException extends RuntimeException +{ + public QueryInterruptedException() { + super(); + } + + public QueryInterruptedException(String message) + { + super(message); + } + + public QueryInterruptedException(Throwable cause) + { + super(cause); + } +} diff --git a/processing/src/main/java/io/druid/query/QueryWatcher.java b/processing/src/main/java/io/druid/query/QueryWatcher.java new file mode 100644 index 00000000000..0a76a54f23a --- /dev/null +++ b/processing/src/main/java/io/druid/query/QueryWatcher.java @@ -0,0 +1,27 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.query; + +import com.google.common.util.concurrent.ListenableFuture; + +public interface QueryWatcher +{ + public void registerQuery(Query query, ListenableFuture future); +} diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/search/SearchQueryRunnerFactory.java index 0c0fdab980d..2cd868e45d8 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunnerFactory.java @@ -24,6 +24,7 @@ import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.search.search.SearchQuery; import io.druid.segment.Segment; @@ -35,13 +36,16 @@ import java.util.concurrent.ExecutorService; public class SearchQueryRunnerFactory implements QueryRunnerFactory, SearchQuery> { private final SearchQueryQueryToolChest toolChest; + private final QueryWatcher queryWatcher; @Inject public SearchQueryRunnerFactory( - SearchQueryQueryToolChest toolChest + SearchQueryQueryToolChest toolChest, + QueryWatcher queryWatcher ) { this.toolChest = toolChest; + this.queryWatcher = queryWatcher; } @Override @@ -56,7 +60,7 @@ public class SearchQueryRunnerFactory implements QueryRunnerFactory>( - queryExecutor, toolChest.getOrdering(), queryRunners + queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners ); } diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java index 6e995b15f44..a1fa77dabd5 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java @@ -20,6 +20,7 @@ package io.druid.query.select; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; @@ -29,6 +30,7 @@ import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.segment.Segment; @@ -43,21 +45,31 @@ public class SelectQueryRunnerFactory { return new SelectQueryRunnerFactory( new SelectQueryQueryToolChest(new QueryConfig(), jsonMapper), - new SelectQueryEngine() + new SelectQueryEngine(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + } + } ); } private final SelectQueryQueryToolChest toolChest; private final SelectQueryEngine engine; + private final QueryWatcher queryWatcher; @Inject public SelectQueryRunnerFactory( SelectQueryQueryToolChest toolChest, - SelectQueryEngine engine + SelectQueryEngine engine, + QueryWatcher queryWatcher ) { this.toolChest = toolChest; this.engine = engine; + this.queryWatcher = queryWatcher; } @Override @@ -72,7 +84,7 @@ public class SelectQueryRunnerFactory ) { return new ChainedExecutionQueryRunner>( - queryExecutor, toolChest.getOrdering(), queryRunners + queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners ); } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 16e9ae832fa..1f78429ead3 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -19,6 +19,7 @@ package io.druid.query.timeboundary; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.BaseSequence; import com.metamx.common.guava.Sequence; @@ -27,6 +28,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; @@ -40,6 +42,13 @@ public class TimeBoundaryQueryRunnerFactory implements QueryRunnerFactory, TimeBoundaryQuery> { private static final TimeBoundaryQueryQueryToolChest toolChest = new TimeBoundaryQueryQueryToolChest(); + private final QueryWatcher queryWatcher; + + @Inject + public TimeBoundaryQueryRunnerFactory(QueryWatcher queryWatcher) + { + this.queryWatcher = queryWatcher; + } @Override public QueryRunner> createRunner(final Segment segment) @@ -53,7 +62,7 @@ public class TimeBoundaryQueryRunnerFactory ) { return new ChainedExecutionQueryRunner>( - queryExecutor, toolChest.getOrdering(), queryRunners + queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners ); } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 740b309e37e..726bc20bb43 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -19,6 +19,7 @@ package io.druid.query.timeseries; +import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; @@ -28,6 +29,7 @@ import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; @@ -39,25 +41,39 @@ import java.util.concurrent.ExecutorService; public class TimeseriesQueryRunnerFactory implements QueryRunnerFactory, TimeseriesQuery> { + /** + * Use only for testing + * @return + */ public static TimeseriesQueryRunnerFactory create() { return new TimeseriesQueryRunnerFactory( new TimeseriesQueryQueryToolChest(new QueryConfig()), - new TimeseriesQueryEngine() + new TimeseriesQueryEngine(), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + } + } ); } private final TimeseriesQueryQueryToolChest toolChest; private final TimeseriesQueryEngine engine; + private final QueryWatcher queryWatcher; @Inject public TimeseriesQueryRunnerFactory( TimeseriesQueryQueryToolChest toolChest, - TimeseriesQueryEngine engine + TimeseriesQueryEngine engine, + QueryWatcher queryWatcher ) { this.toolChest = toolChest; this.engine = engine; + this.queryWatcher = queryWatcher; } @Override @@ -72,7 +88,7 @@ public class TimeseriesQueryRunnerFactory ) { return new ChainedExecutionQueryRunner>( - queryExecutor, toolChest.getOrdering(), queryRunners + queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners ); } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java index 044e6a64eb2..524f9ace6a5 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryRunnerFactory.java @@ -30,6 +30,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.segment.Segment; @@ -43,15 +44,18 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory computationBufferPool; private final TopNQueryQueryToolChest toolchest; + private final QueryWatcher queryWatcher; @Inject public TopNQueryRunnerFactory( @Global StupidPool computationBufferPool, - TopNQueryQueryToolChest toolchest + TopNQueryQueryToolChest toolchest, + QueryWatcher queryWatcher ) { this.computationBufferPool = computationBufferPool; this.toolchest = toolchest; + this.queryWatcher = queryWatcher; } @Override @@ -79,7 +83,7 @@ public class TopNQueryRunnerFactory implements QueryRunnerFactory>( - queryExecutor, toolchest.getOrdering(), queryRunners + queryExecutor, toolchest.getOrdering(), queryWatcher, queryRunners ); } diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java new file mode 100644 index 00000000000..445bea9cf53 --- /dev/null +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -0,0 +1,148 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.query; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.util.concurrent.ListenableFuture; +import com.metamx.common.concurrent.ExecutorServiceConfig; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.lifecycle.Lifecycle; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import org.junit.Ignore; +import org.junit.Test; + +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +public class ChainedExecutionQueryRunnerTest +{ + @Test @Ignore + public void testQueryCancellation() throws Exception + { + ExecutorService exec = PrioritizedExecutorService.create( + new Lifecycle(), new ExecutorServiceConfig() + { + @Override + public String getFormatString() + { + return "test"; + } + + @Override + public int getNumThreads() + { + return 2; + } + } + ); + + final CountDownLatch queriesStarted = new CountDownLatch(2); + final CountDownLatch queryIsRegistered = new CountDownLatch(1); + + final Map queries = Maps.newHashMap(); + QueryWatcher watcher = new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + queries.put(query, future); + queryIsRegistered.countDown(); + } + }; + + ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>( + exec, + Ordering.natural(), + watcher, + Lists.>newArrayList( + new DyingQueryRunner(1, queriesStarted), + new DyingQueryRunner(2, queriesStarted), + new DyingQueryRunner(3, queriesStarted) + ) + ); + + final Sequence seq = chainedRunner.run( + Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals("2014/2015") + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) + .build() + ); + + Future f = Executors.newFixedThreadPool(1).submit( + new Runnable() + { + @Override + public void run() + { + Sequences.toList(seq, Lists.newArrayList()); + } + } + ); + + // wait for query to register + queryIsRegistered.await(); + queriesStarted.await(); + + // cancel the query + queries.values().iterator().next().cancel(true); + f.get(); + } + + private static class DyingQueryRunner implements QueryRunner + { + private final int id; + private final CountDownLatch latch; + + public DyingQueryRunner(int id, CountDownLatch latch) { + this.id = id; + this.latch = latch; + } + + @Override + public Sequence run(Query query) + { + latch.countDown(); + + int i = 0; + while (i >= 0) { + if(Thread.interrupted()) { + throw new QueryInterruptedException("I got killed"); + } + + // do a lot of work + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new QueryInterruptedException("I got killed"); + } + ++i; + } + return Sequences.simple(Lists.newArrayList(i)); + } + } +} diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java index c4767c1c6f9..a858b5e0cdf 100644 --- a/processing/src/test/java/io/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -1,6 +1,7 @@ package io.druid.query; import com.google.common.base.Supplier; +import com.google.common.util.concurrent.ListenableFuture; import io.druid.collections.StupidPool; import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.SearchQueryRunnerFactory; @@ -40,7 +41,14 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = new TopNQueryRunnerFactory(pool, new TopNQueryQueryToolChest(topNConfig)); + QueryRunnerFactory factory = new TopNQueryRunnerFactory(pool, new TopNQueryQueryToolChest(topNConfig), new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() @@ -62,7 +70,14 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig())); + QueryRunnerFactory factory = new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig()), new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() @@ -73,11 +88,18 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = new TimeBoundaryQueryRunnerFactory(); + QueryRunnerFactory factory = new TimeBoundaryQueryRunnerFactory(new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() ); } -} \ No newline at end of file +} diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index 2e5623b4365..0740333eed5 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -23,10 +23,13 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.filter.DimFilter; import io.druid.query.search.search.FragmentSearchQuerySpec; @@ -56,7 +59,14 @@ public class SearchQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig())) + new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig()), new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }) ); } diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 9b92826ac34..de5ac1281b2 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -20,10 +20,13 @@ package io.druid.query.timeboundary; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import org.joda.time.DateTime; import org.junit.Assert; @@ -43,7 +46,14 @@ public class TimeBoundaryQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - new TimeBoundaryQueryRunnerFactory() + new TimeBoundaryQueryRunnerFactory(new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }) ); } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 22b750faf00..39af4459794 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -23,12 +23,15 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.query.BySegmentResultValueClass; import io.druid.query.Druids; +import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; @@ -68,7 +71,15 @@ public class TopNQueryRunnerTest QueryRunnerTestHelper.makeQueryRunners( new TopNQueryRunnerFactory( TestQueryRunners.getPool(), - new TopNQueryQueryToolChest(new TopNQueryConfig()) + new TopNQueryQueryToolChest(new TopNQueryConfig()), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + } ) ) ); @@ -85,7 +96,15 @@ public class TopNQueryRunnerTest } } ), - new TopNQueryQueryToolChest(new TopNQueryConfig()) + new TopNQueryQueryToolChest(new TopNQueryConfig()), + new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + } ) ) ); diff --git a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java index 6f4dc80b059..6c1bb62cafe 100644 --- a/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/io/druid/guice/QueryRunnerFactoryModule.java @@ -24,6 +24,7 @@ import com.google.inject.Binder; import com.google.inject.multibindings.MapBinder; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryWatcher; import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQueryEngine; import io.druid.query.groupby.GroupByQueryRunnerFactory; @@ -39,6 +40,7 @@ import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryRunnerFactory; +import io.druid.server.QueryManager; import java.util.Map; @@ -62,6 +64,12 @@ public class QueryRunnerFactoryModule extends QueryToolChestModule { super.configure(binder); + binder.bind(QueryWatcher.class) + .to(QueryManager.class) + .in(LazySingleton.class); + binder.bind(QueryManager.class) + .in(LazySingleton.class); + final MapBinder, QueryRunnerFactory> queryFactoryBinder = DruidBinders.queryRunnerFactoryBinder( binder ); diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java new file mode 100644 index 00000000000..5910aeda535 --- /dev/null +++ b/server/src/main/java/io/druid/server/QueryManager.java @@ -0,0 +1,48 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.server; + +import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ListenableFuture; +import io.druid.query.Query; +import io.druid.query.QueryWatcher; + +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Future; + +public class QueryManager implements QueryWatcher +{ + final ConcurrentMap queries; + + public QueryManager() { + this.queries = Maps.newConcurrentMap(); + } + + public void cancelQuery(String id) { + Future future = queries.get(id); + if(future != null) { + future.cancel(true); + } + } + public void registerQuery(Query query, ListenableFuture future) + { + queries.put(query.getId(), future); + } +} diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 6319a0d4633..e710a6c97c9 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -37,16 +37,20 @@ import io.druid.guice.annotations.Smile; import io.druid.query.DataSourceUtil; import io.druid.query.Query; import io.druid.query.QuerySegmentWalker; +import io.druid.query.QueryWatcher; import io.druid.server.log.RequestLogger; import org.joda.time.DateTime; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.DELETE; import javax.ws.rs.POST; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; @@ -66,6 +70,7 @@ public class QueryResource private final QuerySegmentWalker texasRanger; private final ServiceEmitter emitter; private final RequestLogger requestLogger; + private final QueryManager queryManager; @Inject public QueryResource( @@ -73,7 +78,8 @@ public class QueryResource @Smile ObjectMapper smileMapper, QuerySegmentWalker texasRanger, ServiceEmitter emitter, - RequestLogger requestLogger + RequestLogger requestLogger, + QueryManager queryManager ) { this.jsonMapper = jsonMapper; @@ -81,6 +87,16 @@ public class QueryResource this.texasRanger = texasRanger; this.emitter = emitter; this.requestLogger = requestLogger; + this.queryManager = queryManager; + } + + @DELETE + @Path("{id}") + @Produces("application/json") + public Response getServer(@PathParam("id") String queryId) + { + queryManager.cancelQuery(queryId); + return Response.status(Response.Status.ACCEPTED).build(); } @POST @@ -124,10 +140,13 @@ public class QueryResource resp.setStatus(200); resp.setContentType("application/x-javascript"); + resp.setHeader("X-Druid-Query-Id", query.getId()); out = resp.getOutputStream(); jsonWriter.writeValue(out, results); +// JsonGenerator jgen = jsonWriter.getFactory().createGenerator(out); + long requestTime = System.currentTimeMillis() - start; emitter.emit( From 1be85af32019afd80252ebc2aa746640afa8fb04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 9 May 2014 22:51:43 -0700 Subject: [PATCH 015/270] handle query interruption at cursor level --- .../io/druid/segment/QueryableIndexStorageAdapter.java | 7 +++++++ .../incremental/IncrementalIndexStorageAdapter.java | 5 +++++ 2 files changed, 12 insertions(+) diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index a71297b3b89..f624cdfa807 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -27,6 +27,7 @@ import com.google.common.io.Closeables; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; +import io.druid.query.QueryInterruptedException; import io.druid.query.filter.Filter; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; @@ -224,6 +225,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public void advance() { + if (Thread.interrupted()) { + throw new QueryInterruptedException(); + } cursorOffset.increment(); } @@ -652,6 +656,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public void advance() { + if (Thread.interrupted()) { + throw new QueryInterruptedException(); + } ++currRow; } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 0eddf59ac98..3fe807b2761 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -29,6 +29,7 @@ import com.metamx.collections.spatial.search.Bound; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; +import io.druid.query.QueryInterruptedException; import io.druid.query.aggregation.Aggregator; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; @@ -239,6 +240,10 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter Iterators.advance(baseIter, numAdvanced); } + if (Thread.interrupted()) { + throw new QueryInterruptedException(); + } + boolean foundMatched = false; while (baseIter.hasNext()) { currEntry.set(baseIter.next()); From 32f6243be0a765bb59389671132b9fb9d35f66cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 9 May 2014 23:28:42 -0700 Subject: [PATCH 016/270] proper closing of resources in case of query cancellation --- .../timeseries/TimeseriesQueryEngine.java | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java index 9f1d0860aa7..ad290536b30 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryEngine.java @@ -60,27 +60,29 @@ public class TimeseriesQueryEngine { Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); - while (!cursor.isDone()) { - for (Aggregator aggregator : aggregators) { - aggregator.aggregate(); + try { + while (!cursor.isDone()) { + for (Aggregator aggregator : aggregators) { + aggregator.aggregate(); + } + cursor.advance(); } - cursor.advance(); + + TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); + + for (Aggregator aggregator : aggregators) { + bob.addMetric(aggregator); + } + + Result retVal = bob.build(); + return retVal; } - - TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); - - for (Aggregator aggregator : aggregators) { - bob.addMetric(aggregator); + finally { + // cleanup + for (Aggregator agg : aggregators) { + agg.close(); + } } - - Result retVal = bob.build(); - - // cleanup - for (Aggregator agg : aggregators) { - agg.close(); - } - - return retVal; } } ); From 4f1e1576397e08eb26413b76108d0c03eb6bbefd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 14 May 2014 16:58:41 -0700 Subject: [PATCH 017/270] pass through errors from computes in DirectDruidClient --- .../io/druid/query/QueryInterruptedException.java | 13 ++++++++++++- .../java/io/druid/client/DirectDruidClient.java | 8 +++++++- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/QueryInterruptedException.java b/processing/src/main/java/io/druid/query/QueryInterruptedException.java index 7b889ef5f2d..00676918dcf 100644 --- a/processing/src/main/java/io/druid/query/QueryInterruptedException.java +++ b/processing/src/main/java/io/druid/query/QueryInterruptedException.java @@ -19,13 +19,17 @@ package io.druid.query; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + public class QueryInterruptedException extends RuntimeException { public QueryInterruptedException() { super(); } - public QueryInterruptedException(String message) + @JsonCreator + public QueryInterruptedException(@JsonProperty("error") String message) { super(message); } @@ -34,4 +38,11 @@ public class QueryInterruptedException extends RuntimeException { super(cause); } + + @JsonProperty("error") + @Override + public String getMessage() + { + return super.getMessage(); + } } diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index ae994d1cda7..c08cd9e2bd4 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -45,6 +45,7 @@ import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; import io.druid.query.BySegmentResultValueClass; import io.druid.query.Query; +import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -283,7 +284,12 @@ public class DirectDruidClient implements QueryRunner if (jp == null) { try { jp = objectMapper.getFactory().createParser(future.get()); - if (jp.nextToken() != JsonToken.START_ARRAY) { + final JsonToken nextToken = jp.nextToken(); + if (nextToken == JsonToken.START_OBJECT) { + QueryInterruptedException e = jp.getCodec().readValue(jp, QueryInterruptedException.class); + throw e; + } + else if (nextToken != JsonToken.START_ARRAY) { throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url); } else { jp.nextToken(); From 1183c68ce4e2c55529be8bcf7fcd7c087af2fdcd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 14 May 2014 16:59:01 -0700 Subject: [PATCH 018/270] formatting --- .../main/java/io/druid/segment/ReferenceCountingSequence.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java index 402e63dc31d..4c5cbfc3d3d 100644 --- a/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java +++ b/processing/src/main/java/io/druid/segment/ReferenceCountingSequence.java @@ -48,4 +48,4 @@ public class ReferenceCountingSequence extends YieldingSequenceBase final Closeable closeable = segment.increment(); return new ResourceClosingYielder(baseSequence.toYielder(initValue, accumulator), closeable); } -} \ No newline at end of file +} From a56a655eae2f4dfb5492f278d5bb0078ec3293bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 14 May 2014 17:00:18 -0700 Subject: [PATCH 019/270] proper query exceptions and add support for query timeout --- .../query/ChainedExecutionQueryRunner.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 6c12d37669b..c7ed29f1ddf 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -39,8 +39,11 @@ import com.metamx.common.logger.Logger; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; /** * A QueryRunner that combines a list of other QueryRunners and executes them in parallel on an executor. @@ -149,15 +152,26 @@ public class ChainedExecutionQueryRunner implements QueryRunner queryWatcher.registerQuery(query, futures); try { + final Number timeout = query.getContextValue("timeout", (Number)null); return new MergeIterable<>( ordering.nullsFirst(), - futures.get() + timeout == null ? + futures.get() : + futures.get(timeout.longValue(), TimeUnit.MILLISECONDS) ).iterator(); } catch (InterruptedException e) { log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); futures.cancel(true); - throw new QueryInterruptedException(e); + throw new QueryInterruptedException("Query interrupted"); + } + catch(CancellationException e) { + log.warn(e, "Query cancelled, query id [%s]", query.getId()); + throw new QueryInterruptedException("Query cancelled"); + } + catch(TimeoutException e) { + log.warn(e, "Query timeout, query id [%s]", query.getId()); + throw new QueryInterruptedException("Query timeout"); } catch (ExecutionException e) { throw Throwables.propagate(e.getCause()); From def62c74f8e60f799b7118d26554e5f340ca9cc5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Wed, 14 May 2014 17:01:18 -0700 Subject: [PATCH 020/270] properly remove completed queries from query manager --- .../java/io/druid/server/QueryManager.java | 52 +++++++++++++------ 1 file changed, 36 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/io/druid/server/QueryManager.java b/server/src/main/java/io/druid/server/QueryManager.java index 5910aeda535..84b947414dd 100644 --- a/server/src/main/java/io/druid/server/QueryManager.java +++ b/server/src/main/java/io/druid/server/QueryManager.java @@ -19,30 +19,50 @@ package io.druid.server; -import com.google.common.collect.Maps; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimaps; +import com.google.common.collect.SetMultimap; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import io.druid.query.Query; import io.druid.query.QueryWatcher; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Future; +import java.util.Set; public class QueryManager implements QueryWatcher { - final ConcurrentMap queries; + final SetMultimap queries; - public QueryManager() { - this.queries = Maps.newConcurrentMap(); - } - - public void cancelQuery(String id) { - Future future = queries.get(id); - if(future != null) { - future.cancel(true); - } - } - public void registerQuery(Query query, ListenableFuture future) + public QueryManager() { - queries.put(query.getId(), future); + this.queries = Multimaps.synchronizedSetMultimap( + HashMultimap.create() + ); + } + + public boolean cancelQuery(String id) { + Set futures = queries.removeAll(id); + boolean success = true; + for (ListenableFuture future : futures) { + success = success && future.cancel(true); + } + return success; + } + + public void registerQuery(Query query, final ListenableFuture future) + { + final String id = query.getId(); + queries.put(id, future); + future.addListener( + new Runnable() + { + @Override + public void run() + { + queries.remove(id, future); + } + }, + MoreExecutors.sameThreadExecutor() + ); } } From d2c729adec3b9b82705646c6f006ad55b8860a5e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 12 May 2014 15:07:49 -0700 Subject: [PATCH 021/270] return proper json errors to propagate query cancellation and timeout --- .../io/druid/jackson/DefaultObjectMapper.java | 13 +- .../DruidDefaultSerializersModule.java | 26 ++- .../java/io/druid/server/QueryResource.java | 154 +++++++++++------- 3 files changed, 127 insertions(+), 66 deletions(-) diff --git a/processing/src/main/java/io/druid/jackson/DefaultObjectMapper.java b/processing/src/main/java/io/druid/jackson/DefaultObjectMapper.java index 952b506bd95..e55e2299d2b 100644 --- a/processing/src/main/java/io/druid/jackson/DefaultObjectMapper.java +++ b/processing/src/main/java/io/druid/jackson/DefaultObjectMapper.java @@ -32,7 +32,12 @@ public class DefaultObjectMapper extends ObjectMapper { public DefaultObjectMapper() { - this(null); + this((JsonFactory)null); + } + + public DefaultObjectMapper(DefaultObjectMapper mapper) + { + super(mapper); } public DefaultObjectMapper(JsonFactory factory) @@ -52,4 +57,10 @@ public class DefaultObjectMapper extends ObjectMapper configure(MapperFeature.AUTO_DETECT_SETTERS, false); configure(SerializationFeature.INDENT_OUTPUT, false); } + + @Override + public ObjectMapper copy() + { + return new DefaultObjectMapper(this); + } } diff --git a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java index 068bfecd963..6184221a1db 100644 --- a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java @@ -105,7 +105,7 @@ public class DruidDefaultSerializersModule extends SimpleModule jgen.writeStartArray(); value.accumulate( null, - new Accumulator() + new Accumulator() { @Override public Object accumulate(Object o, Object o1) @@ -116,7 +116,7 @@ public class DruidDefaultSerializersModule extends SimpleModule catch (IOException e) { throw Throwables.propagate(e); } - return o; + return null; } } ); @@ -124,6 +124,28 @@ public class DruidDefaultSerializersModule extends SimpleModule } } ); + addSerializer( + Yielder.class, + new JsonSerializer() + { + @Override + public void serialize(Yielder yielder, final JsonGenerator jgen, SerializerProvider provider) + throws IOException, JsonProcessingException + { + jgen.writeStartArray(); + try { + while (!yielder.isDone()) { + final Object o = yielder.get(); + jgen.writeObject(o); + yielder = yielder.next(null); + } + } finally { + yielder.close(); + } + jgen.writeEndArray(); + } + } + ); addSerializer(ByteOrder.class, ToStringSerializer.instance); addDeserializer( ByteOrder.class, diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index e710a6c97c9..1e6ea06607f 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -19,16 +19,23 @@ package io.druid.server; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.api.client.repackaged.com.google.common.base.Throwables; import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.inject.Inject; +import com.metamx.common.guava.Accumulator; +import com.metamx.common.guava.Accumulators; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import com.metamx.common.guava.YieldingAccumulators; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -36,21 +43,25 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.query.DataSourceUtil; import io.druid.query.Query; +import io.druid.query.QueryInterruptedException; import io.druid.query.QuerySegmentWalker; -import io.druid.query.QueryWatcher; import io.druid.server.log.RequestLogger; import org.joda.time.DateTime; import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; +import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.StreamingOutput; import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; @@ -64,6 +75,8 @@ public class QueryResource private static final EmittingLogger log = new EmittingLogger(QueryResource.class); private static final Charset UTF8 = Charset.forName("UTF-8"); private static final Joiner COMMA_JOIN = Joiner.on(","); + public static final String APPLICATION_SMILE = "application/smile"; + public static final String APPLICATION_JSON = "application/json"; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; @@ -82,8 +95,12 @@ public class QueryResource QueryManager queryManager ) { - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; + this.jsonMapper = jsonMapper.copy(); + this.jsonMapper.getFactory().configure(JsonGenerator.Feature.AUTO_CLOSE_TARGET, false); + + this.smileMapper = smileMapper.copy(); + this.smileMapper.getFactory().configure(JsonGenerator.Feature.AUTO_CLOSE_TARGET, false); + this.texasRanger = texasRanger; this.emitter = emitter; this.requestLogger = requestLogger; @@ -97,13 +114,13 @@ public class QueryResource { queryManager.cancelQuery(queryId); return Response.status(Response.Status.ACCEPTED).build(); + } @POST - @Produces("application/json") - public void doPost( + public Response doPost( @Context HttpServletRequest req, - @Context HttpServletResponse resp + @Context final HttpServletResponse resp ) throws ServletException, IOException { final long start = System.currentTimeMillis(); @@ -111,13 +128,12 @@ public class QueryResource byte[] requestQuery = null; String queryId; - final boolean isSmile = "application/smile".equals(req.getContentType()); + final boolean isSmile = APPLICATION_SMILE.equals(req.getContentType()); ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - ObjectWriter jsonWriter = req.getParameter("pretty") == null + final ObjectWriter jsonWriter = req.getParameter("pretty") == null ? objectMapper.writer() : objectMapper.writerWithDefaultPrettyPrinter(); - OutputStream out = null; try { requestQuery = ByteStreams.toByteArray(req.getInputStream()); @@ -132,48 +148,70 @@ public class QueryResource log.debug("Got query [%s]", query); } - Sequence results = query.run(texasRanger); + Sequence results = query.run(texasRanger); if (results == null) { results = Sequences.empty(); } - resp.setStatus(200); - resp.setContentType("application/x-javascript"); - resp.setHeader("X-Druid-Query-Id", query.getId()); - - out = resp.getOutputStream(); - jsonWriter.writeValue(out, results); - -// JsonGenerator jgen = jsonWriter.getFactory().createGenerator(out); - - long requestTime = System.currentTimeMillis() - start; - - emitter.emit( - new ServiceMetricEvent.Builder() - .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) - .setUser4(query.getType()) - .setUser5(COMMA_JOIN.join(query.getIntervals())) - .setUser6(String.valueOf(query.hasFilters())) - .setUser7(req.getRemoteAddr()) - .setUser8(queryId) - .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) - .build("request/time", requestTime) - ); - - requestLogger.log( - new RequestLogLine( - new DateTime(), - req.getRemoteAddr(), - query, - new QueryStats( - ImmutableMap.of( - "request/time", requestTime, - "success", true - ) - ) + try ( + final Yielder yielder = results.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Object accumulate(Object accumulated, Object in) + { + yield(); + return in; + } + } ) - ); + ) { + long requestTime = System.currentTimeMillis() - start; + + emitter.emit( + new ServiceMetricEvent.Builder() + .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) + .setUser4(query.getType()) + .setUser5(COMMA_JOIN.join(query.getIntervals())) + .setUser6(String.valueOf(query.hasFilters())) + .setUser7(req.getRemoteAddr()) + .setUser8(queryId) + .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) + .build("request/time", requestTime) + ); + + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "request/time", requestTime, + "success", true + ) + ) + ) + ); + + return Response + .ok( + new StreamingOutput() + { + @Override + public void write(OutputStream outputStream) throws IOException, WebApplicationException + { + jsonWriter.writeValue(outputStream, yielder); + outputStream.close(); + } + }, + isSmile ? APPLICATION_JSON : APPLICATION_SMILE + ) + .header("X-Druid-Query-Id", queryId) + .build(); + } } catch (Exception e) { final String queryString = @@ -183,20 +221,6 @@ public class QueryResource log.warn(e, "Exception occurred on request [%s]", queryString); - if (!resp.isCommitted()) { - resp.setStatus(500); - resp.resetBuffer(); - - if (out == null) { - out = resp.getOutputStream(); - } - - out.write((e.getMessage() == null) ? "Exception null".getBytes(UTF8) : e.getMessage().getBytes(UTF8)); - out.write("\n".getBytes(UTF8)); - } - - resp.flushBuffer(); - try { requestLogger.log( new RequestLogLine( @@ -216,10 +240,14 @@ public class QueryResource .addData("query", queryString) .addData("peer", req.getRemoteAddr()) .emit(); - } - finally { - resp.flushBuffer(); - Closeables.closeQuietly(out); + + return Response.serverError().entity( + jsonWriter.writeValueAsString( + ImmutableMap.of( + "error", (e.getMessage() == null) ? "null Exception" : e.getMessage() + ) + ) + ).build(); } } } From d01f272a7ab71ab7ba565493404f8330af3e774b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 2 Jun 2014 17:40:35 -0700 Subject: [PATCH 022/270] forward cancellation in direct druid client --- pom.xml | 2 +- .../io/druid/client/BrokerServerView.java | 8 +- .../io/druid/client/DirectDruidClient.java | 46 ++++++++-- .../java/io/druid/server/QueryResource.java | 1 - .../druid/client/DirectDruidClientTest.java | 90 +++++++++++++++++-- 5 files changed, 127 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index 2116d7f617f..2a3af54886e 100644 --- a/pom.xml +++ b/pom.xml @@ -78,7 +78,7 @@ com.metamx http-client - 0.9.5 + 0.9.6 com.metamx diff --git a/server/src/main/java/io/druid/client/BrokerServerView.java b/server/src/main/java/io/druid/client/BrokerServerView.java index 57663154156..0070622ac85 100644 --- a/server/src/main/java/io/druid/client/BrokerServerView.java +++ b/server/src/main/java/io/druid/client/BrokerServerView.java @@ -32,10 +32,9 @@ import io.druid.client.selector.TierSelectorStrategy; import io.druid.concurrent.Execs; import io.druid.guice.annotations.Client; import io.druid.query.DataSource; -import io.druid.query.QueryDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChestWarehouse; -import io.druid.query.TableDataSource; +import io.druid.query.QueryWatcher; import io.druid.server.coordination.DruidServerMetadata; import io.druid.timeline.DataSegment; import io.druid.timeline.VersionedIntervalTimeline; @@ -60,6 +59,7 @@ public class BrokerServerView implements TimelineServerView private final Map> timelines; private final QueryToolChestWarehouse warehouse; + private final QueryWatcher queryWatcher; private final ObjectMapper smileMapper; private final HttpClient httpClient; private final ServerInventoryView baseView; @@ -68,6 +68,7 @@ public class BrokerServerView implements TimelineServerView @Inject public BrokerServerView( QueryToolChestWarehouse warehouse, + QueryWatcher queryWatcher, ObjectMapper smileMapper, @Client HttpClient httpClient, ServerInventoryView baseView, @@ -75,6 +76,7 @@ public class BrokerServerView implements TimelineServerView ) { this.warehouse = warehouse; + this.queryWatcher = queryWatcher; this.smileMapper = smileMapper; this.httpClient = httpClient; this.baseView = baseView; @@ -154,7 +156,7 @@ public class BrokerServerView implements TimelineServerView private DirectDruidClient makeDirectClient(DruidServer server) { - return new DirectDruidClient(warehouse, smileMapper, httpClient, server.getHost()); + return new DirectDruidClient(warehouse, queryWatcher, smileMapper, httpClient, server.getHost()); } private QueryableDruidServer removeServer(DruidServer server) diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index c08cd9e2bd4..34584ee2395 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.type.TypeFactory; import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.google.common.io.Closeables; @@ -43,12 +44,15 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.io.AppendableByteArrayInputStream; import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; import io.druid.query.BySegmentResultValueClass; import io.druid.query.Query; import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.aggregation.MetricManipulatorFns; import org.jboss.netty.handler.codec.http.HttpChunk; @@ -61,6 +65,7 @@ import java.io.InputStream; import java.net.URL; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; @@ -74,6 +79,7 @@ public class DirectDruidClient implements QueryRunner private static final Map, Pair> typesMap = Maps.newConcurrentMap(); private final QueryToolChestWarehouse warehouse; + private final QueryWatcher queryWatcher; private final ObjectMapper objectMapper; private final HttpClient httpClient; private final String host; @@ -83,12 +89,14 @@ public class DirectDruidClient implements QueryRunner public DirectDruidClient( QueryToolChestWarehouse warehouse, + QueryWatcher queryWatcher, ObjectMapper objectMapper, HttpClient httpClient, String host ) { this.warehouse = warehouse; + this.queryWatcher = queryWatcher; this.objectMapper = objectMapper; this.httpClient = httpClient; this.host = host; @@ -103,7 +111,7 @@ public class DirectDruidClient implements QueryRunner } @Override - public Sequence run(Query query) + public Sequence run(final Query query) { QueryToolChest> toolChest = warehouse.getToolChest(query); boolean isBySegment = query.getContextBySegment(false); @@ -128,6 +136,7 @@ public class DirectDruidClient implements QueryRunner final ListenableFuture future; final String url = String.format("http://%s/druid/v2/", host); + final String cancelUrl = String.format("http://%s/druid/v2/%s", host, query.getId()); try { log.debug("Querying url[%s]", url); @@ -175,6 +184,9 @@ public class DirectDruidClient implements QueryRunner } } ); + + queryWatcher.registerQuery(query, future); + openConnections.getAndIncrement(); Futures.addCallback( future, new FutureCallback() @@ -189,6 +201,27 @@ public class DirectDruidClient implements QueryRunner public void onFailure(Throwable t) { openConnections.getAndDecrement(); + if (future.isCancelled()) { + // forward the cancellation to underlying queriable node + try { + StatusResponseHolder res = httpClient + .delete(new URL(cancelUrl)) + .setContent(objectMapper.writeValueAsBytes(query)) + .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? "application/smile" : "application/json") + .go(new StatusResponseHandler(Charsets.UTF_8)) + .get(); + if (res.getStatus().getCode() >= 500) { + throw new RE( + "Error cancelling query[%s]: queriable node returned status[%d] [%s].", + res.getStatus().getCode(), + res.getStatus().getReasonPhrase() + ); + } + } + catch (IOException | ExecutionException | InterruptedException e) { + Throwables.propagate(e); + } + } } } ); @@ -197,7 +230,7 @@ public class DirectDruidClient implements QueryRunner throw Throwables.propagate(e); } - Sequence retVal = new BaseSequence>( + Sequence retVal = new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override @@ -296,14 +329,11 @@ public class DirectDruidClient implements QueryRunner objectCodec = jp.getCodec(); } } - catch (IOException e) { + catch (IOException | InterruptedException | ExecutionException e) { throw new RE(e, "Failure getting results from[%s]", url); } - catch (InterruptedException e) { - throw new RE(e, "Failure getting results from[%s]", url); - } - catch (ExecutionException e) { - throw new RE(e, "Failure getting results from[%s]", url); + catch (CancellationException e) { + throw new QueryInterruptedException(); } } } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 1e6ea06607f..c97657b4e35 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -73,7 +73,6 @@ import java.util.UUID; public class QueryResource { private static final EmittingLogger log = new EmittingLogger(QueryResource.class); - private static final Charset UTF8 = Charset.forName("UTF-8"); private static final Joiner COMMA_JOIN = Joiner.on(","); public static final String APPLICATION_SMILE = "application/smile"; public static final String APPLICATION_JSON = "application/json"; diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index aba91657686..84a80058d35 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -21,18 +21,24 @@ package io.druid.client; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.http.client.HttpClient; import com.metamx.http.client.Request; import com.metamx.http.client.RequestBuilder; +import com.metamx.http.client.response.StatusResponseHolder; import io.druid.client.selector.ConnectionCountServerSelectorStrategy; import io.druid.client.selector.HighestPriorityTierSelectorStrategy; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; +import io.druid.query.Query; +import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryWatcher; import io.druid.query.ReflectionQueryToolChestWarehouse; import io.druid.query.Result; import io.druid.query.timeboundary.TimeBoundaryQuery; @@ -41,11 +47,13 @@ import io.druid.timeline.partition.NoneShardSpec; import junit.framework.Assert; import org.easymock.EasyMock; import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.jboss.netty.handler.timeout.ReadTimeoutException; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import java.io.ByteArrayInputStream; import java.io.InputStream; @@ -54,17 +62,22 @@ import java.util.List; public class DirectDruidClientTest { - private HttpClient httpClient; - - @Before - public void setUp() throws Exception + public static final QueryWatcher DUMMY_WATCHER = new QueryWatcher() { - httpClient = EasyMock.createMock(HttpClient.class); - } + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }; + + @Rule + public ExpectedException thrown = ExpectedException.none(); @Test public void testRun() throws Exception { + HttpClient httpClient = EasyMock.createMock(HttpClient.class); RequestBuilder requestBuilder = new RequestBuilder(httpClient, HttpMethod.POST, new URL("http://foo.com")); EasyMock.expect(httpClient.post(EasyMock.anyObject())).andReturn(requestBuilder).atLeastOnce(); @@ -93,12 +106,14 @@ public class DirectDruidClientTest DirectDruidClient client1 = new DirectDruidClient( new ReflectionQueryToolChestWarehouse(), + DUMMY_WATCHER, new DefaultObjectMapper(), httpClient, "foo" ); DirectDruidClient client2 = new DirectDruidClient( new ReflectionQueryToolChestWarehouse(), + DUMMY_WATCHER, new DefaultObjectMapper(), httpClient, "foo2" @@ -149,4 +164,65 @@ public class DirectDruidClientTest EasyMock.verify(httpClient); } + + @Test + public void testCancel() throws Exception + { + HttpClient httpClient = EasyMock.createStrictMock(HttpClient.class); + EasyMock.expect(httpClient.post(EasyMock.anyObject())).andReturn( + new RequestBuilder(httpClient, HttpMethod.POST, new URL("http://foo.com")) + ).once(); + + ListenableFuture cancelledFuture = Futures.immediateCancelledFuture(); + EasyMock.expect(httpClient.go(EasyMock.anyObject())).andReturn(cancelledFuture).once(); + + EasyMock.expect(httpClient.delete(EasyMock.anyObject())) + .andReturn(new RequestBuilder(httpClient, HttpMethod.DELETE, new URL("http://foo.com/delete"))) + .once(); + SettableFuture cancellationFuture = SettableFuture.create(); + EasyMock.expect(httpClient.go(EasyMock.anyObject())).andReturn(cancellationFuture).once(); + + EasyMock.replay(httpClient); + + final ServerSelector serverSelector = new ServerSelector( + new DataSegment( + "test", + new Interval("2013-01-01/2013-01-02"), + new DateTime("2013-01-01").toString(), + Maps.newHashMap(), + Lists.newArrayList(), + Lists.newArrayList(), + new NoneShardSpec(), + 0, + 0L + ), + new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) + ); + + DirectDruidClient client1 = new DirectDruidClient( + new ReflectionQueryToolChestWarehouse(), + DUMMY_WATCHER, + new DefaultObjectMapper(), + httpClient, + "foo" + ); + + QueryableDruidServer queryableDruidServer1 = new QueryableDruidServer( + new DruidServer("test1", "localhost", 0, "historical", DruidServer.DEFAULT_TIER, 0), + client1 + ); + serverSelector.addServer(queryableDruidServer1); + + TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder().dataSource("test").build(); + + cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); + Sequence results = client1.run(query); + Assert.assertEquals(0, client1.getNumOpenConnections()); + + + thrown.expect(QueryInterruptedException.class); + Assert.assertTrue(Sequences.toList(results, Lists.newArrayList()).isEmpty()); + + EasyMock.verify(httpClient); + } } From 855c66c9ad13e0b133597e0324ffaf761c99bbeb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 2 Jun 2014 18:29:42 -0700 Subject: [PATCH 023/270] less stack traces when cancelling queries --- .../query/ChainedExecutionQueryRunner.java | 5 ++-- .../io/druid/client/DirectDruidClient.java | 2 +- .../java/io/druid/server/QueryResource.java | 26 +++++++++++++++++-- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index c7ed29f1ddf..256cefa33a0 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -136,6 +136,9 @@ public class ChainedExecutionQueryRunner implements QueryRunner return retVal; } + catch (QueryInterruptedException e) { + throw Throwables.propagate(e); + } catch (Exception e) { log.error(e, "Exception with one of the sequences!"); throw Throwables.propagate(e); @@ -166,11 +169,9 @@ public class ChainedExecutionQueryRunner implements QueryRunner throw new QueryInterruptedException("Query interrupted"); } catch(CancellationException e) { - log.warn(e, "Query cancelled, query id [%s]", query.getId()); throw new QueryInterruptedException("Query cancelled"); } catch(TimeoutException e) { - log.warn(e, "Query timeout, query id [%s]", query.getId()); throw new QueryInterruptedException("Query timeout"); } catch (ExecutionException e) { diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 34584ee2395..b6030f9755b 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -333,7 +333,7 @@ public class DirectDruidClient implements QueryRunner throw new RE(e, "Failure getting results from[%s]", url); } catch (CancellationException e) { - throw new QueryInterruptedException(); + throw new QueryInterruptedException("Query cancelled"); } } } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index c97657b4e35..33bdd519c83 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -125,7 +125,7 @@ public class QueryResource final long start = System.currentTimeMillis(); Query query = null; byte[] requestQuery = null; - String queryId; + String queryId = null; final boolean isSmile = APPLICATION_SMILE.equals(req.getContentType()); @@ -212,6 +212,28 @@ public class QueryResource .build(); } } + catch (QueryInterruptedException e) { + try { + log.info("%s [%s]", e.getMessage(), queryId); + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats(ImmutableMap.of("success", false, "interrupted", true, "reason", e.toString())) + ) + ); + } catch (Exception e2) { + log.error(e2, "Unable to log query [%s]!", query); + } + return Response.serverError().entity( + jsonWriter.writeValueAsString( + ImmutableMap.of( + "error", e.getMessage() + ) + ) + ).build(); + } catch (Exception e) { final String queryString = query == null @@ -243,7 +265,7 @@ public class QueryResource return Response.serverError().entity( jsonWriter.writeValueAsString( ImmutableMap.of( - "error", (e.getMessage() == null) ? "null Exception" : e.getMessage() + "error", e.getMessage() == null ? "null exception" : e.getMessage() ) ) ).build(); From d0f9c438f826f6ebe072bb193a80e8b5757d0c79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 2 Jun 2014 17:39:08 -0700 Subject: [PATCH 024/270] proper query cancellation tests --- .../ChainedExecutionQueryRunnerTest.java | 115 +++++++++++------- .../io/druid/query/QueryRunnerTestHelper.java | 11 ++ .../java/io/druid/query/TestQueryRunners.java | 13 +- .../druid/query/topn/TopNQueryRunnerTest.java | 18 +-- .../druid/query/topn/TopNUnionQueryTest.java | 9 +- .../druid/client/DirectDruidClientTest.java | 16 +-- 6 files changed, 107 insertions(+), 75 deletions(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 445bea9cf53..b5391605d32 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -20,7 +20,6 @@ package io.druid.query; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.concurrent.ExecutorServiceConfig; @@ -29,18 +28,22 @@ import com.metamx.common.guava.Sequences; import com.metamx.common.lifecycle.Lifecycle; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; -import org.junit.Ignore; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.Assert; import org.junit.Test; -import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; public class ChainedExecutionQueryRunnerTest { - @Test @Ignore + @Test public void testQueryCancellation() throws Exception { ExecutorService exec = PrioritizedExecutorService.create( @@ -63,25 +66,36 @@ public class ChainedExecutionQueryRunnerTest final CountDownLatch queriesStarted = new CountDownLatch(2); final CountDownLatch queryIsRegistered = new CountDownLatch(1); - final Map queries = Maps.newHashMap(); - QueryWatcher watcher = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - queries.put(query, future); - queryIsRegistered.countDown(); - } - }; + Capture capturedFuture = new Capture<>(); + QueryWatcher watcher = EasyMock.createStrictMock(QueryWatcher.class); + watcher.registerQuery(EasyMock.anyObject(), EasyMock.and(EasyMock.anyObject(), EasyMock.capture(capturedFuture))); + EasyMock.expectLastCall() + .andAnswer( + new IAnswer() + { + @Override + public Void answer() throws Throwable + { + queryIsRegistered.countDown(); + return null; + } + } + ) + .once(); + EasyMock.replay(watcher); + + DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted); + DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted); + DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted); ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>( exec, Ordering.natural(), watcher, Lists.>newArrayList( - new DyingQueryRunner(1, queriesStarted), - new DyingQueryRunner(2, queriesStarted), - new DyingQueryRunner(3, queriesStarted) + runner1, + runner2, + runner3 ) ); @@ -93,7 +107,7 @@ public class ChainedExecutionQueryRunnerTest .build() ); - Future f = Executors.newFixedThreadPool(1).submit( + Future resultFuture = Executors.newFixedThreadPool(1).submit( new Runnable() { @Override @@ -104,45 +118,64 @@ public class ChainedExecutionQueryRunnerTest } ); - // wait for query to register - queryIsRegistered.await(); - queriesStarted.await(); + // wait for query to register and start + Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS)); + Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS)); // cancel the query - queries.values().iterator().next().cancel(true); - f.get(); + Assert.assertTrue(capturedFuture.hasCaptured()); + ListenableFuture future = capturedFuture.getValue(); + future.cancel(true); + + QueryInterruptedException cause = null; + try { + resultFuture.get(); + } catch(ExecutionException e) { + Assert.assertTrue(e.getCause() instanceof QueryInterruptedException); + cause = (QueryInterruptedException)e.getCause(); + } + Assert.assertNotNull(cause); + Assert.assertTrue(future.isCancelled()); + Assert.assertTrue(runner1.hasStarted); + Assert.assertTrue(runner2.hasStarted); + Assert.assertFalse(runner3.hasStarted); + Assert.assertFalse(runner1.hasCompleted); + Assert.assertFalse(runner2.hasCompleted); + Assert.assertFalse(runner3.hasCompleted); + + EasyMock.verify(watcher); } private static class DyingQueryRunner implements QueryRunner { - private final int id; private final CountDownLatch latch; + private boolean hasStarted = false; + private boolean hasCompleted = false; - public DyingQueryRunner(int id, CountDownLatch latch) { - this.id = id; + public DyingQueryRunner(CountDownLatch latch) + { this.latch = latch; } @Override public Sequence run(Query query) { + hasStarted = true; latch.countDown(); - - int i = 0; - while (i >= 0) { - if(Thread.interrupted()) { - throw new QueryInterruptedException("I got killed"); - } - - // do a lot of work - try { - Thread.sleep(100); - } catch (InterruptedException e) { - throw new QueryInterruptedException("I got killed"); - } - ++i; + if (Thread.interrupted()) { + throw new QueryInterruptedException("I got killed"); } - return Sequences.simple(Lists.newArrayList(i)); + + // do a lot of work + try { + Thread.sleep(500); + } + catch (InterruptedException e) { + throw new QueryInterruptedException("I got killed"); + } + + hasCompleted = true; + return Sequences.simple(Lists.newArrayList(123)); } } } diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index ffa6b02c236..954cb4fd5ee 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -21,6 +21,7 @@ package io.druid.query; import com.google.common.base.Function; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -53,6 +54,16 @@ import java.util.List; */ public class QueryRunnerTestHelper { + + public static final QueryWatcher DUMMY_QUERYWATCHER = new QueryWatcher() + { + @Override + public void registerQuery(Query query, ListenableFuture future) + { + + } + }; + public static final String segmentId = "testSegment"; public static final String dataSource = "testing"; public static final UnionDataSource unionDataSource = new UnionDataSource( diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java index a858b5e0cdf..90d394f3e3b 100644 --- a/processing/src/test/java/io/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -41,14 +41,11 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = new TopNQueryRunnerFactory(pool, new TopNQueryQueryToolChest(topNConfig), new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }); + QueryRunnerFactory factory = new TopNQueryRunnerFactory( + pool, + new TopNQueryQueryToolChest(topNConfig), + QueryRunnerTestHelper.DUMMY_QUERYWATCHER + ); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 39af4459794..f06258a9a09 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -72,14 +72,7 @@ public class TopNQueryRunnerTest new TopNQueryRunnerFactory( TestQueryRunners.getPool(), new TopNQueryQueryToolChest(new TopNQueryConfig()), - new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - } + QueryRunnerTestHelper.DUMMY_QUERYWATCHER ) ) ); @@ -97,14 +90,7 @@ public class TopNQueryRunnerTest } ), new TopNQueryQueryToolChest(new TopNQueryConfig()), - new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - } + QueryRunnerTestHelper.DUMMY_QUERYWATCHER ) ) ); diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index 1fdc3b11cf5..d2e3e5dea73 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -23,9 +23,12 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.ListenableFuture; import io.druid.collections.StupidPool; +import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; @@ -65,7 +68,8 @@ public class TopNUnionQueryTest QueryRunnerTestHelper.makeUnionQueryRunners( new TopNQueryRunnerFactory( TestQueryRunners.getPool(), - new TopNQueryQueryToolChest(new TopNQueryConfig()) + new TopNQueryQueryToolChest(new TopNQueryConfig()), + QueryRunnerTestHelper.DUMMY_QUERYWATCHER ) ) ); @@ -82,7 +86,8 @@ public class TopNUnionQueryTest } } ), - new TopNQueryQueryToolChest(new TopNQueryConfig()) + new TopNQueryQueryToolChest(new TopNQueryConfig()), + QueryRunnerTestHelper.DUMMY_QUERYWATCHER ) ) ); diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index 84a80058d35..b2396688900 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -44,16 +44,14 @@ import io.druid.query.Result; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; -import junit.framework.Assert; import org.easymock.EasyMock; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.jboss.netty.handler.timeout.ReadTimeoutException; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.junit.Rule; +import org.junit.Assert; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.ByteArrayInputStream; import java.io.InputStream; @@ -71,9 +69,6 @@ public class DirectDruidClientTest } }; - @Rule - public ExpectedException thrown = ExpectedException.none(); - @Test public void testRun() throws Exception { @@ -220,8 +215,13 @@ public class DirectDruidClientTest Assert.assertEquals(0, client1.getNumOpenConnections()); - thrown.expect(QueryInterruptedException.class); - Assert.assertTrue(Sequences.toList(results, Lists.newArrayList()).isEmpty()); + QueryInterruptedException exception = null; + try { + Sequences.toList(results, Lists.newArrayList()); + } catch(QueryInterruptedException e) { + exception = e; + } + Assert.assertNotNull(exception); EasyMock.verify(httpClient); } From 97d5455f3a455fecbb1296f60fdf038ef040f862 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 3 Jun 2014 11:15:03 -0700 Subject: [PATCH 025/270] properly kill timed out queries --- .../query/ChainedExecutionQueryRunner.java | 2 + .../ChainedExecutionQueryRunnerTest.java | 105 ++++++++++++++++++ 2 files changed, 107 insertions(+) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 256cefa33a0..8a5ed51a4df 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -172,6 +172,8 @@ public class ChainedExecutionQueryRunner implements QueryRunner throw new QueryInterruptedException("Query cancelled"); } catch(TimeoutException e) { + log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); + futures.cancel(true); throw new QueryInterruptedException("Query timeout"); } catch (ExecutionException e) { diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index b5391605d32..f2555dd7214 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -19,6 +19,7 @@ package io.druid.query; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListenableFuture; @@ -146,6 +147,110 @@ public class ChainedExecutionQueryRunnerTest EasyMock.verify(watcher); } + @Test + public void testQueryTimeout() throws Exception + { + ExecutorService exec = PrioritizedExecutorService.create( + new Lifecycle(), new ExecutorServiceConfig() + { + @Override + public String getFormatString() + { + return "test"; + } + + @Override + public int getNumThreads() + { + return 2; + } + } + ); + + final CountDownLatch queriesStarted = new CountDownLatch(2); + final CountDownLatch queryIsRegistered = new CountDownLatch(1); + + Capture capturedFuture = new Capture<>(); + QueryWatcher watcher = EasyMock.createStrictMock(QueryWatcher.class); + watcher.registerQuery(EasyMock.anyObject(), EasyMock.and(EasyMock.anyObject(), EasyMock.capture(capturedFuture))); + EasyMock.expectLastCall() + .andAnswer( + new IAnswer() + { + @Override + public Void answer() throws Throwable + { + queryIsRegistered.countDown(); + return null; + } + } + ) + .once(); + + EasyMock.replay(watcher); + + DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted); + DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted); + DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted); + ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>( + exec, + Ordering.natural(), + watcher, + Lists.>newArrayList( + runner1, + runner2, + runner3 + ) + ); + + final Sequence seq = chainedRunner.run( + Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals("2014/2015") + .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) + .context(ImmutableMap.of("timeout", (100), "queryId", "test")) + .build() + ); + + Future resultFuture = Executors.newFixedThreadPool(1).submit( + new Runnable() + { + @Override + public void run() + { + Sequences.toList(seq, Lists.newArrayList()); + } + } + ); + + // wait for query to register and start + Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS)); + Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS)); + + // cancel the query + Assert.assertTrue(capturedFuture.hasCaptured()); + ListenableFuture future = capturedFuture.getValue(); + + QueryInterruptedException cause = null; + try { + resultFuture.get(); + } catch(ExecutionException e) { + Assert.assertTrue(e.getCause() instanceof QueryInterruptedException); + Assert.assertEquals("Query timeout", e.getCause().getMessage()); + cause = (QueryInterruptedException)e.getCause(); + } + Assert.assertNotNull(cause); + Assert.assertTrue(future.isCancelled()); + Assert.assertTrue(runner1.hasStarted); + Assert.assertTrue(runner2.hasStarted); + Assert.assertFalse(runner3.hasStarted); + Assert.assertFalse(runner1.hasCompleted); + Assert.assertFalse(runner2.hasCompleted); + Assert.assertFalse(runner3.hasCompleted); + + EasyMock.verify(watcher); + } + private static class DyingQueryRunner implements QueryRunner { private final CountDownLatch latch; From 99c9a2cf05fc7065a333a6b943cfb3e73fcd29e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 3 Jun 2014 11:35:26 -0700 Subject: [PATCH 026/270] make sure to close yielder in MetricsEmittingQueryRunner --- .../query/MetricsEmittingQueryRunner.java | 24 ++++++++++--------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java index 8425aa97fe2..dbad443cb36 100644 --- a/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java +++ b/processing/src/main/java/io/druid/query/MetricsEmittingQueryRunner.java @@ -167,18 +167,20 @@ public class MetricsEmittingQueryRunner implements QueryRunner @Override public void close() throws IOException { - if (!isDone() && builder.getUser10() == null) { - builder.setUser10("short"); + try { + if (!isDone() && builder.getUser10() == null) { + builder.setUser10("short"); + } + + long timeTaken = System.currentTimeMillis() - startTime; + emitter.emit(builder.build("query/time", timeTaken)); + + if (creationTime > 0) { + emitter.emit(builder.build("query/wait", startTime - creationTime)); + } + } finally { + yielder.close(); } - - long timeTaken = System.currentTimeMillis() - startTime; - emitter.emit(builder.build("query/time", timeTaken)); - - if(creationTime > 0) { - emitter.emit(builder.build("query/wait", startTime - creationTime)); - } - - yielder.close(); } }; } From c08002aa4d9b2c5892ee090833d5cd33ac827e3d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 3 Jun 2014 17:32:14 -0700 Subject: [PATCH 027/270] interrupt queries on incremental indexer --- .../segment/incremental/IncrementalIndexStorageAdapter.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 3fe807b2761..057754e6087 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -201,6 +201,10 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter } while (baseIter.hasNext()) { + if (Thread.interrupted()) { + throw new QueryInterruptedException(); + } + currEntry.set(baseIter.next()); if (filterMatcher.matches()) { From b84884ab7677a34485a7bc6d961cf8d1d29eb043 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 9 Jun 2014 13:48:01 -0700 Subject: [PATCH 028/270] remove methods used only for testing --- .../select/SelectQueryRunnerFactory.java | 15 ---------- .../TimeseriesQueryRunnerFactory.java | 19 ------------ .../io/druid/query/QueryRunnerTestHelper.java | 2 +- .../java/io/druid/query/TestQueryRunners.java | 30 +++++++------------ .../query/search/SearchQueryRunnerTest.java | 12 +++----- .../query/select/SelectQueryRunnerTest.java | 10 ++++++- .../TimeBoundaryQueryRunnerTest.java | 9 +----- .../TimeSeriesUnionQueryRunnerTest.java | 7 ++++- .../TimeseriesQueryRunnerBonusTest.java | 9 +++++- .../timeseries/TimeseriesQueryRunnerTest.java | 7 ++++- .../druid/query/topn/TopNQueryRunnerTest.java | 7 ++--- .../druid/query/topn/TopNUnionQueryTest.java | 7 ++--- .../filter/SpatialFilterBonusTest.java | 20 +++++++++++-- .../segment/filter/SpatialFilterTest.java | 20 +++++++++++-- .../server/coordination/ServerManager.java | 1 - .../druid/client/DirectDruidClientTest.java | 16 +++------- 16 files changed, 88 insertions(+), 103 deletions(-) diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java index a1fa77dabd5..72cce700a6d 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryRunnerFactory.java @@ -41,21 +41,6 @@ import java.util.concurrent.ExecutorService; public class SelectQueryRunnerFactory implements QueryRunnerFactory, SelectQuery> { - public static SelectQueryRunnerFactory create(ObjectMapper jsonMapper) - { - return new SelectQueryRunnerFactory( - new SelectQueryQueryToolChest(new QueryConfig(), jsonMapper), - new SelectQueryEngine(), - new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - } - } - ); - } - private final SelectQueryQueryToolChest toolChest; private final SelectQueryEngine engine; private final QueryWatcher queryWatcher; diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 726bc20bb43..724d4818226 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -41,25 +41,6 @@ import java.util.concurrent.ExecutorService; public class TimeseriesQueryRunnerFactory implements QueryRunnerFactory, TimeseriesQuery> { - /** - * Use only for testing - * @return - */ - public static TimeseriesQueryRunnerFactory create() - { - return new TimeseriesQueryRunnerFactory( - new TimeseriesQueryQueryToolChest(new QueryConfig()), - new TimeseriesQueryEngine(), - new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - } - } - ); - } - private final TimeseriesQueryQueryToolChest toolChest; private final TimeseriesQueryEngine engine; private final QueryWatcher queryWatcher; diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 954cb4fd5ee..55c29752ac1 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -55,7 +55,7 @@ import java.util.List; public class QueryRunnerTestHelper { - public static final QueryWatcher DUMMY_QUERYWATCHER = new QueryWatcher() + public static final QueryWatcher NOOP_QUERYWATCHER = new QueryWatcher() { @Override public void registerQuery(Query query, ListenableFuture future) diff --git a/processing/src/test/java/io/druid/query/TestQueryRunners.java b/processing/src/test/java/io/druid/query/TestQueryRunners.java index 90d394f3e3b..f50e81d038e 100644 --- a/processing/src/test/java/io/druid/query/TestQueryRunners.java +++ b/processing/src/test/java/io/druid/query/TestQueryRunners.java @@ -7,6 +7,8 @@ import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.SearchQueryRunnerFactory; import io.druid.query.search.search.SearchQueryConfig; import io.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.topn.TopNQueryConfig; import io.druid.query.topn.TopNQueryQueryToolChest; @@ -44,7 +46,7 @@ public class TestQueryRunners QueryRunnerFactory factory = new TopNQueryRunnerFactory( pool, new TopNQueryQueryToolChest(topNConfig), - QueryRunnerTestHelper.DUMMY_QUERYWATCHER + QueryRunnerTestHelper.NOOP_QUERYWATCHER ); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), @@ -56,7 +58,12 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() @@ -67,14 +74,7 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig()), new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }); + QueryRunnerFactory factory = new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig()), QueryRunnerTestHelper.NOOP_QUERYWATCHER); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() @@ -85,18 +85,10 @@ public class TestQueryRunners Segment adapter ) { - QueryRunnerFactory factory = new TimeBoundaryQueryRunnerFactory(new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }); + QueryRunnerFactory factory = new TimeBoundaryQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER); return new FinalizeResultsQueryRunner( factory.createRunner(adapter), factory.getToolchest() ); } - } diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java index 0740333eed5..c69ee1c5a27 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryRunnerTest.java @@ -59,14 +59,10 @@ public class SearchQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(new SearchQueryConfig()), new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }) + new SearchQueryRunnerFactory( + new SearchQueryQueryToolChest(new SearchQueryConfig()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) ); } diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index 5015239870e..07f99165873 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -22,11 +22,15 @@ package io.druid.query.select; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.ISE; import com.metamx.common.guava.Sequences; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Query; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.TableDataSource; import io.druid.query.filter.SelectorDimFilter; @@ -54,7 +58,11 @@ public class SelectQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - SelectQueryRunnerFactory.create(new DefaultObjectMapper()) + new SelectQueryRunnerFactory( + new SelectQueryQueryToolChest(new QueryConfig(), new DefaultObjectMapper()), + new SelectQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) ); } diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index de5ac1281b2..7bc499dca80 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -46,14 +46,7 @@ public class TimeBoundaryQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - new TimeBoundaryQueryRunnerFactory(new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }) + new TimeBoundaryQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER) ); } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java index 36e1fc13955..17d61908c3c 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeSeriesUnionQueryRunnerTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -46,7 +47,11 @@ public class TimeSeriesUnionQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeUnionQueryRunners( - TimeseriesQueryRunnerFactory.create() + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) ); } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index d1497a19026..67c91b4be40 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -28,8 +28,10 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -87,7 +89,12 @@ public class TimeseriesQueryRunnerBonusTest private static List> runTimeseriesCount(IncrementalIndex index) { - final QueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + final QueryRunner> runner = makeQueryRunner( factory, new IncrementalIndexSegment(index, null) diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index cfc26c4326a..708a7de1054 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -26,6 +26,7 @@ import com.metamx.common.guava.Sequences; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -62,7 +63,11 @@ public class TimeseriesQueryRunnerTest public static Collection constructorFeeder() throws IOException { return QueryRunnerTestHelper.makeQueryRunners( - TimeseriesQueryRunnerFactory.create() + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) ); } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index f06258a9a09..09d383168cf 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -23,15 +23,12 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.query.BySegmentResultValueClass; import io.druid.query.Druids; -import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; @@ -72,7 +69,7 @@ public class TopNQueryRunnerTest new TopNQueryRunnerFactory( TestQueryRunners.getPool(), new TopNQueryQueryToolChest(new TopNQueryConfig()), - QueryRunnerTestHelper.DUMMY_QUERYWATCHER + QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) ); @@ -90,7 +87,7 @@ public class TopNQueryRunnerTest } ), new TopNQueryQueryToolChest(new TopNQueryConfig()), - QueryRunnerTestHelper.DUMMY_QUERYWATCHER + QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) ); diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index d2e3e5dea73..7dc7b645cad 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -23,12 +23,9 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListenableFuture; import io.druid.collections.StupidPool; -import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; @@ -69,7 +66,7 @@ public class TopNUnionQueryTest new TopNQueryRunnerFactory( TestQueryRunners.getPool(), new TopNQueryQueryToolChest(new TopNQueryConfig()), - QueryRunnerTestHelper.DUMMY_QUERYWATCHER + QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) ); @@ -87,7 +84,7 @@ public class TopNUnionQueryTest } ), new TopNQueryQueryToolChest(new TopNQueryConfig()), - QueryRunnerTestHelper.DUMMY_QUERYWATCHER + QueryRunnerTestHelper.NOOP_QUERYWATCHER ) ) ); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index 0eb327972ee..c8155526a89 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -29,13 +29,17 @@ import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.SpatialDimFilter; import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.IncrementalIndexSegment; @@ -434,7 +438,12 @@ public class SpatialFilterBonusTest ) ); try { - TimeseriesQueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + QueryRunner runner = new FinalizeResultsQueryRunner( factory.createRunner(segment), factory.getToolchest() @@ -516,7 +525,12 @@ public class SpatialFilterBonusTest ) ); try { - TimeseriesQueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + QueryRunner runner = new FinalizeResultsQueryRunner( factory.createRunner(segment), factory.getToolchest() @@ -528,4 +542,4 @@ public class SpatialFilterBonusTest throw Throwables.propagate(e); } } -} \ No newline at end of file +} diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index d342c12c577..84df58a260d 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -29,13 +29,17 @@ import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.granularity.QueryGranularity; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.SpatialDimFilter; import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.IncrementalIndexSegment; @@ -464,7 +468,12 @@ public class SpatialFilterTest ) ); try { - TimeseriesQueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + QueryRunner runner = new FinalizeResultsQueryRunner( factory.createRunner(segment), factory.getToolchest() @@ -546,7 +555,12 @@ public class SpatialFilterTest ) ); try { - TimeseriesQueryRunnerFactory factory = TimeseriesQueryRunnerFactory.create(); + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest(new QueryConfig()), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + QueryRunner runner = new FinalizeResultsQueryRunner( factory.createRunner(segment), factory.getToolchest() @@ -558,4 +572,4 @@ public class SpatialFilterTest throw Throwables.propagate(e); } } -} \ No newline at end of file +} diff --git a/server/src/main/java/io/druid/server/coordination/ServerManager.java b/server/src/main/java/io/druid/server/coordination/ServerManager.java index 537cc0145fb..6bc703297e5 100644 --- a/server/src/main/java/io/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/io/druid/server/coordination/ServerManager.java @@ -21,7 +21,6 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; diff --git a/server/src/test/java/io/druid/client/DirectDruidClientTest.java b/server/src/test/java/io/druid/client/DirectDruidClientTest.java index b2396688900..4ad8ca5cd51 100644 --- a/server/src/test/java/io/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/io/druid/client/DirectDruidClientTest.java @@ -38,6 +38,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryInterruptedException; +import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryWatcher; import io.druid.query.ReflectionQueryToolChestWarehouse; import io.druid.query.Result; @@ -60,15 +61,6 @@ import java.util.List; public class DirectDruidClientTest { - public static final QueryWatcher DUMMY_WATCHER = new QueryWatcher() - { - @Override - public void registerQuery(Query query, ListenableFuture future) - { - - } - }; - @Test public void testRun() throws Exception { @@ -101,14 +93,14 @@ public class DirectDruidClientTest DirectDruidClient client1 = new DirectDruidClient( new ReflectionQueryToolChestWarehouse(), - DUMMY_WATCHER, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, new DefaultObjectMapper(), httpClient, "foo" ); DirectDruidClient client2 = new DirectDruidClient( new ReflectionQueryToolChestWarehouse(), - DUMMY_WATCHER, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, new DefaultObjectMapper(), httpClient, "foo2" @@ -196,7 +188,7 @@ public class DirectDruidClientTest DirectDruidClient client1 = new DirectDruidClient( new ReflectionQueryToolChestWarehouse(), - DUMMY_WATCHER, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, new DefaultObjectMapper(), httpClient, "foo" From 8f7fd93491a6eb97661ebc7dbe7bf70683adeed2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 9 Jun 2014 14:34:44 -0700 Subject: [PATCH 029/270] add comments --- .../java/io/druid/query/QueryWatcher.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/processing/src/main/java/io/druid/query/QueryWatcher.java b/processing/src/main/java/io/druid/query/QueryWatcher.java index 0a76a54f23a..36394b11e34 100644 --- a/processing/src/main/java/io/druid/query/QueryWatcher.java +++ b/processing/src/main/java/io/druid/query/QueryWatcher.java @@ -21,7 +21,27 @@ package io.druid.query; import com.google.common.util.concurrent.ListenableFuture; +/** + * This interface is in a very early stage and should not be considered stable. + * + * The purpose of the QueryWatcher is to give overall visibility into queries running + * or pending at the QueryRunner level. This is currently used to cancel all the + * parts of a pending query, but may be expanded in the future to offer more direct + * visibility into query execution and resource usage. + * + * QueryRunners executing any computation asynchronously must register their queries + * with the QueryWatcher. + * + */ public interface QueryWatcher { + /** + * QueryRunners must use this method to register any pending queries. + * + * The given future may have cancel(true) called at any time, if cancellation of this query has been requested. + * + * @param query a query, which may be a subset of a larger query, as long as the underlying queryId is unchanged + * @param future the future holding the execution status of the query + */ public void registerQuery(Query query, ListenableFuture future); } From 1fb9b21cf0c5206e1f8ccf6ea1d5f771157488e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 9 Jun 2014 17:22:50 -0700 Subject: [PATCH 030/270] async servlet delete support + cleanup --- .../io/druid/client/RoutingDruidClient.java | 22 +- .../server/AsyncQueryForwardingServlet.java | 529 +++++++++--------- 2 files changed, 280 insertions(+), 271 deletions(-) diff --git a/server/src/main/java/io/druid/client/RoutingDruidClient.java b/server/src/main/java/io/druid/client/RoutingDruidClient.java index 10170fcfb9e..79ae1c16f6d 100644 --- a/server/src/main/java/io/druid/client/RoutingDruidClient.java +++ b/server/src/main/java/io/druid/client/RoutingDruidClient.java @@ -22,6 +22,7 @@ package io.druid.client; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMultimap; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -30,7 +31,9 @@ import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.HttpResponseHandler; import io.druid.guice.annotations.Client; import io.druid.query.Query; +import io.druid.server.QueryResource; import io.druid.server.router.Router; +import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.handler.codec.http.HttpHeaders; import javax.inject.Inject; @@ -68,7 +71,7 @@ public class RoutingDruidClient return openConnections.get(); } - public ListenableFuture post( + public ListenableFuture postQuery( String url, Query query, HttpResponseHandler responseHandler @@ -81,7 +84,7 @@ public class RoutingDruidClient future = httpClient .post(new URL(url)) .setContent(objectMapper.writeValueAsBytes(query)) - .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? "application/smile" : "application/json") + .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? QueryResource.APPLICATION_SMILE : QueryResource.APPLICATION_JSON) .go(responseHandler); openConnections.getAndIncrement(); @@ -125,4 +128,19 @@ public class RoutingDruidClient throw Throwables.propagate(e); } } + + public ListenableFuture delete( + String url, + HttpResponseHandler responseHandler + ) + { + try { + return httpClient + .delete(new URL(url)) + .go(responseHandler); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } } diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 5360d529ad3..3d82cc013ea 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -21,7 +21,11 @@ package io.druid.server; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Function; import com.google.common.base.Joiner; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableMap; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; @@ -37,19 +41,20 @@ import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.handler.codec.http.HttpChunk; +import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpResponse; import org.joda.time.DateTime; +import javax.annotation.Nullable; import javax.servlet.AsyncContext; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; import javax.servlet.annotation.WebServlet; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.io.OutputStream; -import java.nio.charset.Charset; +import java.util.Map; import java.util.UUID; /** @@ -59,8 +64,6 @@ import java.util.UUID; public class AsyncQueryForwardingServlet extends HttpServlet { private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); - private static final Charset UTF8 = Charset.forName("UTF-8"); - private static final String DISPATCHED = "dispatched"; private static final Joiner COMMA_JOIN = Joiner.on(","); private final ObjectMapper jsonMapper; @@ -88,275 +91,161 @@ public class AsyncQueryForwardingServlet extends HttpServlet } @Override - protected void doGet(final HttpServletRequest req, final HttpServletResponse resp) + protected void doGet(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException { - OutputStream out = null; - AsyncContext ctx = null; - - try { - ctx = req.startAsync(req, resp); - final AsyncContext asyncContext = ctx; - - if (req.getAttribute(DISPATCHED) != null) { - return; - } - - out = resp.getOutputStream(); - final OutputStream outputStream = out; - - final String host = hostFinder.getDefaultHost(); - - final HttpResponseHandler responseHandler = new HttpResponseHandler() - { - @Override - public ClientResponse handleResponse(HttpResponse response) + final AsyncContext asyncContext = req.startAsync(req, res); + asyncContext.start( + new Runnable() { - resp.setStatus(response.getStatus().getCode()); - resp.setContentType("application/json"); - - try { - ChannelBuffer buf = response.getContent(); - buf.readBytes(outputStream, buf.readableBytes()); - } - catch (Exception e) { - asyncContext.complete(); - throw Throwables.propagate(e); - } - - return ClientResponse.finished(outputStream); - } - - @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk chunk - ) - { - try { - ChannelBuffer buf = chunk.getContent(); - buf.readBytes(outputStream, buf.readableBytes()); - } - catch (Exception e) { - asyncContext.complete(); - throw Throwables.propagate(e); - } - return clientResponse; - } - - @Override - public ClientResponse done(ClientResponse clientResponse) - { - final OutputStream obj = clientResponse.getObj(); - try { - resp.flushBuffer(); - outputStream.close(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - finally { - asyncContext.complete(); - } - - return ClientResponse.finished(obj); - } - - @Override - public void exceptionCaught( - ClientResponse clientResponse, - Throwable e - ) - { - handleException(resp, asyncContext, e); - } - }; - - asyncContext.start( - new Runnable() + @Override + public void run() { - @Override - public void run() - { - routingDruidClient.get(makeUrl(host, req), responseHandler); + try { + final HttpResponseHandler responseHandler = new PassthroughHttpResponseHandler(asyncContext, jsonMapper); + + final String host = hostFinder.getDefaultHost(); + routingDruidClient.get(makeUrl(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler); + } + catch (Exception e) { + handleException(jsonMapper, asyncContext, e); } } - ); - - asyncContext.dispatch(); - req.setAttribute(DISPATCHED, true); - } - catch (Exception e) { - handleException(resp, ctx, e); - } + } + ); } @Override - protected void doPost( - final HttpServletRequest req, final HttpServletResponse resp - ) throws ServletException, IOException + protected void doDelete(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException { - final long start = System.currentTimeMillis(); - Query query = null; - String queryId; - - final boolean isSmile = "application/smile".equals(req.getContentType()); - - final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - - OutputStream out = null; - AsyncContext ctx = null; - - try { - ctx = req.startAsync(req, resp); - final AsyncContext asyncContext = ctx; - - if (req.getAttribute(DISPATCHED) != null) { - return; - } - - query = objectMapper.readValue(req.getInputStream(), Query.class); - queryId = query.getId(); - if (queryId == null) { - queryId = UUID.randomUUID().toString(); - query = query.withId(queryId); - } - - if (log.isDebugEnabled()) { - log.debug("Got query [%s]", query); - } - - out = resp.getOutputStream(); - final OutputStream outputStream = out; - - final String host = hostFinder.getHost(query); - - final Query theQuery = query; - final String theQueryId = queryId; - - final HttpResponseHandler responseHandler = new HttpResponseHandler() - { - @Override - public ClientResponse handleResponse(HttpResponse response) + final AsyncContext asyncContext = req.startAsync(req, res); + asyncContext.start( + new Runnable() { - resp.setStatus(response.getStatus().getCode()); - resp.setContentType("application/x-javascript"); - - try { - ChannelBuffer buf = response.getContent(); - buf.readBytes(outputStream, buf.readableBytes()); - } - catch (Exception e) { - asyncContext.complete(); - throw Throwables.propagate(e); - } - return ClientResponse.finished(outputStream); - } - - @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk chunk - ) - { - try { - ChannelBuffer buf = chunk.getContent(); - buf.readBytes(outputStream, buf.readableBytes()); - } - catch (Exception e) { - asyncContext.complete(); - throw Throwables.propagate(e); - } - return clientResponse; - } - - @Override - public ClientResponse done(ClientResponse clientResponse) - { - final long requestTime = System.currentTimeMillis() - start; - - log.debug("Request time: %d", requestTime); - - emitter.emit( - new ServiceMetricEvent.Builder() - .setUser2(DataSourceUtil.getMetricName(theQuery.getDataSource())) - .setUser4(theQuery.getType()) - .setUser5(COMMA_JOIN.join(theQuery.getIntervals())) - .setUser6(String.valueOf(theQuery.hasFilters())) - .setUser7(req.getRemoteAddr()) - .setUser8(theQueryId) - .setUser9(theQuery.getDuration().toPeriod().toStandardMinutes().toString()) - .build("request/time", requestTime) - ); - - final OutputStream obj = clientResponse.getObj(); - try { - requestLogger.log( - new RequestLogLine( - new DateTime(), - req.getRemoteAddr(), - theQuery, - new QueryStats(ImmutableMap.of("request/time", requestTime, "success", true)) - ) - ); - - resp.flushBuffer(); - outputStream.close(); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - finally { - asyncContext.complete(); - } - - return ClientResponse.finished(obj); - } - - @Override - public void exceptionCaught( - ClientResponse clientResponse, - Throwable e - ) - { - handleException(resp, asyncContext, e); - } - }; - - asyncContext.start( - new Runnable() + @Override + public void run() { - @Override - public void run() - { - routingDruidClient.post(makeUrl(host, req), theQuery, responseHandler); + try { + final HttpResponseHandler responseHandler = new PassthroughHttpResponseHandler(asyncContext, jsonMapper); + + final String host = hostFinder.getDefaultHost(); + routingDruidClient.delete(makeUrl(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler); + } + catch (Exception e) { + handleException(jsonMapper, asyncContext, e); } } - ); + } + ); + } - asyncContext.dispatch(); - req.setAttribute(DISPATCHED, true); - } - catch (Exception e) { - handleException(resp, ctx, e); + @Override + protected void doPost(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException + { + final long start = System.currentTimeMillis(); + final AsyncContext asyncContext = req.startAsync(req, res); + asyncContext.start( + new Runnable() + { + @Override + public void run() + { + final HttpServletRequest request = (HttpServletRequest) asyncContext.getRequest(); - try { - requestLogger.log( - new RequestLogLine( - new DateTime(), - req.getRemoteAddr(), - query, - new QueryStats(ImmutableMap.of("success", false, "exception", e.toString())) - ) - ); - } - catch (Exception e2) { - log.error(e2, "Unable to log query [%s]!", query); - } + final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(request.getContentType()); + final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - log.makeAlert(e, "Exception handling request") - .addData("query", query) - .addData("peer", req.getRemoteAddr()) - .emit(); - } + Query inputQuery = null; + try { + inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); + if (inputQuery.getId() == null) { + inputQuery = inputQuery.withId(UUID.randomUUID().toString()); + } + final Query query = inputQuery; + + if (log.isDebugEnabled()) { + log.debug("Got query [%s]", inputQuery); + } + + final HttpResponseHandler responseHandler = new PassthroughHttpResponseHandler( + asyncContext, + objectMapper + ) + { + @Override + public ClientResponse done(ClientResponse clientResponse) + { + final long requestTime = System.currentTimeMillis() - start; + log.debug("Request time: %d", requestTime); + + emitter.emit( + new ServiceMetricEvent.Builder() + .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) + .setUser4(query.getType()) + .setUser5(COMMA_JOIN.join(query.getIntervals())) + .setUser6(String.valueOf(query.hasFilters())) + .setUser7(request.getRemoteAddr()) + .setUser8(query.getId()) + .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) + .build("request/time", requestTime) + ); + + try { + requestLogger.log( + new RequestLogLine( + new DateTime(), + request.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "request/time", + requestTime, + "success", + true + ) + ) + ) + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + return super.done(clientResponse); + } + }; + + routingDruidClient.postQuery( + makeUrl(hostFinder.getHost(inputQuery), request), + inputQuery, + responseHandler + ); + } + catch (Exception e) { + handleException(objectMapper, asyncContext, e); + + try { + requestLogger.log( + new RequestLogLine( + new DateTime(), + request.getRemoteAddr(), + inputQuery, + new QueryStats(ImmutableMap.of("success", false, "exception", e.toString())) + ) + ); + } + catch (Exception logError) { + log.error(logError, "Unable to log query [%s]!", inputQuery); + } + + log.makeAlert(e, "Exception handling request") + .addData("query", inputQuery) + .addData("peer", request.getRemoteAddr()) + .emit(); + } + } + } + ); } private String makeUrl(final String host, final HttpServletRequest req) @@ -370,24 +259,126 @@ public class AsyncQueryForwardingServlet extends HttpServlet return String.format("http://%s%s?%s", host, requestURI, queryString); } - private static void handleException(HttpServletResponse resp, AsyncContext ctx, Throwable e) + private static void handleException(ObjectMapper objectMapper, AsyncContext asyncContext, Throwable exception) { try { - final ServletOutputStream out = resp.getOutputStream(); - if (!resp.isCommitted()) { - resp.setStatus(500); - resp.resetBuffer(); - out.write((e.getMessage() == null) ? "Exception null".getBytes(UTF8) : e.getMessage().getBytes(UTF8)); - out.write("\n".getBytes(UTF8)); + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + if (!response.isCommitted()) { + final String errorMessage = exception.getMessage() == null ? "null exception" : exception.getMessage(); + + response.resetBuffer(); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + objectMapper.writeValue( + response.getOutputStream(), + ImmutableMap.of( + "error", errorMessage + ) + ); + } + response.flushBuffer(); + } + catch (IOException e) { + Throwables.propagate(e); + } + finally { + asyncContext.complete(); + } + } + + private static class PassthroughHttpResponseHandler implements HttpResponseHandler + { + private final AsyncContext asyncContext; + private final ObjectMapper objectMapper; + private final OutputStream outputStream; + + public PassthroughHttpResponseHandler(AsyncContext asyncContext, ObjectMapper objectMapper) throws IOException + { + this.asyncContext = asyncContext; + this.objectMapper = objectMapper; + this.outputStream = asyncContext.getResponse().getOutputStream(); + } + + protected void copyStatusHeaders(HttpResponse clientResponse) + { + final HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(clientResponse.getStatus().getCode()); + response.setContentType(clientResponse.headers().get(HttpHeaders.Names.CONTENT_TYPE)); + + FluentIterable.from(clientResponse.headers().entries()) + .filter(new Predicate>() + { + @Override + public boolean apply(@Nullable Map.Entry input) + { + return input.getKey().startsWith("X-Druid"); + } + } + ) + .transform( + new Function, Object>() + { + @Nullable + @Override + public Object apply(@Nullable Map.Entry input) + { + response.setHeader(input.getKey(), input.getValue()); + return null; + } + } + ) + .allMatch(Predicates.alwaysTrue()); + } + + @Override + public ClientResponse handleResponse(HttpResponse clientResponse) + { + copyStatusHeaders(clientResponse); + + try { + ChannelBuffer buf = clientResponse.getContent(); + buf.readBytes(outputStream, buf.readableBytes()); + } + catch (Exception e) { + throw Throwables.propagate(e); } - if (ctx != null) { - ctx.complete(); - } - resp.flushBuffer(); + return ClientResponse.finished(outputStream); } - catch (IOException e1) { - Throwables.propagate(e1); + + @Override + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk + ) + { + try { + ChannelBuffer buf = chunk.getContent(); + buf.readBytes(outputStream, buf.readableBytes()); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + return clientResponse; + } + + @Override + public ClientResponse done(ClientResponse clientResponse) + { + asyncContext.complete(); + return ClientResponse.finished(clientResponse.getObj()); + } + + @Override + public void exceptionCaught( + ClientResponse clientResponse, + Throwable e + ) + { + // throwing an exception here may cause resource leak + try { + handleException(objectMapper, asyncContext, e); + } catch(Exception err) { + log.error(err, "Unable to handle exception response"); + } } } } From ae0e36f3ebf91aa66a45062c5a44ebd88d643b45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 2 Jun 2014 15:42:55 -0700 Subject: [PATCH 031/270] warn glacier objects cannot be moved --- pom.xml | 9 ++++++++- .../io/druid/storage/s3/S3DataSegmentMover.java | 4 ++++ .../io/druid/storage/s3/S3StorageDruidModule.java | 5 ----- .../io/druid/storage/s3/S3DataSegmentMoverTest.java | 13 +++++++++++++ 4 files changed, 25 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 2116d7f617f..1c9c5664e47 100644 --- a/pom.xml +++ b/pom.xml @@ -393,7 +393,14 @@ net.java.dev.jets3t jets3t - 0.9.0 + 0.9.1 + + + + com.centerkey.utils + barebonesbrowserlaunch + + org.apache.httpcomponents diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index d259ab185d2..ea585a0be44 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -120,6 +120,10 @@ public class S3DataSegmentMover implements DataSegmentMover if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { if (s3Bucket.equals(targetS3Bucket) && s3Path.equals(targetS3Path)) { log.info("No need to move file[s3://%s/%s] onto itself", s3Bucket, s3Path); + } else if (s3Client.getObjectDetails(s3Bucket, s3Path) + .getStorageClass() + .equals(S3Object.STORAGE_CLASS_GLACIER)) { + log.warn("Cannot move file[s3://%s/%s] of storage class glacier."); } else { log.info( "Moving file[s3://%s/%s] to [s3://%s/%s]", diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index d30f49f976a..21faf74db91 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -73,11 +73,6 @@ public class S3StorageDruidModule implements DruidModule @LazySingleton public RestS3Service getRestS3Service(AWSCredentials credentials) { - try { return new RestS3Service(credentials); - } - catch (S3ServiceException e) { - throw new ProvisionException("Unable to create a RestS3Service", e); - } } } diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java index c13d22de5f1..9497d9a05fc 100644 --- a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java +++ b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java @@ -30,6 +30,7 @@ import io.druid.timeline.partition.NoneShardSpec; import org.jets3t.service.S3ServiceException; import org.jets3t.service.ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; import org.jets3t.service.model.S3Object; import org.jets3t.service.model.StorageObject; import org.joda.time.Interval; @@ -131,6 +132,18 @@ public class S3DataSegmentMoverTest return (objects != null && objects.contains(objectKey)); } + @Override + public StorageObject getObjectDetails(String bucketName, String objectKey) throws ServiceException + { + if (isObjectInBucket(bucketName, objectKey)) { + final S3Object object = new S3Object(objectKey); + object.setStorageClass(S3Object.STORAGE_CLASS_STANDARD); + return object; + } else { + return null; + } + } + @Override public Map moveObject( String sourceBucketName, From 6550cb1776101232cd17086052d46f00b2e3fd0b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 10 Jun 2014 10:15:31 -0700 Subject: [PATCH 032/270] groupBy query cancellation --- .../query/GroupByParallelQueryRunner.java | 108 +++++++++++------- .../groupby/GroupByQueryRunnerFactory.java | 41 +++++-- .../query/groupby/GroupByQueryRunnerTest.java | 1 + .../GroupByTimeseriesQueryRunnerTest.java | 1 + 4 files changed, 105 insertions(+), 46 deletions(-) diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 51c663c6a2e..c9b14b6314b 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -26,6 +26,10 @@ import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; @@ -39,37 +43,44 @@ import io.druid.segment.incremental.IncrementalIndex; import java.util.Arrays; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class GroupByParallelQueryRunner implements QueryRunner { private static final Logger log = new Logger(GroupByParallelQueryRunner.class); private final Iterable> queryables; - private final ExecutorService exec; + private final ListeningExecutorService exec; private final Ordering ordering; private final Supplier configSupplier; + private final QueryWatcher queryWatcher; + public GroupByParallelQueryRunner( ExecutorService exec, Ordering ordering, Supplier configSupplier, + QueryWatcher queryWatcher, QueryRunner... queryables ) { - this(exec, ordering, configSupplier, Arrays.asList(queryables)); + this(exec, ordering, configSupplier, queryWatcher, Arrays.asList(queryables)); } public GroupByParallelQueryRunner( ExecutorService exec, Ordering ordering, Supplier configSupplier, + QueryWatcher queryWatcher, Iterable> queryables ) { - this.exec = exec; + this.exec = MoreExecutors.listeningDecorator(exec); this.ordering = ordering; + this.queryWatcher = queryWatcher; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); this.configSupplier = configSupplier; } @@ -88,48 +99,67 @@ public class GroupByParallelQueryRunner implements QueryRunner if (Iterables.isEmpty(queryables)) { log.warn("No queryables found."); } - List> futures = Lists.newArrayList( - Iterables.transform( - queryables, - new Function, Future>() - { - @Override - public Future apply(final QueryRunner input) - { - return exec.submit( - new AbstractPrioritizedCallable(priority) - { - @Override - public Boolean call() throws Exception - { - try { - input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); - return true; + ListenableFuture> futures = Futures.allAsList( + Lists.newArrayList( + Iterables.transform( + queryables, + new Function, ListenableFuture>() + { + @Override + public ListenableFuture apply(final QueryRunner input) + { + return exec.submit( + new AbstractPrioritizedCallable(priority) + { + @Override + public Boolean call() throws Exception + { + try { + input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + return true; + } + catch (QueryInterruptedException e) { + throw Throwables.propagate(e); + } + catch (Exception e) { + log.error(e, "Exception with one of the sequences!"); + throw Throwables.propagate(e); + } + } } - catch (Exception e) { - log.error(e, "Exception with one of the sequences!"); - throw Throwables.propagate(e); - } - } - } - ); - } - } + ); + } + } + ) ) ); // Let the runners complete - for (Future future : futures) { - try { - future.get(); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - catch (ExecutionException e) { - throw new RuntimeException(e); + try { + queryWatcher.registerQuery(query, futures); + final Number timeout = query.getContextValue("timeout", (Number) null); + if(timeout == null) { + futures.get(); + } else { + futures.get(timeout.longValue(), TimeUnit.MILLISECONDS); } } + catch (InterruptedException e) { + log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); + futures.cancel(true); + throw new QueryInterruptedException("Query interrupted"); + } + catch(CancellationException e) { + throw new QueryInterruptedException("Query cancelled"); + } + catch(TimeoutException e) { + log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); + futures.cancel(true); + throw new QueryInterruptedException("Query timeout"); + } + catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } return Sequences.simple(indexAccumulatorPair.lhs.iterableWithPostAggregations(null)); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java index 714aad37925..e8634089c2f 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -24,42 +24,55 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; import com.google.common.primitives.Longs; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.common.logger.Logger; import io.druid.data.input.Row; import io.druid.query.ConcatQueryRunner; import io.druid.query.GroupByParallelQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; /** */ public class GroupByQueryRunnerFactory implements QueryRunnerFactory { private final GroupByQueryEngine engine; + private final QueryWatcher queryWatcher; private final Supplier config; private final GroupByQueryQueryToolChest toolChest; + private static final Logger log = new Logger(GroupByQueryRunnerFactory.class); + @Inject public GroupByQueryRunnerFactory( GroupByQueryEngine engine, + QueryWatcher queryWatcher, Supplier config, GroupByQueryQueryToolChest toolChest ) { this.engine = engine; + this.queryWatcher = queryWatcher; this.config = config; this.toolChest = toolChest; } @@ -71,8 +84,10 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory mergeRunners(final ExecutorService queryExecutor, Iterable> queryRunners) + public QueryRunner mergeRunners(final ExecutorService exec, Iterable> queryRunners) { + // mergeRunners should take ListeningExecutorService at some point + final ListeningExecutorService queryExecutor = MoreExecutors.listeningDecorator(exec); if (config.get().isSingleThreaded()) { return new ConcatQueryRunner( Sequences.map( @@ -88,7 +103,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory run(final Query query) { - Future> future = queryExecutor.submit( + ListenableFuture> future = queryExecutor.submit( new Callable>() { @Override @@ -102,13 +117,25 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory Date: Tue, 10 Jun 2014 10:24:18 -0700 Subject: [PATCH 033/270] segmentMetadata query cancellation --- .../SegmentMetadataQueryRunnerFactory.java | 48 +++++++++++++++---- .../query/metadata/SegmentAnalyzerTest.java | 3 +- .../metadata/SegmentMetadataQueryTest.java | 2 +- 3 files changed, 42 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index 9fa393c5c2f..c8e7208638c 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -22,15 +22,21 @@ package io.druid.query.metadata; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Maps; -import com.metamx.common.guava.ExecutorExecutingSequence; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Inject; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import com.metamx.common.logger.Logger; import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.ConcatQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryInterruptedException; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; +import io.druid.query.QueryWatcher; import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.ColumnIncluderator; import io.druid.query.metadata.metadata.SegmentAnalysis; @@ -40,16 +46,27 @@ import io.druid.segment.Segment; import java.util.Arrays; import java.util.Map; -import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory { private static final SegmentAnalyzer analyzer = new SegmentAnalyzer(); - private static final SegmentMetadataQueryQueryToolChest toolChest = new SegmentMetadataQueryQueryToolChest(); + private static final Logger log = new Logger(SegmentMetadataQueryRunnerFactory.class); + + private final QueryWatcher queryWatcher; + + @Inject + public SegmentMetadataQueryRunnerFactory( + QueryWatcher queryWatcher + ) + { + this.queryWatcher = queryWatcher; + } @Override public QueryRunner createRunner(final Segment segment) @@ -101,9 +118,10 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory mergeRunners( - final ExecutorService queryExecutor, Iterable> queryRunners + ExecutorService exec, Iterable> queryRunners ) { + final ListeningExecutorService queryExecutor = MoreExecutors.listeningDecorator(exec); return new ConcatQueryRunner( Sequences.map( Sequences.simple(queryRunners), @@ -118,7 +136,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory run(final Query query) { final int priority = query.getContextPriority(0); - Future> future = queryExecutor.submit( + ListenableFuture> future = queryExecutor.submit( new AbstractPrioritizedCallable>(priority) { @Override @@ -129,13 +147,25 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory getSegmentAnalysises(Segment index) { final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner( - (QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(), index + (QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER), index ); final SegmentMetadataQuery query = new SegmentMetadataQuery( diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index 5d596292348..ed1740460f8 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -45,7 +45,7 @@ public class SegmentMetadataQueryTest { @SuppressWarnings("unchecked") private final QueryRunner runner = makeQueryRunner( - new SegmentMetadataQueryRunnerFactory() + new SegmentMetadataQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER) ); private ObjectMapper mapper = new DefaultObjectMapper(); From ababbcadfdca77720717265979cca6df1280ea95 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 10 Jun 2014 15:01:44 -0700 Subject: [PATCH 034/270] fix package groupId --- pom.xml | 2 +- server/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index fcc95f6d28f..a78251143a5 100644 --- a/pom.xml +++ b/pom.xml @@ -282,7 +282,7 @@ 1 - org.glassfish.web + org.glassfish javax.el 3.0.0 diff --git a/server/pom.xml b/server/pom.xml index f01e920cb77..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -59,7 +59,7 @@ javax.inject - org.glassfish.web + org.glassfish javax.el From cde86d81e20fd14255eb95fe7e39f058657e3b04 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 10 Jun 2014 15:50:13 -0700 Subject: [PATCH 035/270] make having and limitspec work for nested queries --- .../io/druid/query/groupby/GroupByQuery.java | 41 +++++---- .../groupby/GroupByQueryQueryToolChest.java | 30 +++--- .../groupby/orderby/DefaultLimitSpec.java | 65 +++++++++---- .../query/groupby/orderby/LimitSpec.java | 8 +- .../query/groupby/orderby/NoopLimitSpec.java | 6 ++ .../query/groupby/GroupByQueryRunnerTest.java | 92 +++++++++++++++++++ 6 files changed, 192 insertions(+), 50 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index d9431849ea2..98ac83e32cd 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -72,7 +72,7 @@ public class GroupByQuery extends BaseQuery private final List aggregatorSpecs; private final List postAggregatorSpecs; - private final Function, Sequence> orderByLimitFn; + private final Function, Sequence> limitFn; @JsonCreator public GroupByQuery( @@ -85,8 +85,9 @@ public class GroupByQuery extends BaseQuery @JsonProperty("postAggregations") List postAggregatorSpecs, @JsonProperty("having") HavingSpec havingSpec, @JsonProperty("limitSpec") LimitSpec limitSpec, - @JsonProperty("orderBy") LimitSpec orderBySpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + // Backwards compatible + @JsonProperty("orderBy") LimitSpec orderBySpec ) { super(dataSource, querySegmentSpec, context); @@ -129,7 +130,7 @@ public class GroupByQuery extends BaseQuery ); } - orderByLimitFn = postProcFn; + limitFn = postProcFn; } /** @@ -146,7 +147,7 @@ public class GroupByQuery extends BaseQuery List postAggregatorSpecs, HavingSpec havingSpec, LimitSpec orderBySpec, - Function, Sequence> orderByLimitFn, + Function, Sequence> limitFn, Map context ) { @@ -159,7 +160,7 @@ public class GroupByQuery extends BaseQuery this.postAggregatorSpecs = postAggregatorSpecs; this.havingSpec = havingSpec; this.limitSpec = orderBySpec; - this.orderByLimitFn = orderByLimitFn; + this.limitFn = limitFn; } @JsonProperty("filter") @@ -199,7 +200,7 @@ public class GroupByQuery extends BaseQuery } @JsonProperty - public LimitSpec getOrderBy() + public LimitSpec getLimitSpec() { return limitSpec; } @@ -218,7 +219,7 @@ public class GroupByQuery extends BaseQuery public Sequence applyLimit(Sequence results) { - return orderByLimitFn.apply(results); + return limitFn.apply(results); } @Override @@ -234,7 +235,7 @@ public class GroupByQuery extends BaseQuery postAggregatorSpecs, havingSpec, limitSpec, - orderByLimitFn, + limitFn, computeOverridenContext(contextOverride) ); } @@ -252,7 +253,7 @@ public class GroupByQuery extends BaseQuery postAggregatorSpecs, havingSpec, limitSpec, - orderByLimitFn, + limitFn, getContext() ); } @@ -270,7 +271,7 @@ public class GroupByQuery extends BaseQuery postAggregatorSpecs, havingSpec, limitSpec, - orderByLimitFn, + limitFn, getContext() ); } @@ -300,7 +301,7 @@ public class GroupByQuery extends BaseQuery { dataSource = query.getDataSource(); querySegmentSpec = query.getQuerySegmentSpec(); - limitSpec = query.getOrderBy(); + limitSpec = query.getLimitSpec(); dimFilter = query.getDimFilter(); granularity = query.getGranularity(); dimensions = query.getDimensions(); @@ -504,7 +505,11 @@ public class GroupByQuery extends BaseQuery { final LimitSpec theLimitSpec; if (limitSpec == null) { - theLimitSpec = new DefaultLimitSpec(orderByColumnSpecs, limit); + if (orderByColumnSpecs.isEmpty() && limit == Integer.MAX_VALUE) { + theLimitSpec = new NoopLimitSpec(); + } else { + theLimitSpec = new DefaultLimitSpec(orderByColumnSpecs, limit); + } } else { theLimitSpec = limitSpec; } @@ -518,9 +523,9 @@ public class GroupByQuery extends BaseQuery aggregatorSpecs, postAggregatorSpecs, havingSpec, - null, theLimitSpec, - context + context, + null ); } } @@ -535,7 +540,7 @@ public class GroupByQuery extends BaseQuery ", dimensions=" + dimensions + ", aggregatorSpecs=" + aggregatorSpecs + ", postAggregatorSpecs=" + postAggregatorSpecs + - ", orderByLimitFn=" + orderByLimitFn + + ", limitFn=" + limitFn + '}'; } @@ -572,7 +577,7 @@ public class GroupByQuery extends BaseQuery if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) { return false; } - if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null) { + if (limitFn != null ? !limitFn.equals(that.limitFn) : that.limitFn != null) { return false; } if (postAggregatorSpecs != null @@ -595,7 +600,7 @@ public class GroupByQuery extends BaseQuery result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0); result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0); result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0); - result = 31 * result + (orderByLimitFn != null ? orderByLimitFn.hashCode() : 0); + result = 31 * result + (limitFn != null ? limitFn.hashCode() : 0); return result; } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 681871cfd93..09cf7e431a8 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -44,7 +44,6 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.incremental.IncrementalIndex; @@ -52,7 +51,6 @@ import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -89,7 +87,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest run(Query input) { - if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { + if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); } else { return runner.run(input); @@ -100,8 +98,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { - Sequence result; - // If there's a subquery, merge subquery results and then apply the aggregator final DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { @@ -117,25 +113,30 @@ public class GroupByQueryQueryToolChest extends QueryToolChestnewArrayList()) - .build(); + + // We need the inner incremental index to have all the columns required by the outer query + final GroupByQuery innerQuery = new GroupByQuery.Builder(query) + .setAggregatorSpecs(aggs) + .setInterval(subquery.getIntervals()) + .setPostAggregatorSpecs(Lists.newArrayList()) + .build(); + + final GroupByQuery outerQuery = new GroupByQuery.Builder(query) + .setLimitSpec(query.getLimitSpec().merge(subquery.getLimitSpec())) + .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter( makeIncrementalIndex(innerQuery, subqueryResult) ); - return engine.process(query, adapter); + return outerQuery.applyLimit(engine.process(outerQuery, adapter)); } else { - result = runner.run(query); - return postAggregate(query, makeIncrementalIndex(query, result)); + return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query)))); } } - private Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) { - Sequence sequence = Sequences.map( + return Sequences.map( Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())), new Function() { @@ -151,7 +152,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest rows) diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index eda54ea0dc3..3d78e112cb5 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -87,12 +87,17 @@ public class DefaultLimitSpec implements LimitSpec if (limit == Integer.MAX_VALUE) { return new SortingFn(ordering); - } - else { + } else { return new TopNFunction(ordering, limit); } } + @Override + public LimitSpec merge(LimitSpec other) + { + return this; + } + private Ordering makeComparator( List dimensions, List aggs, List postAggs ) @@ -200,12 +205,18 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } LimitingFn that = (LimitingFn) o; - if (limit != that.limit) return false; + if (limit != that.limit) { + return false; + } return true; } @@ -232,12 +243,18 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } SortingFn sortingFn = (SortingFn) o; - if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) return false; + if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) { + return false; + } return true; } @@ -273,13 +290,21 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } TopNFunction that = (TopNFunction) o; - if (limit != that.limit) return false; - if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) return false; + if (limit != that.limit) { + return false; + } + if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) { + return false; + } return true; } @@ -296,13 +321,21 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } DefaultLimitSpec that = (DefaultLimitSpec) o; - if (limit != that.limit) return false; - if (columns != null ? !columns.equals(that.columns) : that.columns != null) return false; + if (limit != that.limit) { + return false; + } + if (columns != null ? !columns.equals(that.columns) : that.columns != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java index 0d07f1f91c9..fa50d62016c 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java @@ -38,5 +38,11 @@ import java.util.List; }) public interface LimitSpec { - public Function, Sequence> build(List dimensions, List aggs, List postAggs); + public Function, Sequence> build( + List dimensions, + List aggs, + List postAggs + ); + + public LimitSpec merge(LimitSpec other); } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java index d975e24a65f..e71038d4918 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java @@ -41,6 +41,12 @@ public class NoopLimitSpec implements LimitSpec return Functions.identity(); } + @Override + public LimitSpec merge(LimitSpec other) + { + return other; + } + @Override public String toString() { diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 6b44645ac81..511b042e3e6 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -56,6 +56,7 @@ import io.druid.query.filter.JavaScriptDimFilter; import io.druid.query.filter.RegexDimFilter; import io.druid.query.groupby.having.EqualToHavingSpec; import io.druid.query.groupby.having.GreaterThanHavingSpec; +import io.druid.query.groupby.having.HavingSpec; import io.druid.query.groupby.having.OrHavingSpec; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.LimitSpec; @@ -1113,6 +1114,97 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testSubqueryWithPostAggregatorsAndHaving() + { + final GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }")) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx_subagg", "index") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx_subpostagg", + "+", + Arrays.asList( + new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"), + new ConstantPostAggregator("thousand", 1000, 1000) + ) + ) + + ) + ) + .setHavingSpec( + new HavingSpec() + { + @Override + public boolean eval(Row row) + { + return (row.getFloatMetric("idx_subpostagg") < 3800); + } + } + ) + .addOrderByColumn("alias") + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_agg", "idx"), + new ConstantPostAggregator("ten_thousand", 10000, 10000) + ) + ) + + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } private Iterable runQuery(GroupByQuery query) { From 22e368e79b9d3d370cde224b798c0c9c15789d93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 11:22:41 -0700 Subject: [PATCH 036/270] add dimension/column selector to incremental index --- .../segment/incremental/IncrementalIndex.java | 97 +++++++++++++++++-- 1 file changed, 90 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 057a4b95ff2..8ab7f06f7b8 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -46,6 +46,7 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; @@ -53,6 +54,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.LinkedHashMap; @@ -186,7 +188,7 @@ public class IncrementalIndex implements Iterable dims = newDims; } - TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims); + final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims); Aggregator[] aggs = facts.get(key); if (aggs == null) { @@ -248,8 +250,34 @@ public class IncrementalIndex implements Iterable }; } - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + final List dimensionValues = in.getDimension(columnName); + if (dimensionValues != null) { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + @Override + public Object get() + { + final String[] dimVals = dimensionValues.toArray(new String[]{}); + if (dimVals.length == 1) { + return dimVals[0]; + } + else if (dimVals.length == 0) { + return null; + } + else { + return dimVals; + } + } + }; + } + + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); if (serde == null) { throw new ISE("Don't know how to handle type[%s]", typeName); } @@ -273,12 +301,67 @@ public class IncrementalIndex implements Iterable } @Override - public DimensionSelector makeDimensionSelector(String dimension) + public DimensionSelector makeDimensionSelector(final String dimension) { - // we should implement this, but this is going to be rewritten soon anyways - throw new UnsupportedOperationException( - "Incremental index aggregation does not support dimension selectors" - ); + final String dimensionName = dimension.toLowerCase(); + final List dimensionValues = in.getDimension(dimensionName); + if (dimensionValues == null) { + return null; + } + + final IncrementalIndex.DimDim dimValLookup = getDimension(dimensionName); + final int maxId = dimValLookup.size(); + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + final ArrayList vals = Lists.newArrayList(); + for (String dimVal : dimensionValues) { + int id = dimValLookup.getId(dimVal); + vals.add(id); + } + + return new IndexedInts() + { + @Override + public int size() + { + return vals.size(); + } + + @Override + public int get(int index) + { + return vals.get(index); + } + + @Override + public Iterator iterator() + { + return vals.iterator(); + } + }; + } + + @Override + public int getValueCardinality() + { + return maxId; + } + + @Override + public String lookupName(int id) + { + return dimValLookup.getValue(id); + } + + @Override + public int lookupId(String name) + { + return dimValLookup.getId(name); + } + }; } } From 403962fe0b8e06235d61d511ee87bb226f9de0ab Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 12 Jun 2014 11:36:53 -0700 Subject: [PATCH 037/270] add some failing tests for groupBy --- .../JavaScriptAggregatorFactory.java | 2 +- .../CardinalityAggregatorFactory.java | 2 +- .../segment/incremental/IncrementalIndex.java | 15 +-- .../query/groupby/GroupByQueryRunnerTest.java | 112 ++++++++++++++++++ 4 files changed, 120 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 90175c54b88..9f84c320702 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -150,7 +150,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new JavaScriptAggregatorFactory(input, Arrays.asList(input), fnAggregate, fnReset, fnCombine); + return new JavaScriptAggregatorFactory(input, fieldNames, fnAggregate, fnReset, fnCombine); } } ); diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 4c79b076533..e8c414913f4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -156,7 +156,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new CardinalityAggregatorFactory(input, Arrays.asList(input), byRow); + return new CardinalityAggregatorFactory(input, fieldNames, byRow); } } ); diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 8ab7f06f7b8..1c14d1a0447 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -237,7 +237,7 @@ public class IncrementalIndex implements Iterable return new ObjectColumnSelector() { @Override - public Class classOfObject() + public Class classOfObject() { return Float.TYPE; } @@ -250,12 +250,11 @@ public class IncrementalIndex implements Iterable }; } - final List dimensionValues = in.getDimension(columnName); - if (dimensionValues != null) { + if (getDimension(columnName) != null) { return new ObjectColumnSelector() { @Override - public Class classOfObject() + public Class classOfObject() { return Object.class; } @@ -263,14 +262,12 @@ public class IncrementalIndex implements Iterable @Override public Object get() { - final String[] dimVals = dimensionValues.toArray(new String[]{}); + final String[] dimVals = in.getDimension(columnName).toArray(new String[]{}); if (dimVals.length == 1) { return dimVals[0]; - } - else if (dimVals.length == 0) { + } else if (dimVals.length == 0) { return null; - } - else { + } else { return dimVals; } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 511b042e3e6..5cfc101dcd9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -42,6 +42,7 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -1206,6 +1207,117 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testSubqueryWithEverything() + { + final GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("provider", "function(dim){ return true; }")) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx_subagg", "index"), + new JavaScriptAggregatorFactory( + "js_agg", + Arrays.asList("index", "provider"), + "function(index, dim){return index + dim.length;}", + "function(){return 0;}", + "function(a,b){return a + b;}" + ) + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx_subpostagg", + "+", + Arrays.asList( + new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"), + new ConstantPostAggregator("thousand", 1000, 1000) + ) + ) + + ) + ) + .setHavingSpec( + new HavingSpec() + { + @Override + public boolean eval(Row row) + { + return (row.getFloatMetric("idx_subpostagg") < 3800); + } + } + ) + .addOrderByColumn("alias") + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg"), + new DoubleSumAggregatorFactory("js_outer_agg", "js_agg") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_agg", "idx"), + new ConstantPostAggregator("ten_thousand", 10000, 10000) + ) + ) + + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Arrays.asList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + private Iterable runQuery(GroupByQuery query) { From e84bcca40f868b0738f60fc49ccaf3e39d969366 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 12 Jun 2014 13:36:40 -0700 Subject: [PATCH 038/270] rename method --- .../java/io/druid/query/aggregation/AggregatorFactory.java | 6 +++--- .../io/druid/query/aggregation/CountAggregatorFactory.java | 2 +- .../druid/query/aggregation/DoubleSumAggregatorFactory.java | 2 +- .../druid/query/aggregation/HistogramAggregatorFactory.java | 2 +- .../query/aggregation/JavaScriptAggregatorFactory.java | 2 +- .../druid/query/aggregation/LongSumAggregatorFactory.java | 2 +- .../io/druid/query/aggregation/MaxAggregatorFactory.java | 2 +- .../io/druid/query/aggregation/MinAggregatorFactory.java | 2 +- .../query/aggregation/ToLowerCaseAggregatorFactory.java | 4 ++-- .../cardinality/CardinalityAggregatorFactory.java | 2 +- .../hyperloglog/HyperUniquesAggregatorFactory.java | 2 +- .../io/druid/query/groupby/GroupByQueryQueryToolChest.java | 2 +- 12 files changed, 15 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index e236e683d08..ad355c71b6b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -65,11 +65,11 @@ public interface AggregatorFactory public AggregatorFactory getCombiningFactory(); /** - * Gets a list of aggregator factories using the actual column names of the data + * Gets a list of all columns that this AggregatorFactory will scan * - * @return Base AggregatorFactories for different fields of this AggregatorFactory + * @return AggregatorFactories for the columns to scan of the parent AggregatorFactory */ - public List getBaseFactories(); + public List getRequiredColumns(); /** * A method that knows how to "deserialize" the object from whatever form it might have been put into diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 1d1e2e149d7..9549015f687 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -78,7 +78,7 @@ public class CountAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new CountAggregatorFactory(name)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index 7e395c3c3ca..083a16589dc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -86,7 +86,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new DoubleSumAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java index 76809eb39a4..422d8279a2e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -101,7 +101,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new HistogramAggregatorFactory(fieldName, fieldName, breaksList)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 9f84c320702..fc5603f2fe9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -141,7 +141,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Lists.transform( fieldNames, diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 0eaae184e9f..963d9458beb 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -86,7 +86,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index 78abafa3d76..56e6de5f56e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -83,7 +83,7 @@ public class MaxAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new MaxAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index 6c373eded4b..e7256d0ccb9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -83,7 +83,7 @@ public class MinAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new MinAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java index b649f1fa5b3..a884bb9be4d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java @@ -66,9 +66,9 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { - return baseAggregatorFactory.getBaseFactories(); + return baseAggregatorFactory.getRequiredColumns(); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index e8c414913f4..12da44bea07 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -147,7 +147,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Lists.transform( fieldNames, diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index 2193be3419e..26449929703 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -132,7 +132,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new HyperUniquesAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 09cf7e431a8..025af21a2d7 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -111,7 +111,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); final List aggs = Lists.newArrayList(); for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { - aggs.addAll(aggregatorFactory.getBaseFactories()); + aggs.addAll(aggregatorFactory.getRequiredColumns()); } // We need the inner incremental index to have all the columns required by the outer query From 150480bdf98e82bb781b67d127370ca15312fe46 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 12:44:26 -0700 Subject: [PATCH 039/270] complexmetricserde workaround for groupby --- .../query/groupby/GroupByQueryHelper.java | 2 +- .../segment/incremental/IncrementalIndex.java | 95 +++++++++---------- 2 files changed, 45 insertions(+), 52 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 44fb82b7c6e..7041fa970da 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -83,7 +83,7 @@ public class GroupByQueryHelper @Override public IncrementalIndex accumulate(IncrementalIndex accumulated, Row in) { - if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions)) > config.getMaxResults()) { + if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions), false) > config.getMaxResults()) { throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults()); } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 1c14d1a0447..25aa47463ed 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -135,17 +135,29 @@ public class IncrementalIndex implements Iterable ); } + public int add(InputRow row) + { + // this is an ugly workaround to call ComplexMetricExtractor.extractValue at ingestion time + return add(row, true); + } + /** * Adds a new row. The row might correspond with another row that already exists, in which case this will * update that row instead of inserting a new one. *

- * This is *not* thread-safe. Calls to add() should always happen on the same thread. + * + * Calls to add() are thread safe. + * + * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that + * should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics. * * @param row the row of data to add + * @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input + * value for aggregators that return metrics other than float. * * @return the number of rows in the data set after adding the InputRow */ - public int add(InputRow row) + public int add(InputRow row, final boolean deserializeComplexMetrics) { row = spatialDimensionRowFormatter.formatRow(row); @@ -233,25 +245,7 @@ public class IncrementalIndex implements Iterable final String typeName = agg.getTypeName(); final String columnName = column.toLowerCase(); - if (typeName.equals("float")) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Float.TYPE; - } - - @Override - public Float get() - { - return in.getFloatMetric(columnName); - } - }; - } - - if (getDimension(columnName) != null) { - return new ObjectColumnSelector() + final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() { @Override public Class classOfObject() @@ -262,39 +256,38 @@ public class IncrementalIndex implements Iterable @Override public Object get() { - final String[] dimVals = in.getDimension(columnName).toArray(new String[]{}); - if (dimVals.length == 1) { - return dimVals[0]; - } else if (dimVals.length == 0) { - return null; - } else { - return dimVals; - } + return in.getRaw(columnName); + } + }; + + if(!deserializeComplexMetrics) { + return rawColumnSelector; + } else { + if (typeName.equals("float")) { + return rawColumnSelector; + } + + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + if (serde == null) { + throw new ISE("Don't know how to handle type[%s]", typeName); + } + + final ComplexMetricExtractor extractor = serde.getExtractor(); + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return extractor.extractedClass(); + } + + @Override + public Object get() + { + return extractor.extractValue(in, columnName); } }; } - - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); - if (serde == null) { - throw new ISE("Don't know how to handle type[%s]", typeName); - } - - final ComplexMetricExtractor extractor = serde.getExtractor(); - - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return extractor.extractedClass(); - } - - @Override - public Object get() - { - return extractor.extractValue(in, columnName); - } - }; } @Override From fd47efd59e695cb6ecb97af0829bbcfe3b6a0dc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 15:14:30 -0700 Subject: [PATCH 040/270] fix cardinality combining factory --- .../aggregation/cardinality/CardinalityAggregatorFactory.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 12da44bea07..10443c828cd 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -32,6 +32,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; import org.apache.commons.codec.binary.Base64; @@ -143,7 +144,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new CardinalityAggregatorFactory(name, fieldNames, byRow); + return new HyperUniquesAggregatorFactory(name, name); } @Override From bfa3caa58949b5b987d3fea4911b8677cbc4fa58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 15:16:07 -0700 Subject: [PATCH 041/270] js/cardinality/hyperUnique happy groupBy family --- .../HyperUniquesAggregatorFactory.java | 10 ++-- .../query/groupby/GroupByQueryHelper.java | 2 + .../segment/incremental/IncrementalIndex.java | 23 +++----- .../io/druid/query/QueryRunnerTestHelper.java | 6 ++ .../query/groupby/GroupByQueryRunnerTest.java | 58 ++++++++++++------- 5 files changed, 61 insertions(+), 38 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index 26449929703..e4597c71a0b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -73,12 +73,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory return Aggregators.noopAggregator(); } - if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) { + final Class classOfObject = selector.classOfObject(); + if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) { return new HyperUniquesAggregator(name, selector); } throw new IAE( - "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject() + "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject ); } @@ -91,12 +92,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory return Aggregators.noopBufferAggregator(); } - if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) { + final Class classOfObject = selector.classOfObject(); + if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) { return new HyperUniquesBufferAggregator(selector); } throw new IAE( - "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject() + "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject ); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 7041fa970da..b7183b12354 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -24,12 +24,14 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; +import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.data.input.Rows; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; import javax.annotation.Nullable; import java.util.List; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 25aa47463ed..4f6f6a0f67f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -248,7 +248,7 @@ public class IncrementalIndex implements Iterable final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() { @Override - public Class classOfObject() + public Class classOfObject() { return Object.class; } @@ -294,22 +294,17 @@ public class IncrementalIndex implements Iterable public DimensionSelector makeDimensionSelector(final String dimension) { final String dimensionName = dimension.toLowerCase(); - final List dimensionValues = in.getDimension(dimensionName); - if (dimensionValues == null) { - return null; - } - - final IncrementalIndex.DimDim dimValLookup = getDimension(dimensionName); - final int maxId = dimValLookup.size(); return new DimensionSelector() { @Override public IndexedInts getRow() { + final List dimensionValues = in.getDimension(dimensionName); final ArrayList vals = Lists.newArrayList(); - for (String dimVal : dimensionValues) { - int id = dimValLookup.getId(dimVal); - vals.add(id); + if (dimensionValues != null) { + for (int i = 0; i < dimensionValues.size(); ++i) { + vals.add(i); + } } return new IndexedInts() @@ -337,19 +332,19 @@ public class IncrementalIndex implements Iterable @Override public int getValueCardinality() { - return maxId; + throw new UnsupportedOperationException("value cardinality is unknown in incremental index"); } @Override public String lookupName(int id) { - return dimValLookup.getValue(id); + return in.getDimension(dimensionName).get(id); } @Override public int lookupId(String name) { - return dimValLookup.getId(name); + return in.getDimension(dimensionName).indexOf(name); } }; } diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 55c29752ac1..deeaff563e8 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -28,6 +28,7 @@ import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; @@ -110,6 +111,11 @@ public class QueryRunnerTestHelper "uniques", "quality_uniques" ); + public static final CardinalityAggregatorFactory qualityCardinality = new CardinalityAggregatorFactory( + "cardinality", + Arrays.asList("quality"), + false + ); public static final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); public static final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index a43cec90f77..d718c0775d9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -217,6 +217,36 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testGroupByWithCardinality() + { + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.qualityCardinality + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow( + "2011-04-01", + "rows", + 26L, + "cardinality", + QueryRunnerTestHelper.UNIQUES_9 + ) + ); + + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + @Test public void testGroupByWithDimExtractionFn() { @@ -1209,7 +1239,7 @@ public class GroupByQueryRunnerTest } @Test - public void testSubqueryWithEverything() + public void testSubqueryWithMultiColumnAggregators() { final GroupByQuery subquery = GroupByQuery .builder() @@ -1220,11 +1250,11 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx_subagg", "index"), + new DoubleSumAggregatorFactory("idx_subagg", "index"), new JavaScriptAggregatorFactory( "js_agg", Arrays.asList("index", "provider"), - "function(index, dim){return index + dim.length;}", + "function(current, index, dim){return current + index + dim.length;}", "function(){return 0;}", "function(a,b){return a + b;}" ) @@ -1295,23 +1325,11 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), - - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0, "js_outer_agg", 123.92274475097656), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0, "js_outer_agg", 82.62254333496094), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0, "js_outer_agg", 125.58358001708984), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0, "js_outer_agg", 124.13470458984375), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0, "js_outer_agg", 162.74722290039062) ); // Subqueries are handled by the ToolChest From 8f7c82d35152f9f869506b9495e768feb05fcb31 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 15:44:04 -0700 Subject: [PATCH 042/270] avoid closing yielder prematurely --- .../DruidDefaultSerializersModule.java | 4 +- .../java/io/druid/server/QueryResource.java | 46 +++++++++++-------- 2 files changed, 30 insertions(+), 20 deletions(-) diff --git a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java index 6184221a1db..a0cc2b87f73 100644 --- a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java @@ -132,17 +132,17 @@ public class DruidDefaultSerializersModule extends SimpleModule public void serialize(Yielder yielder, final JsonGenerator jgen, SerializerProvider provider) throws IOException, JsonProcessingException { - jgen.writeStartArray(); try { + jgen.writeStartArray(); while (!yielder.isDone()) { final Object o = yielder.get(); jgen.writeObject(o); yielder = yielder.next(null); } + jgen.writeEndArray(); } finally { yielder.close(); } - jgen.writeEndArray(); } } ); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 33bdd519c83..8300b028ff7 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -147,28 +147,29 @@ public class QueryResource log.debug("Got query [%s]", query); } - Sequence results = query.run(texasRanger); - - if (results == null) { + Sequence res = query.run(texasRanger); + final Sequence results; + if (res == null) { results = Sequences.empty(); + } else { + results = res; } - try ( - final Yielder yielder = results.toYielder( - null, - new YieldingAccumulator() - { - @Override - public Object accumulate(Object accumulated, Object in) - { - yield(); - return in; - } - } - ) - ) { - long requestTime = System.currentTimeMillis() - start; + final Yielder yielder = results.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Object accumulate(Object accumulated, Object in) + { + yield(); + return in; + } + } + ); + try { + long requestTime = System.currentTimeMillis() - start; emitter.emit( new ServiceMetricEvent.Builder() .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) @@ -202,6 +203,7 @@ public class QueryResource @Override public void write(OutputStream outputStream) throws IOException, WebApplicationException { + // json serializer will always close the yielder jsonWriter.writeValue(outputStream, yielder); outputStream.close(); } @@ -211,6 +213,14 @@ public class QueryResource .header("X-Druid-Query-Id", queryId) .build(); } + catch (Exception e) { + // make sure to close yieder if anything happened before starting to serialize the response. + yielder.close(); + } + finally { + // do not close yielder here, since we do not want to close the yielder prior to + // StreamingOutput having iterated over all the results + } } catch (QueryInterruptedException e) { try { From b2419b1530764ce72ea47f12f89208f9b2293488 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 17:09:21 -0700 Subject: [PATCH 043/270] fix missing return statement --- server/src/main/java/io/druid/server/QueryResource.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 8300b028ff7..7d7d39b402d 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -216,6 +216,7 @@ public class QueryResource catch (Exception e) { // make sure to close yieder if anything happened before starting to serialize the response. yielder.close(); + throw Throwables.propagate(e); } finally { // do not close yielder here, since we do not want to close the yielder prior to From 4c4047165f626e9dbcae42bdcd00c916a8ec8069 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 11:28:48 -0700 Subject: [PATCH 044/270] fix query cancellation test race condition --- .../io/druid/query/ChainedExecutionQueryRunnerTest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index f2555dd7214..a8c464a8703 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -243,7 +243,9 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); Assert.assertTrue(runner2.hasStarted); - Assert.assertFalse(runner3.hasStarted); + Assert.assertTrue(runner1.interrupted); + Assert.assertTrue(runner2.interrupted); + Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); Assert.assertFalse(runner1.hasCompleted); Assert.assertFalse(runner2.hasCompleted); Assert.assertFalse(runner3.hasCompleted); @@ -256,6 +258,7 @@ public class ChainedExecutionQueryRunnerTest private final CountDownLatch latch; private boolean hasStarted = false; private boolean hasCompleted = false; + private boolean interrupted = false; public DyingQueryRunner(CountDownLatch latch) { @@ -268,6 +271,7 @@ public class ChainedExecutionQueryRunnerTest hasStarted = true; latch.countDown(); if (Thread.interrupted()) { + interrupted = true; throw new QueryInterruptedException("I got killed"); } @@ -276,6 +280,7 @@ public class ChainedExecutionQueryRunnerTest Thread.sleep(500); } catch (InterruptedException e) { + interrupted = true; throw new QueryInterruptedException("I got killed"); } From 5bd4ad3f0f29d6b9e8a955f0b582eaacb6238582 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 11:29:49 -0700 Subject: [PATCH 045/270] address code review --- .../indexing/common/task/RealtimeIndexTask.java | 3 ++- .../firehose/kafka/KafkaEightFirehoseFactory.java | 9 +-------- .../firehose/kafka/KafkaSevenFirehoseFactory.java | 9 +-------- pom.xml | 4 ++-- .../incremental/SpatialDimensionRowFormatter.java | 12 ++++++++++-- .../druid/query/ChainedExecutionQueryRunnerTest.java | 2 +- .../firehose/rabbitmq/RabbitMQFirehoseFactory.java | 2 +- .../io/druid/curator/discovery/DiscoveryModule.java | 6 ++++++ .../curator/discovery/ServerDiscoveryFactory.java | 7 +++++++ .../io/druid/segment/realtime/RealtimeManager.java | 4 ++-- 10 files changed, 33 insertions(+), 25 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index bd04596f047..84ef51a9c4f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -26,6 +26,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.metamx.common.Granularity; import com.metamx.common.guava.CloseQuietly; +import com.metamx.common.parsers.ParseException; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; @@ -352,7 +353,7 @@ public class RealtimeIndexTask extends AbstractTask nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); } } - catch (Exception e) { + catch (ParseException e) { log.warn(e, "unparseable line"); fireDepartment.getMetrics().incrementUnparseable(); } diff --git a/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java index 70d373d7319..a97f18ae1b7 100644 --- a/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java +++ b/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java @@ -22,7 +22,6 @@ package io.druid.firehose.kafka; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.metamx.common.logger.Logger; @@ -123,13 +122,7 @@ public class KafkaEightFirehoseFactory implements FirehoseFactory UTF-8 - 0.26.0-SNAPSHOT - 2.4.0 + 0.26.5 + 2.5.0 0.2.4-SNAPSHOT diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java index ca98d740dc3..82311045d27 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java @@ -23,12 +23,14 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Predicate; import com.google.common.base.Splitter; +import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.primitives.Floats; import com.metamx.common.ISE; +import com.metamx.common.parsers.ParseException; import io.druid.data.input.InputRow; import io.druid.data.input.impl.SpatialDimensionSchema; @@ -134,14 +136,20 @@ public class SpatialDimensionRowFormatter } @Override - public Object getRaw(String dimension) { + public Object getRaw(String dimension) + { return row.getRaw(dimension); } @Override public float getFloatMetric(String metric) { - return row.getFloatMetric(metric); + try { + return row.getFloatMetric(metric); + } + catch (ParseException e) { + throw Throwables.propagate(e); + } } @Override diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index f2555dd7214..54b474830b2 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -108,7 +108,7 @@ public class ChainedExecutionQueryRunnerTest .build() ); - Future resultFuture = Executors.newFixedThreadPool(1).submit( + Future resultFuture = Executors.newSingleThreadExecutor().submit( new Runnable() { @Override diff --git a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index 71e53b5089c..c2e43e3917a 100644 --- a/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -128,7 +128,7 @@ public class RabbitMQFirehoseFactory implements FirehoseFactory> getAllInstances() throws Exception + { + return null; + } + @Override public void noteError(ServiceInstance tServiceInstance) { diff --git a/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java b/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java index 0e66df0b9ed..d8e5814e0d6 100644 --- a/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java +++ b/server/src/main/java/io/druid/curator/discovery/ServerDiscoveryFactory.java @@ -25,6 +25,7 @@ import org.apache.curator.x.discovery.ServiceInstance; import org.apache.curator.x.discovery.ServiceProvider; import java.io.IOException; +import java.util.Collection; /** */ @@ -62,6 +63,12 @@ public class ServerDiscoveryFactory return null; } + @Override + public Collection> getAllInstances() throws Exception + { + return null; + } + @Override public void noteError(ServiceInstance tServiceInstance) { // do nothing diff --git a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java index f249d5e3570..a1cfb220972 100644 --- a/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java +++ b/server/src/main/java/io/druid/segment/realtime/RealtimeManager.java @@ -27,6 +27,7 @@ import com.google.inject.Inject; import com.metamx.common.guava.CloseQuietly; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; +import com.metamx.common.parsers.ParseException; import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; import io.druid.data.input.InputRow; @@ -213,12 +214,11 @@ public class RealtimeManager implements QuerySegmentWalker } metrics.incrementProcessed(); } - catch (Exception e) { + catch (ParseException e) { if (inputRow != null) { log.error(e, "unparseable line: %s", inputRow); } metrics.incrementUnparseable(); - continue; } } } From 332ba3ef20fccf68471f4af87eb2942630a0f222 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 11:36:36 -0700 Subject: [PATCH 046/270] fix bad merge --- .../java/io/druid/query/ChainedExecutionQueryRunnerTest.java | 2 +- server/src/main/java/io/druid/server/QueryResource.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 90d9c447a96..a8c464a8703 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -108,7 +108,7 @@ public class ChainedExecutionQueryRunnerTest .build() ); - Future resultFuture = Executors.newSingleThreadExecutor().submit( + Future resultFuture = Executors.newFixedThreadPool(1).submit( new Runnable() { @Override diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 4d9188f48f9..4787b17465a 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.inject.Inject; From 294a9ba78f0b86fb8e369ee138d77277a1f33c05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 11:42:03 -0700 Subject: [PATCH 047/270] fix same race condition in other test --- .../java/io/druid/query/ChainedExecutionQueryRunnerTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index a8c464a8703..3a87e5c648d 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -139,7 +139,9 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); Assert.assertTrue(runner2.hasStarted); - Assert.assertFalse(runner3.hasStarted); + Assert.assertTrue(runner1.interrupted); + Assert.assertTrue(runner2.interrupted); + Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); Assert.assertFalse(runner1.hasCompleted); Assert.assertFalse(runner2.hasCompleted); Assert.assertFalse(runner3.hasCompleted); From 93f97a5d865d4184b2a2fc2ca8e10df183d38eb4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 13:20:14 -0700 Subject: [PATCH 048/270] fix bysegment-query-runner --- .../io/druid/query/BySegmentQueryRunner.java | 37 +++---------------- 1 file changed, 6 insertions(+), 31 deletions(-) diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index 44094d0216a..355f7538b9f 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -28,6 +28,7 @@ import com.metamx.common.guava.Yielders; import com.metamx.common.guava.YieldingAccumulator; import org.joda.time.DateTime; +import java.util.Arrays; import java.util.List; /** @@ -55,15 +56,10 @@ public class BySegmentQueryRunner implements QueryRunner { if (query.getContextBySegment(false)) { final Sequence baseSequence = base.run(query); - return new Sequence() - { - @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) - { - List results = Sequences.toList(baseSequence, Lists.newArrayList()); - return accumulator.accumulate( - initValue, + final List results = Sequences.toList(baseSequence, Lists.newArrayList()); + return Sequences.simple( + Arrays.asList( (T) new Result>( timestamp, new BySegmentResultValueClass( @@ -72,29 +68,8 @@ public class BySegmentQueryRunner implements QueryRunner query.getIntervals().get(0) ) ) - ); - } - - @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) - { - List results = Sequences.toList(baseSequence, Lists.newArrayList()); - - final OutType retVal = accumulator.accumulate( - initValue, - (T) new Result>( - timestamp, - new BySegmentResultValueClass( - results, - segmentIdentifier, - query.getIntervals().get(0) - ) - ) - ); - - return Yielders.done(retVal, null); - } - }; + ) + ); } return base.run(query); From 394d0cfbca4207ba45873448c77818573fb6a07d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:45:43 -0700 Subject: [PATCH 049/270] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From f6d319dac2d1772d0f81d7f72b8a61c7f0f3ef66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:46:08 -0700 Subject: [PATCH 050/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..96dfbca7ed3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..19badad6ea7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..01acc892db9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..393a1fad858 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..0876ffd3ff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..43822fc8df3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..98f5c2c79f8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..a7c848dbeb6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..4b811976653 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.122 + 0.6.123-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..cbc1e2753cf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..e56a55f4192 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..969a275696d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..ad805bc2df3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..8b418ce6972 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT From 6505f82053965c0bdbfbe0fc3279b6b18f735bc6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:54:23 -0700 Subject: [PATCH 051/270] [maven-release-plugin] rollback the release of druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 5 +++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +++-- services/pom.xml | 2 +- 14 files changed, 18 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 96dfbca7ed3..86ecc759553 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 19badad6ea7..4e828108927 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 01acc892db9..5b9cbc47847 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 393a1fad858..46b6a44ce6d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0876ffd3ff5..6decbae2e15 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 43822fc8df3..9db9badd919 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98f5c2c79f8..5840e57e491 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a7c848dbeb6..25bfd488014 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/pom.xml b/pom.xml index 4b811976653..6717674b368 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 io.druid druid pom - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT druid druid diff --git a/processing/pom.xml b/processing/pom.xml index cbc1e2753cf..f23eb94be46 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e56a55f4192..19a463d66e9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 969a275696d..1f6ba89a2cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ad805bc2df3..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 8b418ce6972..f8ed887cba2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT From 1159e2e41569fd88e121049b0536417218884c88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:56:12 -0700 Subject: [PATCH 052/270] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From bd18fef47432255396e84be26a3aa7d70eb9f672 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:57:31 -0700 Subject: [PATCH 053/270] [maven-release-plugin] rollback the release of druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 ++++--- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +++-- services/pom.xml | 2 +- 14 files changed, 19 insertions(+), 17 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..86ecc759553 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..4e828108927 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..5b9cbc47847 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..46b6a44ce6d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..6decbae2e15 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..9db9badd919 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..5840e57e491 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..25bfd488014 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..6717674b368 100644 --- a/pom.xml +++ b/pom.xml @@ -18,19 +18,20 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122 + 0.6.122-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..f23eb94be46 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..19a463d66e9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..1f6ba89a2cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..f8ed887cba2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT From 83b2434cad4d017497f0c26140c246480e2e464f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:01:09 -0700 Subject: [PATCH 054/270] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From 695a2946a9f4fd1c13d4d3f83942f0abff8a3228 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:01:13 -0700 Subject: [PATCH 055/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..96dfbca7ed3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..19badad6ea7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..01acc892db9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..393a1fad858 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..0876ffd3ff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..43822fc8df3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..98f5c2c79f8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..a7c848dbeb6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..4b811976653 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.122 + 0.6.123-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..cbc1e2753cf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..e56a55f4192 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..969a275696d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..ad805bc2df3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..8b418ce6972 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT From b7b28ad2f90e53d8316b5afa066c80a4da1bf08d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:03:03 -0700 Subject: [PATCH 056/270] [maven-release-plugin] rollback the release of druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 5 +++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +++-- services/pom.xml | 2 +- 14 files changed, 18 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 96dfbca7ed3..86ecc759553 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 19badad6ea7..4e828108927 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 01acc892db9..5b9cbc47847 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 393a1fad858..46b6a44ce6d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0876ffd3ff5..6decbae2e15 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 43822fc8df3..9db9badd919 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98f5c2c79f8..5840e57e491 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a7c848dbeb6..25bfd488014 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/pom.xml b/pom.xml index 4b811976653..6717674b368 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 io.druid druid pom - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT druid druid diff --git a/processing/pom.xml b/processing/pom.xml index cbc1e2753cf..f23eb94be46 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e56a55f4192..19a463d66e9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 969a275696d..1f6ba89a2cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ad805bc2df3..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 8b418ce6972..f8ed887cba2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT From bcc6e779196bc13b1792d0b24ffc2646be72a452 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 15:05:14 -0700 Subject: [PATCH 057/270] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From 0fd57dbc75e74a0b5887770eff730ef43f3aa6f2 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 15:05:18 -0700 Subject: [PATCH 058/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..96dfbca7ed3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..19badad6ea7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..01acc892db9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..393a1fad858 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..0876ffd3ff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..43822fc8df3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..98f5c2c79f8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..a7c848dbeb6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..4b811976653 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.122 + 0.6.123-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..cbc1e2753cf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..e56a55f4192 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..969a275696d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..ad805bc2df3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..8b418ce6972 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT From 903f1a695ff456533eb7fcede833914c0036c2c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:16:36 -0700 Subject: [PATCH 059/270] [maven-release-plugin] prepare release druid-0.6.123 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 96dfbca7ed3..08989c9ff5f 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/common/pom.xml b/common/pom.xml index 19badad6ea7..c26f423f5ba 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/examples/pom.xml b/examples/pom.xml index 01acc892db9..0f1cbe02d83 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 393a1fad858..0004336cdf8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0876ffd3ff5..19ac02b1834 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 43822fc8df3..617c9222b10 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98f5c2c79f8..1aa7a84bc33 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a7c848dbeb6..5a139be19c6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/pom.xml b/pom.xml index 4b811976653..632077b5edf 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.123-SNAPSHOT + 0.6.123 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.123 diff --git a/processing/pom.xml b/processing/pom.xml index cbc1e2753cf..db7a06c4e68 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e56a55f4192..3702c7d0d6a 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 969a275696d..a8f4d7c358d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/server/pom.xml b/server/pom.xml index ad805bc2df3..bb81e8aba72 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/services/pom.xml b/services/pom.xml index 8b418ce6972..bd186d66c98 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 From a5c533dcab9c156e498b587f845714f33232c2f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:17:00 -0700 Subject: [PATCH 060/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 08989c9ff5f..7729770ef86 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c26f423f5ba..4a515fc0675 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 0f1cbe02d83..177c43086cb 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 0004336cdf8..d0aa54ac33f 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 19ac02b1834..063a60bc0f3 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 617c9222b10..2aa51f4f21c 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 1aa7a84bc33..74cf5d1bfc6 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 5a139be19c6..b30908f6d83 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/pom.xml b/pom.xml index 632077b5edf..1aa12d28497 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.123 + 0.6.124-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.123 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index db7a06c4e68..3cfcf495797 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 3702c7d0d6a..507e95d266f 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index a8f4d7c358d..2e7da9309d8 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index bb81e8aba72..c6f55aa8a7a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index bd186d66c98..9db334a63ce 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT From d4a47fe6e8f86d3bceb825935960ffaa38c92de5 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 13:45:32 -0700 Subject: [PATCH 061/270] enable tb to return just max or min time --- .../src/main/java/io/druid/query/Druids.java | 10 +++ .../query/timeboundary/TimeBoundaryQuery.java | 69 ++++++++++++++----- .../TimeBoundaryQueryQueryToolChest.java | 52 ++++++++++---- .../timeboundary/TimeBoundaryResultValue.java | 4 ++ .../TimeBoundaryQueryRunnerTest.java | 42 +++++++++++ 5 files changed, 145 insertions(+), 32 deletions(-) diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index 084652b322f..4f94b8caf40 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -692,12 +692,14 @@ public class Druids { private DataSource dataSource; private QuerySegmentSpec querySegmentSpec; + private String exclude; private Map context; public TimeBoundaryQueryBuilder() { dataSource = null; querySegmentSpec = null; + exclude = null; context = null; } @@ -706,6 +708,7 @@ public class Druids return new TimeBoundaryQuery( dataSource, querySegmentSpec, + exclude, context ); } @@ -715,6 +718,7 @@ public class Druids return new TimeBoundaryQueryBuilder() .dataSource(builder.dataSource) .intervals(builder.querySegmentSpec) + .exclude(builder.exclude) .context(builder.context); } @@ -748,6 +752,12 @@ public class Druids return this; } + public TimeBoundaryQueryBuilder exclude(String ex) + { + exclude = ex; + return this; + } + public TimeBoundaryQueryBuilder context(Map c) { context = c; diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 6f3e70b9851..3ed1f17a604 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -50,10 +50,13 @@ public class TimeBoundaryQuery extends BaseQuery public static final String MIN_TIME = "minTime"; private static final byte CACHE_TYPE_ID = 0x0; + private final String exclude; + @JsonCreator public TimeBoundaryQuery( @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, + @JsonProperty("exclude") String exclude, @JsonProperty("context") Map context ) { @@ -63,6 +66,8 @@ public class TimeBoundaryQuery extends BaseQuery : querySegmentSpec, context ); + + this.exclude = exclude == null ? "" : exclude; } @Override @@ -77,12 +82,19 @@ public class TimeBoundaryQuery extends BaseQuery return Query.TIME_BOUNDARY; } + @JsonProperty + public String getExclude() + { + return exclude; + } + @Override public TimeBoundaryQuery withOverriddenContext(Map contextOverrides) { return new TimeBoundaryQuery( getDataSource(), getQuerySegmentSpec(), + exclude, computeOverridenContext(contextOverrides) ); } @@ -93,6 +105,7 @@ public class TimeBoundaryQuery extends BaseQuery return new TimeBoundaryQuery( getDataSource(), spec, + exclude, getContext() ); } @@ -103,14 +116,17 @@ public class TimeBoundaryQuery extends BaseQuery return new TimeBoundaryQuery( dataSource, getQuerySegmentSpec(), + exclude, getContext() ); } public byte[] getCacheKey() { - return ByteBuffer.allocate(1) + final byte[] excludeBytes = exclude.getBytes(); + return ByteBuffer.allocate(1 + excludeBytes.length) .put(CACHE_TYPE_ID) + .put(excludeBytes) .array(); } @@ -121,6 +137,7 @@ public class TimeBoundaryQuery extends BaseQuery "dataSource='" + getDataSource() + '\'' + ", querySegmentSpec=" + getQuerySegmentSpec() + ", duration=" + getDuration() + + ", exclude" + exclude + '}'; } @@ -129,14 +146,14 @@ public class TimeBoundaryQuery extends BaseQuery List> results = Lists.newArrayList(); Map result = Maps.newHashMap(); - if (min != null) { - result.put(TimeBoundaryQuery.MIN_TIME, min); + if (min != null && !exclude.equalsIgnoreCase(MIN_TIME)) { + result.put(MIN_TIME, min); } - if (max != null) { - result.put(TimeBoundaryQuery.MAX_TIME, max); + if (max != null && !exclude.equalsIgnoreCase(MAX_TIME)) { + result.put(MAX_TIME, max); } if (!result.isEmpty()) { - results.add(new Result(timestamp, new TimeBoundaryResultValue(result))); + results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result))); } return results; @@ -153,24 +170,40 @@ public class TimeBoundaryQuery extends BaseQuery for (Result result : results) { TimeBoundaryResultValue val = result.getValue(); - DateTime currMinTime = val.getMinTime(); - if (currMinTime.isBefore(min)) { - min = currMinTime; + if (!exclude.equalsIgnoreCase(MIN_TIME)) { + DateTime currMinTime = val.getMinTime(); + if (currMinTime.isBefore(min)) { + min = currMinTime; + } } - DateTime currMaxTime = val.getMaxTime(); - if (currMaxTime.isAfter(max)) { - max = currMaxTime; + if (!exclude.equalsIgnoreCase(MAX_TIME)) { + DateTime currMaxTime = val.getMaxTime(); + if (currMaxTime.isAfter(max)) { + max = currMaxTime; + } } } + final ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + final DateTime ts; + + if (exclude.equalsIgnoreCase(MIN_TIME)) { + ts = max; + builder.put(MAX_TIME, max); + } else if (exclude.equalsIgnoreCase(MAX_TIME)) { + ts = min; + builder.put(MIN_TIME, min); + } else { + ts = min; + builder.put(MAX_TIME, max); + builder.put(MIN_TIME, min); + } + return Arrays.asList( - new Result( - min, + new Result<>( + ts, new TimeBoundaryResultValue( - ImmutableMap.of( - TimeBoundaryQuery.MIN_TIME, min, - TimeBoundaryQuery.MAX_TIME, max - ) + builder.build() ) ) ); diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index cc4aaa46579..6a765db9ed3 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -67,21 +67,45 @@ public class TimeBoundaryQueryQueryToolChest return segments; } - final T first = segments.get(0); - final T second = segments.get(segments.size() - 1); + final T min = segments.get(0); + final T max = segments.get(segments.size() - 1); + + final Predicate filterPredicate; + // optimizations to avoid hitting too many segments + if (query.getExclude().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)) { + filterPredicate = new Predicate() + { + @Override + public boolean apply(T input) + { + return input.getInterval().overlaps(min.getInterval()); + } + }; + } else if (query.getExclude().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)) { + filterPredicate = new Predicate() + { + @Override + public boolean apply(T input) + { + return input.getInterval().overlaps(max.getInterval()); + } + }; + } else { + filterPredicate = new Predicate() + { + @Override + public boolean apply(T input) + { + return input.getInterval().overlaps(min.getInterval()) || input.getInterval() + .overlaps(max.getInterval()); + } + }; + } return Lists.newArrayList( Iterables.filter( segments, - new Predicate() - { - @Override - public boolean apply(T input) - { - return input.getInterval().overlaps(first.getInterval()) || input.getInterval() - .overlaps(second.getInterval()); - } - } + filterPredicate ) ); } @@ -146,9 +170,9 @@ public class TimeBoundaryQueryQueryToolChest public byte[] computeCacheKey(TimeBoundaryQuery query) { return ByteBuffer.allocate(2) - .put(TIMEBOUNDARY_QUERY) - .put(query.getCacheKey()) - .array(); + .put(TIMEBOUNDARY_QUERY) + .put(query.getCacheKey()) + .array(); } @Override diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java index 1210e95af2c..ba5777879e6 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryResultValue.java @@ -99,6 +99,10 @@ public class TimeBoundaryResultValue private DateTime getDateTimeValue(Object val) { + if (val == null) { + return null; + } + if (val instanceof DateTime) { return (DateTime) val; } else if (val instanceof String) { diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 7bc499dca80..470b80cf3b5 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -78,4 +78,46 @@ public class TimeBoundaryQueryRunnerTest Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), minTime); Assert.assertEquals(new DateTime("2011-04-15T00:00:00.000Z"), maxTime); } + + @Test + @SuppressWarnings("unchecked") + public void testTimeBoundaryExcludesMin() + { + TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() + .dataSource("testing") + .exclude(TimeBoundaryQuery.MIN_TIME) + .build(); + + Iterable> results = Sequences.toList( + runner.run(timeBoundaryQuery), + Lists.>newArrayList() + ); + TimeBoundaryResultValue val = results.iterator().next().getValue(); + DateTime minTime = val.getMinTime(); + DateTime maxTime = val.getMaxTime(); + + Assert.assertNull(minTime); + Assert.assertEquals(new DateTime("2011-04-15T00:00:00.000Z"), maxTime); + } + + @Test + @SuppressWarnings("unchecked") + public void testTimeBoundaryExcludesMax() + { + TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() + .dataSource("testing") + .exclude(TimeBoundaryQuery.MAX_TIME) + .build(); + + Iterable> results = Sequences.toList( + runner.run(timeBoundaryQuery), + Lists.>newArrayList() + ); + TimeBoundaryResultValue val = results.iterator().next().getValue(); + DateTime minTime = val.getMinTime(); + DateTime maxTime = val.getMaxTime(); + + Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), minTime); + Assert.assertNull(maxTime); + } } From d2cf7d3f0a4010c6bdfea0f19ece0948ff30ffdd Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 14:00:01 -0700 Subject: [PATCH 062/270] address cr --- .../src/main/java/io/druid/query/Druids.java | 12 +++--- .../query/timeboundary/TimeBoundaryQuery.java | 40 +++++++++---------- .../TimeBoundaryQueryQueryToolChest.java | 4 +- .../TimeBoundaryQueryRunnerTest.java | 11 ++--- 4 files changed, 32 insertions(+), 35 deletions(-) diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index 4f94b8caf40..932af432ad1 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -692,14 +692,14 @@ public class Druids { private DataSource dataSource; private QuerySegmentSpec querySegmentSpec; - private String exclude; + private String bound; private Map context; public TimeBoundaryQueryBuilder() { dataSource = null; querySegmentSpec = null; - exclude = null; + bound = null; context = null; } @@ -708,7 +708,7 @@ public class Druids return new TimeBoundaryQuery( dataSource, querySegmentSpec, - exclude, + bound, context ); } @@ -718,7 +718,7 @@ public class Druids return new TimeBoundaryQueryBuilder() .dataSource(builder.dataSource) .intervals(builder.querySegmentSpec) - .exclude(builder.exclude) + .bound(builder.bound) .context(builder.context); } @@ -752,9 +752,9 @@ public class Druids return this; } - public TimeBoundaryQueryBuilder exclude(String ex) + public TimeBoundaryQueryBuilder bound(String b) { - exclude = ex; + bound = b; return this; } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 3ed1f17a604..eecc60251a0 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -50,13 +50,13 @@ public class TimeBoundaryQuery extends BaseQuery public static final String MIN_TIME = "minTime"; private static final byte CACHE_TYPE_ID = 0x0; - private final String exclude; + private final String bound; @JsonCreator public TimeBoundaryQuery( @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, - @JsonProperty("exclude") String exclude, + @JsonProperty("bound") String bound, @JsonProperty("context") Map context ) { @@ -67,7 +67,7 @@ public class TimeBoundaryQuery extends BaseQuery context ); - this.exclude = exclude == null ? "" : exclude; + this.bound = bound == null ? "" : bound; } @Override @@ -83,9 +83,9 @@ public class TimeBoundaryQuery extends BaseQuery } @JsonProperty - public String getExclude() + public String getBound() { - return exclude; + return bound; } @Override @@ -94,7 +94,7 @@ public class TimeBoundaryQuery extends BaseQuery return new TimeBoundaryQuery( getDataSource(), getQuerySegmentSpec(), - exclude, + bound, computeOverridenContext(contextOverrides) ); } @@ -105,7 +105,7 @@ public class TimeBoundaryQuery extends BaseQuery return new TimeBoundaryQuery( getDataSource(), spec, - exclude, + bound, getContext() ); } @@ -116,14 +116,14 @@ public class TimeBoundaryQuery extends BaseQuery return new TimeBoundaryQuery( dataSource, getQuerySegmentSpec(), - exclude, + bound, getContext() ); } public byte[] getCacheKey() { - final byte[] excludeBytes = exclude.getBytes(); + final byte[] excludeBytes = bound.getBytes(); return ByteBuffer.allocate(1 + excludeBytes.length) .put(CACHE_TYPE_ID) .put(excludeBytes) @@ -137,7 +137,7 @@ public class TimeBoundaryQuery extends BaseQuery "dataSource='" + getDataSource() + '\'' + ", querySegmentSpec=" + getQuerySegmentSpec() + ", duration=" + getDuration() + - ", exclude" + exclude + + ", bound" + bound + '}'; } @@ -146,10 +146,10 @@ public class TimeBoundaryQuery extends BaseQuery List> results = Lists.newArrayList(); Map result = Maps.newHashMap(); - if (min != null && !exclude.equalsIgnoreCase(MIN_TIME)) { + if (min != null && !bound.equalsIgnoreCase(MAX_TIME)) { result.put(MIN_TIME, min); } - if (max != null && !exclude.equalsIgnoreCase(MAX_TIME)) { + if (max != null && !bound.equalsIgnoreCase(MIN_TIME)) { result.put(MAX_TIME, max); } if (!result.isEmpty()) { @@ -170,13 +170,13 @@ public class TimeBoundaryQuery extends BaseQuery for (Result result : results) { TimeBoundaryResultValue val = result.getValue(); - if (!exclude.equalsIgnoreCase(MIN_TIME)) { + if (!bound.equalsIgnoreCase(MAX_TIME)) { DateTime currMinTime = val.getMinTime(); if (currMinTime.isBefore(min)) { min = currMinTime; } } - if (!exclude.equalsIgnoreCase(MAX_TIME)) { + if (!bound.equalsIgnoreCase(MIN_TIME)) { DateTime currMaxTime = val.getMaxTime(); if (currMaxTime.isAfter(max)) { max = currMaxTime; @@ -187,16 +187,16 @@ public class TimeBoundaryQuery extends BaseQuery final ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); final DateTime ts; - if (exclude.equalsIgnoreCase(MIN_TIME)) { - ts = max; - builder.put(MAX_TIME, max); - } else if (exclude.equalsIgnoreCase(MAX_TIME)) { + if (bound.equalsIgnoreCase(MIN_TIME)) { ts = min; - builder.put(MIN_TIME, min); + builder.put(MIN_TIME, max); + } else if (bound.equalsIgnoreCase(MAX_TIME)) { + ts = max; + builder.put(MAX_TIME, min); } else { ts = min; - builder.put(MAX_TIME, max); builder.put(MIN_TIME, min); + builder.put(MAX_TIME, max); } return Arrays.asList( diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 6a765db9ed3..ec9da74eef8 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -72,7 +72,7 @@ public class TimeBoundaryQueryQueryToolChest final Predicate filterPredicate; // optimizations to avoid hitting too many segments - if (query.getExclude().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)) { + if (query.getBound().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)) { filterPredicate = new Predicate() { @Override @@ -81,7 +81,7 @@ public class TimeBoundaryQueryQueryToolChest return input.getInterval().overlaps(min.getInterval()); } }; - } else if (query.getExclude().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)) { + } else if (query.getBound().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)) { filterPredicate = new Predicate() { @Override diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 470b80cf3b5..32ff68ab350 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -20,13 +20,10 @@ package io.druid.query.timeboundary; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; -import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.Result; import org.joda.time.DateTime; import org.junit.Assert; @@ -81,11 +78,11 @@ public class TimeBoundaryQueryRunnerTest @Test @SuppressWarnings("unchecked") - public void testTimeBoundaryExcludesMin() + public void testTimeBoundaryMax() { TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() .dataSource("testing") - .exclude(TimeBoundaryQuery.MIN_TIME) + .bound(TimeBoundaryQuery.MAX_TIME) .build(); Iterable> results = Sequences.toList( @@ -102,11 +99,11 @@ public class TimeBoundaryQueryRunnerTest @Test @SuppressWarnings("unchecked") - public void testTimeBoundaryExcludesMax() + public void testTimeBoundaryMin() { TimeBoundaryQuery timeBoundaryQuery = Druids.newTimeBoundaryQueryBuilder() .dataSource("testing") - .exclude(TimeBoundaryQuery.MAX_TIME) + .bound(TimeBoundaryQuery.MIN_TIME) .build(); Iterable> results = Sequences.toList( From 08c88e8fb7c1ff6fd329cefc9caa83ca38e795fb Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 14:07:10 -0700 Subject: [PATCH 063/270] address cr --- .../io/druid/query/timeboundary/TimeBoundaryQuery.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index eecc60251a0..636c373b98f 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -21,6 +21,7 @@ package io.druid.query.timeboundary; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -123,10 +124,10 @@ public class TimeBoundaryQuery extends BaseQuery public byte[] getCacheKey() { - final byte[] excludeBytes = bound.getBytes(); - return ByteBuffer.allocate(1 + excludeBytes.length) + final byte[] boundBytes = bound.getBytes(Charsets.UTF_8); + return ByteBuffer.allocate(1 + boundBytes.length) .put(CACHE_TYPE_ID) - .put(excludeBytes) + .put(boundBytes) .array(); } From 9ca4f564ee23ccea7611b15cb1d26b1d16bd3f9e Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 14:20:16 -0700 Subject: [PATCH 064/270] address cr --- .../query/timeboundary/TimeBoundaryQuery.java | 20 +++----- .../TimeBoundaryQueryQueryToolChest.java | 50 +++++-------------- .../TimeBoundaryQueryRunnerFactory.java | 17 +++++-- 3 files changed, 34 insertions(+), 53 deletions(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 636c373b98f..6d90a6a50c1 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -147,10 +147,10 @@ public class TimeBoundaryQuery extends BaseQuery List> results = Lists.newArrayList(); Map result = Maps.newHashMap(); - if (min != null && !bound.equalsIgnoreCase(MAX_TIME)) { + if (min != null) { result.put(MIN_TIME, min); } - if (max != null && !bound.equalsIgnoreCase(MIN_TIME)) { + if (max != null) { result.put(MAX_TIME, max); } if (!result.isEmpty()) { @@ -171,17 +171,13 @@ public class TimeBoundaryQuery extends BaseQuery for (Result result : results) { TimeBoundaryResultValue val = result.getValue(); - if (!bound.equalsIgnoreCase(MAX_TIME)) { - DateTime currMinTime = val.getMinTime(); - if (currMinTime.isBefore(min)) { - min = currMinTime; - } + DateTime currMinTime = val.getMinTime(); + if (currMinTime != null && currMinTime.isBefore(min)) { + min = currMinTime; } - if (!bound.equalsIgnoreCase(MIN_TIME)) { - DateTime currMaxTime = val.getMaxTime(); - if (currMaxTime.isAfter(max)) { - max = currMaxTime; - } + DateTime currMaxTime = val.getMaxTime(); + if (currMaxTime != null && currMaxTime.isAfter(max)) { + max = currMaxTime; } } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index ec9da74eef8..fdde44b8581 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -70,42 +70,18 @@ public class TimeBoundaryQueryQueryToolChest final T min = segments.get(0); final T max = segments.get(segments.size() - 1); - final Predicate filterPredicate; - // optimizations to avoid hitting too many segments - if (query.getBound().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME)) { - filterPredicate = new Predicate() - { - @Override - public boolean apply(T input) - { - return input.getInterval().overlaps(min.getInterval()); - } - }; - } else if (query.getBound().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME)) { - filterPredicate = new Predicate() - { - @Override - public boolean apply(T input) - { - return input.getInterval().overlaps(max.getInterval()); - } - }; - } else { - filterPredicate = new Predicate() - { - @Override - public boolean apply(T input) - { - return input.getInterval().overlaps(min.getInterval()) || input.getInterval() - .overlaps(max.getInterval()); - } - }; - } - return Lists.newArrayList( Iterables.filter( segments, - filterPredicate + new Predicate() + { + @Override + public boolean apply(T input) + { + return (min != null && input.getInterval().overlaps(min.getInterval())) || + (max != null && input.getInterval().overlaps(max.getInterval())); + } + } ) ); } @@ -135,7 +111,7 @@ public class TimeBoundaryQueryQueryToolChest @Override public Sequence> mergeSequences(Sequence>> seqOfSequences) { - return new OrderedMergeSequence>(getOrdering(), seqOfSequences); + return new OrderedMergeSequence<>(getOrdering(), seqOfSequences); } @Override @@ -201,11 +177,11 @@ public class TimeBoundaryQueryQueryToolChest { @Override @SuppressWarnings("unchecked") - public Result apply(@Nullable Object input) + public Result apply(Object input) { List result = (List) input; - return new Result( + return new Result<>( new DateTime(result.get(0)), new TimeBoundaryResultValue(result.get(1)) ); @@ -216,7 +192,7 @@ public class TimeBoundaryQueryQueryToolChest @Override public Sequence> mergeSequences(Sequence>> seqOfSequences) { - return new MergeSequence>(getOrdering(), seqOfSequences); + return new MergeSequence<>(getOrdering(), seqOfSequences); } }; } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java index 1f78429ead3..bdb17694346 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerFactory.java @@ -32,6 +32,7 @@ import io.druid.query.QueryWatcher; import io.druid.query.Result; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +import org.joda.time.DateTime; import java.util.Iterator; import java.util.concurrent.ExecutorService; @@ -61,7 +62,7 @@ public class TimeBoundaryQueryRunnerFactory ExecutorService queryExecutor, Iterable>> queryRunners ) { - return new ChainedExecutionQueryRunner>( + return new ChainedExecutionQueryRunner<>( queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners ); } @@ -90,7 +91,7 @@ public class TimeBoundaryQueryRunnerFactory final TimeBoundaryQuery legacyQuery = (TimeBoundaryQuery) input; - return new BaseSequence, Iterator>>( + return new BaseSequence<>( new BaseSequence.IteratorMaker, Iterator>>() { @Override @@ -102,10 +103,18 @@ public class TimeBoundaryQueryRunnerFactory ); } + final DateTime minTime = legacyQuery.getBound().equalsIgnoreCase(TimeBoundaryQuery.MAX_TIME) + ? null + : adapter.getMinTime(); + final DateTime maxTime = legacyQuery.getBound().equalsIgnoreCase(TimeBoundaryQuery.MIN_TIME) + ? null + : adapter.getMaxTime(); + + return legacyQuery.buildResult( adapter.getInterval().getStart(), - adapter.getMinTime(), - adapter.getMaxTime() + minTime, + maxTime ).iterator(); } From 5aa46d08a8d04ac5ea6a8f325cfa5026820d3d37 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 14:24:36 -0700 Subject: [PATCH 065/270] address cr --- .../query/timeboundary/TimeBoundaryQuery.java | 59 +++++++++---------- 1 file changed, 28 insertions(+), 31 deletions(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 6d90a6a50c1..574d0704602 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -49,8 +49,27 @@ public class TimeBoundaryQuery extends BaseQuery ); public static final String MAX_TIME = "maxTime"; public static final String MIN_TIME = "minTime"; + private static final byte CACHE_TYPE_ID = 0x0; + public static Iterable> buildResult(DateTime timestamp, DateTime min, DateTime max) + { + List> results = Lists.newArrayList(); + Map result = Maps.newHashMap(); + + if (min != null) { + result.put(MIN_TIME, min); + } + if (max != null) { + result.put(MAX_TIME, max); + } + if (!result.isEmpty()) { + results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result))); + } + + return results; + } + private final String bound; @JsonCreator @@ -142,24 +161,6 @@ public class TimeBoundaryQuery extends BaseQuery '}'; } - public Iterable> buildResult(DateTime timestamp, DateTime min, DateTime max) - { - List> results = Lists.newArrayList(); - Map result = Maps.newHashMap(); - - if (min != null) { - result.put(MIN_TIME, min); - } - if (max != null) { - result.put(MAX_TIME, max); - } - if (!result.isEmpty()) { - results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result))); - } - - return results; - } - public Iterable> mergeResults(List> results) { if (results == null || results.isEmpty()) { @@ -181,28 +182,24 @@ public class TimeBoundaryQuery extends BaseQuery } } - final ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); final DateTime ts; + final DateTime minTime; + final DateTime maxTime; if (bound.equalsIgnoreCase(MIN_TIME)) { ts = min; - builder.put(MIN_TIME, max); + minTime = min; + maxTime = null; } else if (bound.equalsIgnoreCase(MAX_TIME)) { ts = max; - builder.put(MAX_TIME, min); + minTime = null; + maxTime = max; } else { ts = min; - builder.put(MIN_TIME, min); - builder.put(MAX_TIME, max); + minTime = min; + maxTime = max; } - return Arrays.asList( - new Result<>( - ts, - new TimeBoundaryResultValue( - builder.build() - ) - ) - ); + return buildResult(ts, minTime, maxTime); } } From a71f7152a54002629b2e2da54d7a24ec2308fb56 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 14:53:26 -0700 Subject: [PATCH 066/270] fix broken UT --- .../TimeBoundaryQueryRunnerTest.java | 45 +++++++++++++++++++ .../server/router/QueryHostFinderTest.java | 1 + 2 files changed, 46 insertions(+) diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 32ff68ab350..3c39b3274d6 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -19,12 +19,14 @@ package io.druid.query.timeboundary; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; +import io.druid.query.TableDataSource; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -32,7 +34,9 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.IOException; +import java.util.Arrays; import java.util.Collection; +import java.util.List; /** */ @@ -117,4 +121,45 @@ public class TimeBoundaryQueryRunnerTest Assert.assertEquals(new DateTime("2011-01-12T00:00:00.000Z"), minTime); Assert.assertNull(maxTime); } + + @Test + public void testMergeResults() throws Exception + { + List> results = Arrays.asList( + new Result<>( + new DateTime(), + new TimeBoundaryResultValue( + ImmutableMap.of( + "maxTime", "2012-01-01", + "minTime", "2011-01-01" + ) + ) + ), + new Result<>( + new DateTime(), + new TimeBoundaryResultValue( + ImmutableMap.of( + "maxTime", "2012-02-01", + "minTime", "2011-01-01" + ) + ) + ) + ); + + TimeBoundaryQuery query = new TimeBoundaryQuery(new TableDataSource("test"), null, null, null); + Iterable> actual = query.mergeResults(results); + + Assert.assertTrue(actual.iterator().next().getValue().getMaxTime().equals(new DateTime("2012-02-01"))); + } + + @Test + public void testMergeResultsEmptyResults() throws Exception + { + List> results = Lists.newArrayList(); + + TimeBoundaryQuery query = new TimeBoundaryQuery(new TableDataSource("test"), null, null, null); + Iterable> actual = query.mergeResults(results); + + Assert.assertFalse(actual.iterator().hasNext()); + } } diff --git a/server/src/test/java/io/druid/server/router/QueryHostFinderTest.java b/server/src/test/java/io/druid/server/router/QueryHostFinderTest.java index e8bf260d5ac..d3a015d616f 100644 --- a/server/src/test/java/io/druid/server/router/QueryHostFinderTest.java +++ b/server/src/test/java/io/druid/server/router/QueryHostFinderTest.java @@ -126,6 +126,7 @@ public class QueryHostFinderTest new TimeBoundaryQuery( new TableDataSource("test"), new MultipleIntervalSegmentSpec(Arrays.asList(new Interval("2011-08-31/2011-09-01"))), + null, null ) ); From d01be2f85bad3927cb32b090a699770db04fa96a Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 15:04:42 -0700 Subject: [PATCH 067/270] revert submodule code --- services/pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/services/pom.xml b/services/pom.xml index 9db334a63ce..1cb60beb8e9 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -17,7 +17,8 @@ ~ along with this program; if not, write to the Free Software ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-services @@ -90,7 +91,8 @@ - + @@ -101,6 +103,10 @@ distro-assembly + package + + assembly + src/assembly/assembly.xml From c2149f8a006e664320cf5826eb50f0f5becde474 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 15:32:28 -0700 Subject: [PATCH 068/270] add more tests for time boundary and caching --- .../query/timeboundary/TimeBoundaryQuery.java | 52 +++++++++++++++---- .../client/CachingClusteredClientTest.java | 47 +++++++++++++++++ 2 files changed, 88 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index 574d0704602..a07f83c5480 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -150,17 +150,6 @@ public class TimeBoundaryQuery extends BaseQuery .array(); } - @Override - public String toString() - { - return "TimeBoundaryQuery{" + - "dataSource='" + getDataSource() + '\'' + - ", querySegmentSpec=" + getQuerySegmentSpec() + - ", duration=" + getDuration() + - ", bound" + bound + - '}'; - } - public Iterable> mergeResults(List> results) { if (results == null || results.isEmpty()) { @@ -202,4 +191,45 @@ public class TimeBoundaryQuery extends BaseQuery return buildResult(ts, minTime, maxTime); } + + @Override + public String toString() + { + return "TimeBoundaryQuery{" + + "dataSource='" + getDataSource() + '\'' + + ", querySegmentSpec=" + getQuerySegmentSpec() + + ", duration=" + getDuration() + + ", bound" + bound + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + TimeBoundaryQuery that = (TimeBoundaryQuery) o; + + if (!bound.equals(that.bound)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = super.hashCode(); + result = 31 * result + bound.hashCode(); + return result; + } } diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 7e151f3818d..90ac6f24e03 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -73,6 +73,7 @@ import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQueryConfig; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; import io.druid.query.timeboundary.TimeBoundaryResultValue; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; @@ -695,6 +696,51 @@ public class CachingClusteredClientTest ); } + @Test + public void testTimeBoundaryCaching() throws Exception + { + testQueryCaching( + client, + Druids.newTimeBoundaryQueryBuilder() + .dataSource(CachingClusteredClientTest.DATA_SOURCE) + .intervals(CachingClusteredClientTest.SEG_SPEC) + .context(CachingClusteredClientTest.CONTEXT) + .build(), + new Interval("2011-01-01/2011-01-02"), + makeTimeBoundaryResult(new DateTime("2011-01-01"), new DateTime("2011-01-01"), new DateTime("2011-01-02")), + + new Interval("2011-01-01/2011-01-03"), + makeTimeBoundaryResult(new DateTime("2011-01-02"), new DateTime("2011-01-02"), new DateTime("2011-01-03")), + + new Interval("2011-01-01/2011-01-10"), + makeTimeBoundaryResult(new DateTime("2011-01-05"), new DateTime("2011-01-05"), new DateTime("2011-01-10")), + + new Interval("2011-01-01/2011-01-10"), + makeTimeBoundaryResult(new DateTime("2011-01-05T01"), new DateTime("2011-01-05T01"), new DateTime("2011-01-10")) + ); + } + + private Iterable> makeTimeBoundaryResult( + DateTime timestamp, + DateTime minTime, + DateTime maxTime + ) + { + return Arrays.asList( + new Result<>( + timestamp, + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + minTime.toString(), + TimeBoundaryQuery.MAX_TIME, + maxTime.toString() + ) + ) + ) + ); + } + public void testQueryCaching(QueryRunner runner, final Query query, Object... args) { testQueryCaching(runner, 3, true, query, args); @@ -1287,6 +1333,7 @@ public class CachingClusteredClientTest ) .put(TopNQuery.class, new TopNQueryQueryToolChest(new TopNQueryConfig())) .put(SearchQuery.class, new SearchQueryQueryToolChest(new SearchQueryConfig())) + .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) .build() ), new TimelineServerView() From ce61e60ee82c927db9a4952cca880ccc377b3fa1 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 16:19:26 -0700 Subject: [PATCH 069/270] all manners of new tests --- .../java/io/druid/segment/AppendTest.java | 700 ++++++++ .../io/druid/segment/SchemalessIndex.java | 512 ++++++ .../io/druid/segment/SchemalessTestFull.java | 1546 +++++++++++++++++ .../druid/segment/SchemalessTestSimple.java | 295 ++++ processing/src/test/resources/append.json.1 | 24 + processing/src/test/resources/append.json.2 | 34 + processing/src/test/resources/append.json.3 | 23 + processing/src/test/resources/append.json.4 | 28 + processing/src/test/resources/append.json.5 | 32 + processing/src/test/resources/append.json.6 | 12 + processing/src/test/resources/append.json.7 | 12 + .../src/test/resources/druid.sample.json | 66 + .../test/resources/druid.sample.json.bottom | 49 + .../src/test/resources/druid.sample.json.top | 19 + 14 files changed, 3352 insertions(+) create mode 100644 processing/src/test/java/io/druid/segment/AppendTest.java create mode 100644 processing/src/test/java/io/druid/segment/SchemalessIndex.java create mode 100644 processing/src/test/java/io/druid/segment/SchemalessTestFull.java create mode 100644 processing/src/test/java/io/druid/segment/SchemalessTestSimple.java create mode 100644 processing/src/test/resources/append.json.1 create mode 100644 processing/src/test/resources/append.json.2 create mode 100644 processing/src/test/resources/append.json.3 create mode 100644 processing/src/test/resources/append.json.4 create mode 100644 processing/src/test/resources/append.json.5 create mode 100644 processing/src/test/resources/append.json.6 create mode 100644 processing/src/test/resources/append.json.7 create mode 100644 processing/src/test/resources/druid.sample.json create mode 100644 processing/src/test/resources/druid.sample.json.bottom create mode 100644 processing/src/test/resources/druid.sample.json.top diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java new file mode 100644 index 00000000000..b98d3266df7 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -0,0 +1,700 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.Pair; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.TestQueryRunners; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.filter.DimFilter; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import io.druid.query.topn.TopNResultValue; +import io.druid.segment.QueryableIndex; +import io.druid.segment.QueryableIndexSegment; +import io.druid.segment.Segment; +import io.druid.segment.TestHelper; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + */ +public class AppendTest +{ + private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("index", "index"), + new CountAggregatorFactory("count"), + new HyperUniquesAggregatorFactory("quality_uniques", "quality") + }; + private static final AggregatorFactory[] METRIC_AGGS_NO_UNIQ = new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("index", "index"), + new CountAggregatorFactory("count") + }; + + final String dataSource = "testing"; + final QueryGranularity allGran = QueryGranularity.ALL; + final String dimensionValue = "dimension"; + final String valueValue = "value"; + final String providerDimension = "provider"; + final String qualityDimension = "quality"; + final String placementDimension = "placement"; + final String placementishDimension = "placementish"; + final String indexMetric = "index"; + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); + final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); + final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); + final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); + final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); + final ArithmeticPostAggregator addRowsIndexConstant = + new ArithmeticPostAggregator( + "addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + ); + final List commonAggregators = Arrays.asList(rowsCount, indexDoubleSum, uniques); + + final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) + ); + + private Segment segment; + private Segment segment2; + private Segment segment3; + + @Before + public void setUp() throws Exception + { + // (1, 2) cover overlapping segments of the form + // |------| + // |--------| + QueryableIndex appendedIndex = SchemalessIndex.getAppendedIncrementalIndex( + Arrays.asList( + new Pair("append.json.1", METRIC_AGGS_NO_UNIQ), + new Pair("append.json.2", METRIC_AGGS) + ), + Arrays.asList( + new Interval("2011-01-12T00:00:00.000Z/2011-01-16T00:00:00.000Z"), + new Interval("2011-01-14T22:00:00.000Z/2011-01-16T00:00:00.000Z") + ) + ); + segment = new QueryableIndexSegment(null, appendedIndex); + + // (3, 4) cover overlapping segments of the form + // |------------| + // |-----| + QueryableIndex append2 = SchemalessIndex.getAppendedIncrementalIndex( + Arrays.asList( + new Pair("append.json.3", METRIC_AGGS_NO_UNIQ), + new Pair("append.json.4", METRIC_AGGS) + ), + Arrays.asList( + new Interval("2011-01-12T00:00:00.000Z/2011-01-16T00:00:00.000Z"), + new Interval("2011-01-13T00:00:00.000Z/2011-01-14T00:00:00.000Z") + ) + ); + segment2 = new QueryableIndexSegment(null, append2); + + // (5, 6, 7) test gaps that can be created in data because of rows being discounted + // |-------------| + // |---| + // |---| + QueryableIndex append3 = SchemalessIndex.getAppendedIncrementalIndex( + Arrays.asList( + new Pair("append.json.5", METRIC_AGGS), + new Pair("append.json.6", METRIC_AGGS), + new Pair("append.json.7", METRIC_AGGS) + ), + Arrays.asList( + new Interval("2011-01-12T00:00:00.000Z/2011-01-22T00:00:00.000Z"), + new Interval("2011-01-13T00:00:00.000Z/2011-01-16T00:00:00.000Z"), + new Interval("2011-01-18T00:00:00.000Z/2011-01-21T00:00:00.000Z") + ) + ); + segment3 = new QueryableIndexSegment(null, append3); + } + + @Test + public void testTimeBoundary() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-15T02:00:00.000Z") + ) + ) + ) + ); + + TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() + .dataSource(dataSource) + .build(); + QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTimeBoundary2() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-15T00:00:00.000Z") + ) + ) + ) + ); + + TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() + .dataSource(dataSource) + .build(); + QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTimeSeries() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 8L) + .put("index", 700.0D) + .put("addRowsIndexConstant", 709.0D) + .put("uniques", 1.0002442201269182D) + .put("maxIndex", 100.0D) + .put("minIndex", 0.0D) + .build() + ) + ) + ); + + TimeseriesQuery query = makeTimeseriesQuery(); + QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTimeSeries2() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 7L) + .put("index", 500.0D) + .put("addRowsIndexConstant", 508.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 0.0D) + .build() + ) + ) + ); + + TimeseriesQuery query = makeTimeseriesQuery(); + QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFilteredTimeSeries() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 5L) + .put("index", 500.0D) + .put("addRowsIndexConstant", 506.0D) + .put("uniques", 1.0002442201269182D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + TimeseriesQuery query = makeFilteredTimeseriesQuery(); + QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFilteredTimeSeries2() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 4L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 405.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + TimeseriesQuery query = makeFilteredTimeseriesQuery(); + QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNSeries() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 3L) + .put("index", 300.0D) + .put("addRowsIndexConstant", 304.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + new HashMap() + {{ + put("provider", null); + put("rows", 3L); + put("index", 200.0D); + put("addRowsIndexConstant", 204.0D); + put("uniques", 0.0D); + put("maxIndex", 100.0); + put("minIndex", 0.0); + }}, + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", 1.0002442201269182D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ) + ); + + TopNQuery query = makeTopNQuery(); + QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTopNSeries2() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 3L) + .put("index", 300.0D) + .put("addRowsIndexConstant", 304.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build(), + new HashMap() + {{ + put("provider", null); + put("rows", 3L); + put("index", 100.0D); + put("addRowsIndexConstant", 104.0D); + put("uniques", 0.0D); + put("maxIndex", 100.0); + put("minIndex", 0.0); + }}, + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + TopNQuery query = makeTopNQuery(); + QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFilteredTopNSeries() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + TopNQuery query = makeFilteredTopNQuery(); + QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFilteredTopNSeries2() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Lists.>newArrayList() + ) + ) + ); + + TopNQuery query = makeFilteredTopNQuery(); + QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testSearch() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive"), + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + SearchQuery query = makeSearchQuery(); + QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); + QueryRunner runner2 = TestQueryRunners.makeSearchQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); +/* TODO: this is broken. Actually, it's a bug in the appending code that keeps dimension values around that don't + TODO: exist anymore. The bug should be fixed and this uncommented + TestHelper.assertExpectedResults(expectedResults, runner2.run(query)); +*/ + } + + @Test + public void testFilteredSearch() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + SearchQuery query = makeFilteredSearchQuery(); + QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFilteredSearch2() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + SearchQuery query = makeFilteredSearchQuery(); + QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testRowFiltering() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 5L) + .put("index", 500.0D) + .put("addRowsIndexConstant", 506.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .filters(providerDimension, "breakstuff") + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment3); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + private TimeseriesQuery makeTimeseriesQuery() + { + return Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + } + + private TimeseriesQuery makeFilteredTimeseriesQuery() + { + return Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .filters( + Druids.newOrDimFilterBuilder() + .fields( + Arrays.asList( + Druids.newSelectorDimFilterBuilder() + .dimension(providerDimension) + .value("spot") + .build(), + Druids.newSelectorDimFilterBuilder() + .dimension(providerDimension) + .value("total_market") + .build() + ) + ).build() + ) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + } + + private TopNQuery makeTopNQuery() + { + return new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(3) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + } + + private TopNQuery makeFilteredTopNQuery() + { + return new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(3) + .filters( + Druids.newAndDimFilterBuilder() + .fields( + Arrays.asList( + Druids.newSelectorDimFilterBuilder() + .dimension(providerDimension) + .value("spot") + .build(), + Druids.newSelectorDimFilterBuilder() + .dimension(placementDimension) + .value("preferred") + .build() + ) + ).build() + ) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + } + + private SearchQuery makeSearchQuery() + { + return Druids.newSearchQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .query("a") + .build(); + } + + private SearchQuery makeFilteredSearchQuery() + { + return Druids.newSearchQueryBuilder() + .dataSource(dataSource) + .filters( + Druids.newNotDimFilterBuilder() + .field( + Druids.newSelectorDimFilterBuilder() + .dimension(providerDimension) + .value("spot") + .build() + ).build() + ) + .granularity(allGran) + .intervals(fullOnInterval) + .query("a") + .build(); + } +} diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndex.java b/processing/src/test/java/io/druid/segment/SchemalessIndex.java new file mode 100644 index 00000000000..836127120f5 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/SchemalessIndex.java @@ -0,0 +1,512 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.hash.Hashing; +import com.metamx.common.Pair; +import com.metamx.common.logger.Logger; +import io.druid.data.input.MapBasedInputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.serde.ComplexMetrics; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NoneShardSpec; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + */ +public class SchemalessIndex +{ + private static final Logger log = new Logger(SchemalessIndex.class); + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + private static final String testFile = "druid.sample.json"; + private static final String TIMESTAMP = "timestamp"; + private static final List METRICS = Arrays.asList("index"); + private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("index", "index"), + new CountAggregatorFactory("count"), + new HyperUniquesAggregatorFactory("quality_uniques", "quality") + }; + private static final AggregatorFactory[] METRIC_AGGS_NO_UNIQ = new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("index", "index"), + new CountAggregatorFactory("count") + }; + + private static final List> events = Lists.newArrayList(); + + private static final Map> incrementalIndexes = Maps.newHashMap(); + private static final Map> mergedIndexes = Maps.newHashMap(); + private static final List rowPersistedIndexes = Lists.newArrayList(); + + private static IncrementalIndex index = null; + private static QueryableIndex mergedIndex = null; + + static { + if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128())); + } + } + + public static IncrementalIndex getIncrementalIndex() + { + synchronized (log) { + if (index != null) { + return index; + } + + index = makeIncrementalIndex(testFile, METRIC_AGGS); + + return index; + } + } + + public static QueryableIndex getIncrementalIndex(int index1, int index2) + { + synchronized (log) { + if (events.isEmpty()) { + makeEvents(); + } + + Map entry = incrementalIndexes.get(index1); + if (entry != null) { + QueryableIndex index = entry.get(index2); + if (index != null) { + return index; + } + } else { + entry = Maps.newHashMap(); + incrementalIndexes.put(index1, entry); + } + + IncrementalIndex theIndex = null; + + int count = 0; + for (final Map event : events) { + if (count != index1 && count != index2) { + count++; + continue; + } + + final long timestamp = new DateTime(event.get(TIMESTAMP)).getMillis(); + + if (theIndex == null) { + theIndex = new IncrementalIndex(timestamp, QueryGranularity.MINUTE, METRIC_AGGS); + } + + final List dims = Lists.newArrayList(); + for (final Map.Entry val : event.entrySet()) { + if (!val.getKey().equalsIgnoreCase(TIMESTAMP) && !METRICS.contains(val.getKey())) { + dims.add(val.getKey()); + } + } + + theIndex.add(new MapBasedInputRow(timestamp, dims, event)); + + count++; + } + QueryableIndex retVal = TestIndex.persistRealtimeAndLoadMMapped(theIndex); + entry.put(index2, retVal); + return retVal; + } + } + + public static QueryableIndex getMergedIncrementalIndex() + { + synchronized (log) { + if (mergedIndex != null) { + return mergedIndex; + } + + try { + IncrementalIndex top = makeIncrementalIndex("druid.sample.json.top", METRIC_AGGS); + IncrementalIndex bottom = makeIncrementalIndex("druid.sample.json.bottom", METRIC_AGGS); + + File tmpFile = File.createTempFile("yay", "who"); + tmpFile.delete(); + + File topFile = new File(tmpFile, "top"); + File bottomFile = new File(tmpFile, "bottom"); + File mergedFile = new File(tmpFile, "merged"); + + topFile.mkdirs(); + topFile.deleteOnExit(); + bottomFile.mkdirs(); + bottomFile.deleteOnExit(); + mergedFile.mkdirs(); + mergedFile.deleteOnExit(); + + IndexMerger.persist(top, topFile); + IndexMerger.persist(bottom, bottomFile); + + mergedIndex = io.druid.segment.IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex( + Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile + ) + ); + + return mergedIndex; + } + catch (IOException e) { + mergedIndex = null; + throw Throwables.propagate(e); + } + } + } + + public static QueryableIndex getMergedIncrementalIndex(int index1, int index2) + { + synchronized (log) { + if (rowPersistedIndexes.isEmpty()) { + makeRowPersistedIndexes(); + } + + Map entry = mergedIndexes.get(index1); + if (entry != null) { + QueryableIndex index = entry.get(index2); + if (index != null) { + return index; + } + } else { + entry = Maps.newHashMap(); + mergedIndexes.put(index1, entry); + } + + try { + File tmpFile = File.createTempFile("yay", "who"); + tmpFile.delete(); + + File mergedFile = new File(tmpFile, "merged"); + + mergedFile.mkdirs(); + mergedFile.deleteOnExit(); + + QueryableIndex index = IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex( + Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), METRIC_AGGS, mergedFile + ) + ); + + entry.put(index2, index); + + return index; + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + + public static QueryableIndex getMergedIncrementalIndex(int[] indexes) + { + synchronized (log) { + if (rowPersistedIndexes.isEmpty()) { + makeRowPersistedIndexes(); + } + + try { + File tmpFile = File.createTempFile("yay", "who"); + tmpFile.delete(); + + File mergedFile = new File(tmpFile, "merged"); + + mergedFile.mkdirs(); + mergedFile.deleteOnExit(); + + List indexesToMerge = Lists.newArrayList(); + for (int i = 0; i < indexes.length; i++) { + indexesToMerge.add(rowPersistedIndexes.get(indexes[i])); + } + + QueryableIndex index = IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex(indexesToMerge, METRIC_AGGS, mergedFile) + ); + + return index; + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + + public static QueryableIndex getAppendedIncrementalIndex( + Iterable> files, + List intervals + ) + { + return makeAppendedMMappedIndex(files, intervals); + } + + public static QueryableIndex getMergedIncrementalIndexDiffMetrics() + { + return getMergedIncrementalIndex( + Arrays.>asList( + new Pair("druid.sample.json.top", METRIC_AGGS_NO_UNIQ), + new Pair("druid.sample.json.bottom", METRIC_AGGS) + ) + ); + } + + public static QueryableIndex getMergedIncrementalIndex(Iterable> files) + { + return makeMergedMMappedIndex(files); + } + + private static void makeEvents() + { + URL resource = TestIndex.class.getClassLoader().getResource(testFile); + String filename = resource.getFile(); + log.info("Realtime loading index file[%s]", filename); + try { + for (Object obj : jsonMapper.readValue(new File(filename), List.class)) { + final Map event = jsonMapper.convertValue(obj, Map.class); + events.add(event); + } + } + catch (Exception e) { + Throwables.propagate(e); + } + } + + private static void makeRowPersistedIndexes() + { + synchronized (log) { + try { + if (events.isEmpty()) { + makeEvents(); + } + + for (final Map event : events) { + + final long timestamp = new DateTime(event.get(TIMESTAMP)).getMillis(); + final List dims = Lists.newArrayList(); + for (Map.Entry entry : event.entrySet()) { + if (!entry.getKey().equalsIgnoreCase(TIMESTAMP) && !METRICS.contains(entry.getKey())) { + dims.add(entry.getKey()); + } + } + + final IncrementalIndex rowIndex = new IncrementalIndex( + timestamp, QueryGranularity.MINUTE, METRIC_AGGS + ); + + rowIndex.add( + new MapBasedInputRow(timestamp, dims, event) + ); + + File tmpFile = File.createTempFile("billy", "yay"); + tmpFile.delete(); + tmpFile.mkdirs(); + tmpFile.deleteOnExit(); + + IndexMerger.persist(rowIndex, tmpFile); + rowPersistedIndexes.add(IndexIO.loadIndex(tmpFile)); + } + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + + private static IncrementalIndex makeIncrementalIndex(final String resourceFilename, AggregatorFactory[] aggs) + { + URL resource = TestIndex.class.getClassLoader().getResource(resourceFilename); + log.info("Realtime loading resource[%s]", resource); + String filename = resource.getFile(); + log.info("Realtime loading index file[%s]", filename); + + final IncrementalIndex retVal = new IncrementalIndex( + new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.MINUTE, aggs + ); + + try { + final List events = jsonMapper.readValue(new File(filename), List.class); + for (Object obj : events) { + final Map event = jsonMapper.convertValue(obj, Map.class); + + final List dims = Lists.newArrayList(); + for (Map.Entry entry : event.entrySet()) { + if (!entry.getKey().equalsIgnoreCase(TIMESTAMP) && !METRICS.contains(entry.getKey())) { + dims.add(entry.getKey()); + } + } + + retVal.add( + new MapBasedInputRow( + new DateTime(event.get(TIMESTAMP)).getMillis(), + dims, + event + ) + ); + } + } + catch (IOException e) { + index = null; + throw Throwables.propagate(e); + } + + return retVal; + } + + private static List makeFilesToMap(File tmpFile, Iterable> files) + throws IOException + { + List filesToMap = Lists.newArrayList(); + for (Pair file : files) { + IncrementalIndex index = makeIncrementalIndex(file.lhs, file.rhs); + File theFile = new File(tmpFile, file.lhs); + theFile.mkdirs(); + theFile.deleteOnExit(); + filesToMap.add(theFile); + IndexMerger.persist(index, theFile); + } + + return filesToMap; + } + + private static QueryableIndex makeAppendedMMappedIndex( + Iterable> files, + final List intervals + ) + { + try { + File tmpFile = File.createTempFile("yay", "boo"); + tmpFile.delete(); + File mergedFile = new File(tmpFile, "merged"); + mergedFile.mkdirs(); + mergedFile.deleteOnExit(); + + List filesToMap = makeFilesToMap(tmpFile, files); + + List adapters = Lists.newArrayList(); + + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( + Ordering.natural().nullsFirst() + ); + + ShardSpec noneShardSpec = new NoneShardSpec(); + + for (int i = 0; i < intervals.size(); i++) { + timeline.add(intervals.get(i), i, noneShardSpec.createChunk(filesToMap.get(i))); + } + + List> intervalsToMerge = Lists.transform( + timeline.lookup(new Interval("1000-01-01/3000-01-01")), + new Function, Pair>() + { + @Override + public Pair apply(@Nullable TimelineObjectHolder input) + { + return new Pair(input.getObject().getChunk(0).getObject(), input.getInterval()); + } + } + ); + + for (final Pair pair : intervalsToMerge) { + adapters.add( + new RowboatFilteringIndexAdapter( + new QueryableIndexIndexableAdapter(IndexIO.loadIndex(pair.lhs)), + new Predicate() + { + @Override + public boolean apply(@Nullable Rowboat input) + { + return pair.rhs.contains(input.getTimestamp()); + } + } + ) + ); + } + + return IndexIO.loadIndex(IndexMerger.append(adapters, mergedFile)); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private static QueryableIndex makeMergedMMappedIndex(Iterable> files) + { + try { + File tmpFile = File.createTempFile("yay", "who"); + tmpFile.delete(); + File mergedFile = new File(tmpFile, "merged"); + mergedFile.mkdirs(); + mergedFile.deleteOnExit(); + + List filesToMap = makeFilesToMap(tmpFile, files); + + return IndexIO.loadIndex( + IndexMerger.mergeQueryableIndex( + Lists.newArrayList( + Iterables.transform( + filesToMap, + new Function() + { + @Override + public QueryableIndex apply(@Nullable File input) + { + try { + return IndexIO.loadIndex(input); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + ) + ), + METRIC_AGGS, + mergedFile + ) + ); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java new file mode 100644 index 00000000000..83f6a62aa99 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -0,0 +1,1546 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.metamx.common.Pair; +import com.metamx.common.guava.Sequences; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.TestQueryRunners; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import io.druid.query.topn.TopNResultValue; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + */ +public class SchemalessTestFull +{ + final double UNIQUES_2 = 2.000977198748901d; + final double UNIQUES_1 = 1.0002442201269182d; + + final String dataSource = "testing"; + final QueryGranularity allGran = QueryGranularity.ALL; + final String dimensionValue = "dimension"; + final String valueValue = "value"; + final String providerDimension = "provider"; + final String qualityDimension = "quality"; + final String placementDimension = "placement"; + final String placementishDimension = "placementish"; + final String indexMetric = "index"; + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); + final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); + final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); + final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); + final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); + final ArithmeticPostAggregator addRowsIndexConstant = + new ArithmeticPostAggregator( + "addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + ); + final List commonAggregators = Arrays.asList(rowsCount, indexDoubleSum, uniques); + + final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) + ); + + @Test + public void testCompleteIntersectingSchemas() + { + List> expectedTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", UNIQUES_2) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive"), + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-13T00:00:00.000Z") + ) + ) + ) + ); + + testAll( + 1, + 5, + expectedTimeSeriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults + ); + } + + @Test + public void testEmptyStrings() + { + List> expectedTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + ) + ) + ) + ); + + List> expectedFilteredSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + ) + ) + ) + ); + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + testAll( + 9, + 10, + expectedTimeSeriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults + ); + } + + + @Test + public void testNonIntersectingSchemas() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 2L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 103.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 0.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + new HashMap() + {{ + put("provider", null); + put("rows", 1L); + put("index", 0.0D); + put("addRowsIndexConstant", 2.0D); + put("uniques", 0.0D); + put("maxIndex", 0.0); + put("minIndex", 0.0); + }} + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + testAll( + 2, + 3, + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults + ); + } + + @Test + public void testPartialIntersectingSchemas() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(qualityDimension, "automotive"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-13T00:00:00.000Z") + ) + ) + ) + ); + + testAll( + 2, + 4, + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults + ); + } + + @Test + public void testSupersetSchemas() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = expectedTimeseriesResults; + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = expectedTopNResults; + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = expectedSearchResults; + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + testAll( + 1, + 2, + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults + ); + } + + @Test + public void testValueAndEmptySchemas() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 2L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 103.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 0.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + new HashMap() + {{ + put("provider", null); + put("rows", 1L); + put("index", 0.0D); + put("addRowsIndexConstant", 2.0D); + put("uniques", 0.0D); + put("maxIndex", 0.0); + put("minIndex", 0.0); + }} + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = expectedSearchResults; + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + testAll( + 0, + 1, + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults + ); + } + + @Test + public void testEmptySchemas() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 0.0D) + .put("addRowsIndexConstant", 2.0D) + .put("uniques", 0.0D) + .put("maxIndex", 0.0D) + .put("minIndex", 0.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 0L) + .put("index", 0.0D) + .put("addRowsIndexConstant", 1.0D) + .put("uniques", 0.0D) + .put("maxIndex", Double.NEGATIVE_INFINITY) + .put("minIndex", Double.POSITIVE_INFINITY) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList(); + List> expectedFilteredTopNResults = Arrays.asList(); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList() + ) + ) + ); + List> expectedFilteredSearchResults = expectedSearchResults; + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + runTests( + new QueryableIndexSegment( + null, SchemalessIndex.getMergedIncrementalIndex(0, 0) + ), + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults, + "Failed: MII[0, 0]" + ); + } + + @Test + public void testExactSameSchemas() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 202.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 200.0D) + .put("minIndex", 200.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = expectedTimeseriesResults; + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 202.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 200.0) + .put("minIndex", 200.0) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = expectedTopNResults; + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = expectedSearchResults; + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + runTests( + new QueryableIndexSegment( + null, SchemalessIndex.getMergedIncrementalIndex(1, 1) + ), + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults, + "Failed: MII[1, 1]" + ); + } + + @Test + public void testMultiDimensionalValues() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 3L) + .put("index", 300.0D) + .put("addRowsIndexConstant", 304.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + new HashMap() + {{ + put("provider", null); + put("rows", 2L); + put("index", 200.0D); + put("addRowsIndexConstant", 203.0D); + put("uniques", 0.0D); + put("maxIndex", 100.0); + put("minIndex", 100.0); + }}, + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementDimension, "mezzanine") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList() + ) + ) + ); + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-12T00:00:00.000Z") + ) + ) + ) + ); + + runTests( + new QueryableIndexSegment( + null, SchemalessIndex.getMergedIncrementalIndex(new int[]{6, 7, 8}) + ) + , + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults, + "Failed: MII[6, 7]" + ); + } + + @Test + public void testDifferentMetrics() + { + List> expectedTimeseriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 11L) + .put("index", 900.0D) + .put("addRowsIndexConstant", 912.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0D) + .put("minIndex", 0.0D) + .build() + ) + ) + ); + + List> expectedFilteredTimeSeriesResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 4L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 405.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ); + + List> expectedTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 4L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 405.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("provider", "") + .put("rows", 3L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 204.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 0.0) + .build(), + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", UNIQUES_1) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedFilteredTopNResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 4L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 405.0D) + .put("uniques", 0.0D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build(), + ImmutableMap.builder() + .put("provider", "") + .put("rows", 1L) + .put("index", 100.0D) + .put("addRowsIndexConstant", 102.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ) + ) + ) + ); + + List> expectedSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive"), + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + List> expectedFilteredSearchResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive") + ) + ) + ) + ); + + List> expectedTimeBoundaryResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-13T00:00:00.000Z") + ) + ) + ) + ); + + runTests( + new QueryableIndexSegment(null, SchemalessIndex.getMergedIncrementalIndexDiffMetrics()), + expectedTimeseriesResults, + expectedFilteredTimeSeriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults, + "Failed: MIIDM" + ); + } + + private List> getIndexes(int index1, int index2) + { + return Arrays.asList( + new Pair( + SchemalessIndex.getIncrementalIndex(index1, index2), + String.format("Failed: II[%,d, %,d]", index1, index2) + ), + new Pair( + SchemalessIndex.getIncrementalIndex(index2, index1), + String.format("Failed: II[%,d, %,d]", index2, index1) + ), + new Pair( + SchemalessIndex.getMergedIncrementalIndex(index1, index2), + String.format("Failed: MII[%,d, %,d]", index1, index2) + ), + new Pair( + SchemalessIndex.getMergedIncrementalIndex(index2, index1), + String.format("Failed: MII[%,d, %,d]", index2, index1) + ) + ); + } + + private void testAll( + int index1, + int index2, + List> expectedTimeseriesResults, + List> expectedFilteredTimeseriesResults, + List> expectedTopNResults, + List> expectedFilteredTopNResults, + List> expectedSearchResults, + List> expectedFilteredSearchResults, + List> expectedTimeBoundaryResults + ) + { + for (Pair entry : getIndexes(index1, index2)) { + runTests( + new QueryableIndexSegment(null, entry.lhs), + expectedTimeseriesResults, + expectedFilteredTimeseriesResults, + expectedTopNResults, + expectedFilteredTopNResults, + expectedSearchResults, + expectedFilteredSearchResults, + expectedTimeBoundaryResults, + entry.rhs + ); + } + } + + private void runTests( + Segment adapter, + List> expectedTimeseriesResults, + List> expectedFilteredTimeseriesResults, + List> expectedTopNResults, + List> expectedFilteredTopNResults, + List> expectedSearchResults, + List> expectedFilteredSearchResults, + List> expectedTimeBoundaryResults, + String failMsg + ) + { + testFullOnTimeseries(TestQueryRunners.makeTimeSeriesQueryRunner(adapter), expectedTimeseriesResults, failMsg); + testFilteredTimeseries( + TestQueryRunners.makeTimeSeriesQueryRunner(adapter), + expectedFilteredTimeseriesResults, + failMsg + ); + testFullOnTopN(TestQueryRunners.makeTopNQueryRunner(adapter), expectedTopNResults, failMsg); + testFilteredTopN(TestQueryRunners.makeTopNQueryRunner(adapter), expectedFilteredTopNResults, failMsg); + testFullOnSearch(TestQueryRunners.makeSearchQueryRunner(adapter), expectedSearchResults, failMsg); + testFilteredSearch(TestQueryRunners.makeSearchQueryRunner(adapter), expectedFilteredSearchResults, failMsg); + testTimeBoundary(TestQueryRunners.makeTimeBoundaryQueryRunner(adapter), expectedTimeBoundaryResults, failMsg); + } + + private void testFullOnTimeseries( + QueryRunner runner, + List> expectedResults, + String failMsg + ) + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + failMsg += " timeseries "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } + + private void testFilteredTimeseries( + QueryRunner runner, + List> expectedResults, + String failMsg + ) + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .filters(providerDimension, "spot") + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + failMsg += " filtered timeseries "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } + + + private void testFullOnTopN(QueryRunner runner, List> expectedResults, String failMsg) + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(3) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + failMsg += " topN "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } + + private void testFilteredTopN(QueryRunner runner, List> expectedResults, String failMsg) + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .filters(providerDimension, "spot") + .metric(indexMetric) + .threshold(3) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + failMsg += " filtered topN "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } + + private void testFullOnSearch(QueryRunner runner, List> expectedResults, String failMsg) + { + SearchQuery query = Druids.newSearchQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .query("a") + .build(); + + failMsg += " search "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } + + private void testFilteredSearch(QueryRunner runner, List> expectedResults, String failMsg) + { + SearchQuery query = Druids.newSearchQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .filters(providerDimension, "spot") + .intervals(fullOnInterval) + .query("a") + .build(); + + failMsg += " filtered search "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } + + private void testTimeBoundary( + QueryRunner runner, + List> expectedResults, + String failMsg + ) + { + TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() + .dataSource("testing") + .build(); + + failMsg += " timeBoundary "; + Iterable> actualResults = Sequences.toList( + runner.run(query), + Lists.>newArrayList() + ); + TestHelper.assertExpectedResults(expectedResults, actualResults, failMsg); + } +} diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java new file mode 100644 index 00000000000..85226b3256d --- /dev/null +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -0,0 +1,295 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.QueryRunner; +import io.druid.query.Result; +import io.druid.query.TestQueryRunners; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.search.SearchResultValue; +import io.druid.query.search.search.SearchHit; +import io.druid.query.search.search.SearchQuery; +import io.druid.query.spec.MultipleIntervalSegmentSpec; +import io.druid.query.spec.QuerySegmentSpec; +import io.druid.query.timeboundary.TimeBoundaryQuery; +import io.druid.query.timeboundary.TimeBoundaryResultValue; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.query.topn.DimensionAndMetricValueExtractor; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import io.druid.query.topn.TopNResultValue; +import io.druid.segment.incremental.IncrementalIndex; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +/** + */ +@RunWith(Parameterized.class) +public class SchemalessTestSimple +{ + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final IncrementalIndex incrementalIndex = SchemalessIndex.getIncrementalIndex(); + final QueryableIndex persistedIncrementalIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex); + final QueryableIndex mergedIncrementalIndex = SchemalessIndex.getMergedIncrementalIndex(); + + return Arrays.asList( + new Object[][]{ + { + new IncrementalIndexSegment(incrementalIndex, null) + }, + { + new QueryableIndexSegment( + null, persistedIncrementalIndex + ) + }, + { + new QueryableIndexSegment( + null, mergedIncrementalIndex + ) + } + } + ); + } + + final String dataSource = "testing"; + final QueryGranularity allGran = QueryGranularity.ALL; + final String dimensionValue = "dimension"; + final String valueValue = "value"; + final String providerDimension = "provider"; + final String qualityDimension = "quality"; + final String placementDimension = "placement"; + final String placementishDimension = "placementish"; + final String indexMetric = "index"; + final CountAggregatorFactory rowsCount = new CountAggregatorFactory("rows"); + final DoubleSumAggregatorFactory indexDoubleSum = new DoubleSumAggregatorFactory("index", "index"); + final HyperUniquesAggregatorFactory uniques = new HyperUniquesAggregatorFactory("uniques", "quality_uniques"); + final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); + final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); + final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); + final ArithmeticPostAggregator addRowsIndexConstant = + new ArithmeticPostAggregator( + "addRowsIndexConstant", "+", Lists.newArrayList(constant, rowsPostAgg, indexPostAgg) + ); + final List commonAggregators = Arrays.asList(rowsCount, indexDoubleSum, uniques); + + final QuerySegmentSpec fullOnInterval = new MultipleIntervalSegmentSpec( + Arrays.asList(new Interval("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) + ); + + private Segment segment; + + public SchemalessTestSimple( + Segment segment + ) + { + this.segment = segment; + } + + @Test + public void testFullOnTimeseries() + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 11L) + .put("index", 900.0) + .put("addRowsIndexConstant", 912.0) + .put("uniques", 2.000977198748901D) + .put("maxIndex", 100.0) + .put("minIndex", 0.0) + .build() + ) + ) + ); + QueryRunner runner = TestQueryRunners.makeTimeSeriesQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + + // @Test TODO: Handling of null values is inconsistent right now, need to make it all consistent and re-enable test + // TODO: Complain to Eric when you see this. It shouldn't be like this... + public void testFullOnTopN() + { + TopNQuery query = new TopNQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .dimension(providerDimension) + .metric(indexMetric) + .threshold(3) + .intervals(fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(Arrays.asList(addRowsIndexConstant)) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.asList( + new DimensionAndMetricValueExtractor( + ImmutableMap.builder() + .put("provider", "spot") + .put("rows", 4L) + .put("index", 400.0D) + .put("addRowsIndexConstant", 405.0D) + .put("uniques", 1.0002442201269182D) + .put("maxIndex", 100.0) + .put("minIndex", 100.0) + .build() + ), + new DimensionAndMetricValueExtractor( + ImmutableMap.builder() + .put("provider", "") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", 0.0) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ), + new DimensionAndMetricValueExtractor( + ImmutableMap.builder() + .put("provider", "total_market") + .put("rows", 2L) + .put("index", 200.0D) + .put("addRowsIndexConstant", 203.0D) + .put("uniques", 1.0002442201269182D) + .put("maxIndex", 100.0D) + .put("minIndex", 100.0D) + .build() + ) + ) + ) + ) + ); + + QueryRunner runner = TestQueryRunners.makeTopNQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testFullOnSearch() + { + SearchQuery query = Druids.newSearchQueryBuilder() + .dataSource(dataSource) + .granularity(allGran) + .intervals(fullOnInterval) + .query("a") + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(qualityDimension, "automotive"), + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } + + @Test + public void testTimeBoundary() + { + TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() + .dataSource("testing") + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TimeBoundaryResultValue( + ImmutableMap.of( + TimeBoundaryQuery.MIN_TIME, + new DateTime("2011-01-12T00:00:00.000Z"), + TimeBoundaryQuery.MAX_TIME, + new DateTime("2011-01-13T00:00:00.000Z") + ) + ) + ) + ); + + QueryRunner runner = TestQueryRunners.makeTimeBoundaryQueryRunner(segment); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } +} diff --git a/processing/src/test/resources/append.json.1 b/processing/src/test/resources/append.json.1 new file mode 100644 index 00000000000..aa55a99694a --- /dev/null +++ b/processing/src/test/resources/append.json.1 @@ -0,0 +1,24 @@ +[ + { + "timestamp":"2011-01-12T00:00:00.000Z" + }, + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "placement":"preferred", + "placementish":["a", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-14T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "index":100.000000 + }, + { + "timestamp":"2011-01-15T00:00:00.000Z", + "placement":"preferred", + "placementish":["a", "preferred"] + } +] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.2 b/processing/src/test/resources/append.json.2 new file mode 100644 index 00000000000..fd3f916e9ec --- /dev/null +++ b/processing/src/test/resources/append.json.2 @@ -0,0 +1,34 @@ +[ + { + "timestamp":"2011-01-14T22:00:00.000Z", + "provider":"total_market", + "placement":"preferred", + "placementish":["h", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-14T23:00:00.000Z", + "provider":"total_market", + "quality":"business", + "placement":"mezzanine", + "placementish":["p", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-15T00:00:00.000Z", + "placementish":"preferred", + "provider":"spot", + "index":100.000000 + }, + { + "timestamp":"2011-01-15T01:00:00.000Z", + "placementish":["p", "preferred"], + "placement":["q", "mezzanine"], + "index":100.000000 + }, + { + "timestamp":"2011-01-15T02:00:00.000Z", + "placement":"preferred", + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.3 b/processing/src/test/resources/append.json.3 new file mode 100644 index 00000000000..7a963dc4e78 --- /dev/null +++ b/processing/src/test/resources/append.json.3 @@ -0,0 +1,23 @@ +[ + { + "timestamp":"2011-01-12T00:00:00.000Z" + }, + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "placement":"preferred", + "placementish":["a", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-14T00:00:00.000Z", + "provider":"total_market", + "index":100.000000 + }, + { + "timestamp":"2011-01-15T00:00:00.000Z", + "placement":"preferred", + "placementish":["a", "preferred"] + } +] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.4 b/processing/src/test/resources/append.json.4 new file mode 100644 index 00000000000..5049fdc5e69 --- /dev/null +++ b/processing/src/test/resources/append.json.4 @@ -0,0 +1,28 @@ +[ + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"total_market", + "placement":"preferred", + "placementish":["h", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-13T01:00:00.000Z", + "provider":"total_market", + "placement":"mezzanine", + "placementish":["p", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-13T15:00:00.000Z", + "placementish":"preferred", + "provider":"spot", + "index":100.000000 + }, + { + "timestamp":"2011-01-13T23:00:00.000Z", + "placementish":["p", "preferred"], + "placement":["q", "mezzanine"], + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.5 b/processing/src/test/resources/append.json.5 new file mode 100644 index 00000000000..0119fead368 --- /dev/null +++ b/processing/src/test/resources/append.json.5 @@ -0,0 +1,32 @@ +[ + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + }, + { + "timestamp":"2011-01-14T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + }, + { + "timestamp":"2011-01-16T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + }, + { + "timestamp":"2011-01-17T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + }, + { + "timestamp":"2011-01-19T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + }, + { + "timestamp":"2011-01-21T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.6 b/processing/src/test/resources/append.json.6 new file mode 100644 index 00000000000..386769ce0ef --- /dev/null +++ b/processing/src/test/resources/append.json.6 @@ -0,0 +1,12 @@ +[ + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"breakstuff", + "index":100.000000 + }, + { + "timestamp":"2011-01-15T00:00:00.000Z", + "provider":"spot", + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/append.json.7 b/processing/src/test/resources/append.json.7 new file mode 100644 index 00000000000..8645df52b8f --- /dev/null +++ b/processing/src/test/resources/append.json.7 @@ -0,0 +1,12 @@ +[ + { + "timestamp":"2011-01-18T00:00:00.000Z", + "provider":"spot", + "index":100.000000 + }, + { + "timestamp":"2011-01-20T00:00:00.000Z", + "provider":"spot", + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json b/processing/src/test/resources/druid.sample.json new file mode 100644 index 00000000000..e188e7be089 --- /dev/null +++ b/processing/src/test/resources/druid.sample.json @@ -0,0 +1,66 @@ +[ + { + "timestamp":"2011-01-12T00:00:00.000Z" + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "placement":"preferred", + "placementish":["a", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placement":"preferred", + "placementish":["a", "preferred"] + }, + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"total_market", + "placementish":["h", "preferred"], + "placement":"preferred", + "index":100.000000 + }, + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"total_market", + "placementish":["p", "preferred"], + "quality":"business", + "placement":"mezzanine", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placementish":"preferred", + "provider":"spot", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placementish":["p", "preferred"], + "placement":["q", "mezzanine"], + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placement":"preferred", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":["", "spot"], + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json.bottom b/processing/src/test/resources/druid.sample.json.bottom new file mode 100644 index 00000000000..a7b3b648a2a --- /dev/null +++ b/processing/src/test/resources/druid.sample.json.bottom @@ -0,0 +1,49 @@ +[ + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placement":"preferred", + "placementish":["a", "preferred"] + }, + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"total_market", + "placement":"preferred", + "placementish":["h", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-13T00:00:00.000Z", + "provider":"total_market", + "quality":"business", + "placement":"mezzanine", + "placementish":["p", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placementish":"preferred", + "provider":"spot", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placementish":["p", "preferred"], + "placement":["q", "mezzanine"], + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "placement":"preferred", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"", + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":["", "spot"], + "index":100.000000 + } +] \ No newline at end of file diff --git a/processing/src/test/resources/druid.sample.json.top b/processing/src/test/resources/druid.sample.json.top new file mode 100644 index 00000000000..8439291578d --- /dev/null +++ b/processing/src/test/resources/druid.sample.json.top @@ -0,0 +1,19 @@ +[ + { + "timestamp":"2011-01-12T00:00:00.000Z" + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "placement":"preferred", + "placementish":["a", "preferred"], + "index":100.000000 + }, + { + "timestamp":"2011-01-12T00:00:00.000Z", + "provider":"spot", + "quality":"automotive", + "index":100.000000 + } +] \ No newline at end of file From 2a88f8c4ce44ed2fc32a992596df218a2d979c80 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:36:05 -0700 Subject: [PATCH 070/270] Downgrade guice to 4.0-beta. --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 1aa12d28497..87b8c9f182c 100644 --- a/pom.xml +++ b/pom.xml @@ -203,17 +203,17 @@ com.google.inject guice - 4.0-beta4 + 4.0-beta com.google.inject.extensions guice-servlet - 4.0-beta4 + 4.0-beta com.google.inject.extensions guice-multibindings - 4.0-beta4 + 4.0-beta com.ibm.icu From 96be9faec702b98e07588b0491f2e7cc216a2cf7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:37:59 -0700 Subject: [PATCH 071/270] [maven-release-plugin] prepare release druid-0.6.124 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 8 +++----- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 7729770ef86..d88374396f1 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/common/pom.xml b/common/pom.xml index 4a515fc0675..6bfc191ea28 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/examples/pom.xml b/examples/pom.xml index 177c43086cb..e9525df5a7a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index d0aa54ac33f..c747b332323 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 063a60bc0f3..c889981e2fc 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 2aa51f4f21c..ad27a82ccf8 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 74cf5d1bfc6..78b7a2ab356 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index b30908f6d83..071ef41e29c 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/pom.xml b/pom.xml index 87b8c9f182c..1163f5187e5 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.124-SNAPSHOT + 0.6.124 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.124 diff --git a/processing/pom.xml b/processing/pom.xml index 3cfcf495797..c9e36773768 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 507e95d266f..b948b14a17f 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 2e7da9309d8..df83f18d357 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/server/pom.xml b/server/pom.xml index c6f55aa8a7a..78ed9c3517e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 diff --git a/services/pom.xml b/services/pom.xml index 1cb60beb8e9..34360acbc40 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -17,8 +17,7 @@ ~ along with this program; if not, write to the Free Software ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-services @@ -28,7 +27,7 @@ io.druid druid - 0.6.124-SNAPSHOT + 0.6.124 @@ -91,8 +90,7 @@ - + From ef7e786b96d4c2072f762ae56749671561b4049b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:38:08 -0700 Subject: [PATCH 072/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index d88374396f1..657c4070eb1 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 6bfc191ea28..e1a56dff2c8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index e9525df5a7a..fb7b2eaeb8b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c747b332323..2d56e5fabd7 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index c889981e2fc..0e8003619e0 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index ad27a82ccf8..dae7dc7ac42 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 78b7a2ab356..f1f98777de2 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 071ef41e29c..6bdd972f13a 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/pom.xml b/pom.xml index 1163f5187e5..4bdb1cc1413 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.124 + 0.6.125-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.124 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index c9e36773768..6d22cd6b463 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index b948b14a17f..1527c778810 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index df83f18d357..d9efb0ed44c 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 78ed9c3517e..852ad8d36a3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 34360acbc40..9da20c8854f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.124 + 0.6.125-SNAPSHOT From 3db07d2a55d5c946db4097b456b1c95b2d066f82 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:54:31 -0700 Subject: [PATCH 073/270] Make TimeBoundaryQuery buildResult no longer static. --- .../query/timeboundary/TimeBoundaryQuery.java | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index a07f83c5480..db9fa913bc9 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -52,24 +52,6 @@ public class TimeBoundaryQuery extends BaseQuery private static final byte CACHE_TYPE_ID = 0x0; - public static Iterable> buildResult(DateTime timestamp, DateTime min, DateTime max) - { - List> results = Lists.newArrayList(); - Map result = Maps.newHashMap(); - - if (min != null) { - result.put(MIN_TIME, min); - } - if (max != null) { - result.put(MAX_TIME, max); - } - if (!result.isEmpty()) { - results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result))); - } - - return results; - } - private final String bound; @JsonCreator @@ -150,6 +132,24 @@ public class TimeBoundaryQuery extends BaseQuery .array(); } + public Iterable> buildResult(DateTime timestamp, DateTime min, DateTime max) + { + List> results = Lists.newArrayList(); + Map result = Maps.newHashMap(); + + if (min != null) { + result.put(MIN_TIME, min); + } + if (max != null) { + result.put(MAX_TIME, max); + } + if (!result.isEmpty()) { + results.add(new Result<>(timestamp, new TimeBoundaryResultValue(result))); + } + + return results; + } + public Iterable> mergeResults(List> results) { if (results == null || results.isEmpty()) { From 4e184e3793a3f790be9f033527598550c38e3150 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:56:37 -0700 Subject: [PATCH 074/270] [maven-release-plugin] prepare release druid-0.6.125 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 657c4070eb1..3295e1d8154 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/common/pom.xml b/common/pom.xml index e1a56dff2c8..32a327f784d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/examples/pom.xml b/examples/pom.xml index fb7b2eaeb8b..ef74be26c35 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2d56e5fabd7..6cd995accc9 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0e8003619e0..b4c898b1845 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index dae7dc7ac42..74b28d0dea8 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index f1f98777de2..93fc9d6e1bb 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 6bdd972f13a..3e6cc2e2e93 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/pom.xml b/pom.xml index 4bdb1cc1413..bf2804f9997 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.125-SNAPSHOT + 0.6.125 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.125 diff --git a/processing/pom.xml b/processing/pom.xml index 6d22cd6b463..9b44d895566 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 1527c778810..8d3a37ea496 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index d9efb0ed44c..e9e954ac9f3 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/server/pom.xml b/server/pom.xml index 852ad8d36a3..f3551dfc9c1 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 diff --git a/services/pom.xml b/services/pom.xml index 9da20c8854f..39f33724f62 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.125-SNAPSHOT + 0.6.125 From 964f12b7d68320900045ff1da6401b6ab7db2a3d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:56:44 -0700 Subject: [PATCH 075/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 3295e1d8154..1f038e50446 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 32a327f784d..aebebbb244d 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index ef74be26c35..2e9efdd777a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 6cd995accc9..5c86e8d920d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index b4c898b1845..04583e2dfc8 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 74b28d0dea8..0d97db35c9a 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 93fc9d6e1bb..b09d8775b5a 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 3e6cc2e2e93..7a8216f197c 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/pom.xml b/pom.xml index bf2804f9997..98715e16e69 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.125 + 0.6.126-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.125 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 9b44d895566..e748601bba7 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 8d3a37ea496..62136155dc7 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index e9e954ac9f3..cb28f81dd0b 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index f3551dfc9c1..0a27559e459 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 39f33724f62..0ba5709b0c0 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.125 + 0.6.126-SNAPSHOT From 54833fa43d84f273b7a94199be67ed9f4fbcb2c6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 08:53:39 -0700 Subject: [PATCH 076/270] Indexing: Make default hadoop coordinates configurable. --- .../indexing/common/config/TaskConfig.java | 32 +++++++++++++------ .../indexing/common/task/HadoopIndexTask.java | 20 ++++++++---- 2 files changed, 37 insertions(+), 15 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index 8651a65e70d..76d5f2ed4ea 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -21,57 +21,71 @@ package io.druid.indexing.common.config; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; import java.io.File; +import java.util.List; public class TaskConfig { - @JsonProperty + public static List DEFAULT_DEFAULT_HADOOP_COORDINATES = ImmutableList.of( + "org.apache.hadoop:hadoop-client:2.3.0" + ); + private final String baseDir; - - @JsonProperty private final File baseTaskDir; - - @JsonProperty private final String hadoopWorkingPath; - - @JsonProperty private final int defaultRowFlushBoundary; + private final List defaultHadoopCoordinates; @JsonCreator public TaskConfig( @JsonProperty("baseDir") String baseDir, @JsonProperty("baseTaskDir") String baseTaskDir, @JsonProperty("hadoopWorkingPath") String hadoopWorkingPath, - @JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary + @JsonProperty("defaultRowFlushBoundary") Integer defaultRowFlushBoundary, + @JsonProperty("defaultHadoopCoordinates") List defaultHadoopCoordinates ) { this.baseDir = baseDir == null ? "/tmp" : baseDir; this.baseTaskDir = new File(defaultDir(baseTaskDir, "persistent/task")); this.hadoopWorkingPath = defaultDir(hadoopWorkingPath, "druid-indexing"); this.defaultRowFlushBoundary = defaultRowFlushBoundary == null ? 500000 : defaultRowFlushBoundary; + this.defaultHadoopCoordinates = defaultHadoopCoordinates == null + ? DEFAULT_DEFAULT_HADOOP_COORDINATES + : defaultHadoopCoordinates; } + @JsonProperty public String getBaseDir() { return baseDir; } + @JsonProperty public File getBaseTaskDir() { return baseTaskDir; } + @JsonProperty public String getHadoopWorkingPath() { return hadoopWorkingPath; } + @JsonProperty public int getDefaultRowFlushBoundary() { return defaultRowFlushBoundary; } + @JsonProperty + public List getDefaultHadoopCoordinates() + { + return defaultHadoopCoordinates; + } + private String defaultDir(String configParameter, final String defaultVal) { if (configParameter == null) { @@ -80,4 +94,4 @@ public class TaskConfig return configParameter; } -} \ No newline at end of file +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 4f017b96ce6..ce6324e32e8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -27,6 +27,7 @@ import com.google.api.client.util.Lists; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; @@ -65,8 +66,6 @@ public class HadoopIndexTask extends AbstractTask extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class); } - public static String DEFAULT_HADOOP_COORDINATES = "org.apache.hadoop:hadoop-client:2.3.0"; - private static String getTheDataSource(HadoopIngestionSpec spec, HadoopIngestionSpec config) { if (spec != null) { @@ -115,9 +114,14 @@ public class HadoopIndexTask extends AbstractTask Preconditions.checkArgument(this.spec.getTuningConfig().getWorkingPath() == null, "workingPath must be absent"); Preconditions.checkArgument(this.spec.getIOConfig().getMetadataUpdateSpec() == null, "updaterJobSpec must be absent"); - this.hadoopDependencyCoordinates = hadoopDependencyCoordinates == null ? Arrays.asList( - hadoopCoordinates == null ? DEFAULT_HADOOP_COORDINATES : hadoopCoordinates - ) : hadoopDependencyCoordinates; + if (hadoopDependencyCoordinates != null) { + this.hadoopDependencyCoordinates = hadoopDependencyCoordinates; + } else if (hadoopCoordinates != null) { + this.hadoopDependencyCoordinates = ImmutableList.of(hadoopCoordinates); + } else { + // Will be defaulted to something at runtime, based on taskConfig. + this.hadoopDependencyCoordinates = null; + } } @Override @@ -158,6 +162,10 @@ public class HadoopIndexTask extends AbstractTask @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { + final List finalHadoopDependencyCoordinates = hadoopDependencyCoordinates != null + ? hadoopDependencyCoordinates + : toolbox.getConfig().getDefaultHadoopCoordinates(); + final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig); final List extensionURLs = Lists.newArrayList(); @@ -174,7 +182,7 @@ public class HadoopIndexTask extends AbstractTask final List driverURLs = Lists.newArrayList(); driverURLs.addAll(nonHadoopURLs); // put hadoop dependencies last to avoid jets3t & apache.httpcore version conflicts - for (String hadoopDependencyCoordinate : hadoopDependencyCoordinates) { + for (String hadoopDependencyCoordinate : finalHadoopDependencyCoordinates) { final ClassLoader hadoopLoader = Initialization.getClassLoaderForCoordinates( aetherClient, hadoopDependencyCoordinate ); From 9b03f5b13a123151a075b626e93778b8df6ea2c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 17 Jun 2014 09:00:07 -0700 Subject: [PATCH 077/270] fix timeout test race condition --- .../ChainedExecutionQueryRunnerTest.java | 34 ++++++++++++------- 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 3a87e5c648d..b6469893e4b 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -65,6 +65,7 @@ public class ChainedExecutionQueryRunnerTest ); final CountDownLatch queriesStarted = new CountDownLatch(2); + final CountDownLatch queriesInterrupted = new CountDownLatch(2); final CountDownLatch queryIsRegistered = new CountDownLatch(1); Capture capturedFuture = new Capture<>(); @@ -86,9 +87,9 @@ public class ChainedExecutionQueryRunnerTest EasyMock.replay(watcher); - DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted); - DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted); - DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted); + DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted, queriesInterrupted); + DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted, queriesInterrupted); + DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted, queriesInterrupted); ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>( exec, Ordering.natural(), @@ -135,6 +136,7 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(e.getCause() instanceof QueryInterruptedException); cause = (QueryInterruptedException)e.getCause(); } + Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS)); Assert.assertNotNull(cause); Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); @@ -170,6 +172,7 @@ public class ChainedExecutionQueryRunnerTest ); final CountDownLatch queriesStarted = new CountDownLatch(2); + final CountDownLatch queriesInterrupted = new CountDownLatch(2); final CountDownLatch queryIsRegistered = new CountDownLatch(1); Capture capturedFuture = new Capture<>(); @@ -191,9 +194,9 @@ public class ChainedExecutionQueryRunnerTest EasyMock.replay(watcher); - DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted); - DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted); - DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted); + DyingQueryRunner runner1 = new DyingQueryRunner(queriesStarted, queriesInterrupted); + DyingQueryRunner runner2 = new DyingQueryRunner(queriesStarted, queriesInterrupted); + DyingQueryRunner runner3 = new DyingQueryRunner(queriesStarted, queriesInterrupted); ChainedExecutionQueryRunner chainedRunner = new ChainedExecutionQueryRunner<>( exec, Ordering.natural(), @@ -210,7 +213,7 @@ public class ChainedExecutionQueryRunnerTest .dataSource("test") .intervals("2014/2015") .aggregators(Lists.newArrayList(new CountAggregatorFactory("count"))) - .context(ImmutableMap.of("timeout", (100), "queryId", "test")) + .context(ImmutableMap.of("timeout", 100, "queryId", "test")) .build() ); @@ -229,10 +232,10 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(queryIsRegistered.await(1, TimeUnit.SECONDS)); Assert.assertTrue(queriesStarted.await(1, TimeUnit.SECONDS)); - // cancel the query Assert.assertTrue(capturedFuture.hasCaptured()); ListenableFuture future = capturedFuture.getValue(); + // wait for query to time out QueryInterruptedException cause = null; try { resultFuture.get(); @@ -241,6 +244,7 @@ public class ChainedExecutionQueryRunnerTest Assert.assertEquals("Query timeout", e.getCause().getMessage()); cause = (QueryInterruptedException)e.getCause(); } + Assert.assertTrue(queriesInterrupted.await(500, TimeUnit.MILLISECONDS)); Assert.assertNotNull(cause); Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); @@ -257,23 +261,27 @@ public class ChainedExecutionQueryRunnerTest private static class DyingQueryRunner implements QueryRunner { - private final CountDownLatch latch; + private final CountDownLatch start; + private final CountDownLatch stop; + private boolean hasStarted = false; private boolean hasCompleted = false; private boolean interrupted = false; - public DyingQueryRunner(CountDownLatch latch) + public DyingQueryRunner(CountDownLatch start, CountDownLatch stop) { - this.latch = latch; + this.start = start; + this.stop = stop; } @Override public Sequence run(Query query) { hasStarted = true; - latch.countDown(); + start.countDown(); if (Thread.interrupted()) { interrupted = true; + stop.countDown(); throw new QueryInterruptedException("I got killed"); } @@ -283,10 +291,12 @@ public class ChainedExecutionQueryRunnerTest } catch (InterruptedException e) { interrupted = true; + stop.countDown(); throw new QueryInterruptedException("I got killed"); } hasCompleted = true; + stop.countDown(); return Sequences.simple(Lists.newArrayList(123)); } } From 4c4abbbb0cd9da249c5c500d2951c59acbcf56c6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 17 Jun 2014 09:07:32 -0700 Subject: [PATCH 078/270] update maven-release-plugin to 2.5 --- pom.xml | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 98715e16e69..9f64111908c 100644 --- a/pom.xml +++ b/pom.xml @@ -561,15 +561,7 @@ org.apache.maven.plugins maven-release-plugin - 2.4.2 - - - org.apache.maven.scm - maven-scm-provider-gitexe - - 1.8.1 - - + 2.5 From 4a835a9f0f63e7c65b728fd9fa7637d95e185d18 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 17 Jun 2014 10:07:20 -0700 Subject: [PATCH 079/270] fix initialization test static state dependency --- .../druid/initialization/Initialization.java | 8 ++++ .../initialization/InitializationTest.java | 45 +++++++++---------- 2 files changed, 28 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 292aa3c95cf..ed4674aa5d8 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -120,6 +120,14 @@ public class Initialization return retVal; } + /** + * Used for testing only + */ + protected static void clearLoadedModules() + { + extensionsMap.clear(); + } + public synchronized static Collection getFromExtensions(ExtensionsConfig config, Class clazz) { final TeslaAether aether = getAetherClient(config); diff --git a/server/src/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java index d1cb0afd0c5..8e93336d967 100644 --- a/server/src/test/java/io/druid/initialization/InitializationTest.java +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -26,10 +26,12 @@ import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Injector; +import com.google.inject.Key; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.server.DruidNode; import io.druid.server.initialization.ExtensionsConfig; -import junit.framework.Assert; -import org.junit.After; -import org.junit.Before; +import org.junit.Assert; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -42,30 +44,10 @@ import java.util.Set; @FixMethodOrder(MethodSorters.NAME_ASCENDING) public class InitializationTest { - private String oldService; - private String oldHost; - private String oldPort; - - @Before - public void messWithSystemProperties() - { - // required to test Initialization.makeInjectorWithModules - oldService = System.setProperty("druid.service", "test-service"); - oldHost = System.setProperty("druid.host", "test-host"); - oldPort = System.setProperty("druid.port", "8080"); - } - - @After - public void cleanup() - { - System.setProperty("druid.service", oldService == null ? "" : oldService); - System.setProperty("druid.host", oldHost == null ? "" : oldHost); - System.setProperty("druid.port", oldPort == null ? "" : oldPort); - } - @Test public void test01InitialModulesEmpty() throws Exception { + Initialization.clearLoadedModules(); Assert.assertEquals( "Initial set of loaded modules must be empty", 0, @@ -118,7 +100,20 @@ public class InitializationTest public void test04MakeInjectorWithModules() throws Exception { Injector startupInjector = Initialization.makeStartupInjector(); - Injector injector = Initialization.makeInjectorWithModules(startupInjector, ImmutableList.of()); + Injector injector = Initialization.makeInjectorWithModules( + startupInjector, ImmutableList.of( + new com.google.inject.Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, Key.get(DruidNode.class, Self.class), new DruidNode("hadoop-indexer", "localhost", -1) + ); + } + } + ) + ); Assert.assertNotNull(injector); } From d1c462703266a610822f8a2cd239e614dd828727 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Jun 2014 10:18:46 -0700 Subject: [PATCH 080/270] fix bridge test --- .../io/druid/server/bridge/DruidClusterBridgeTest.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java index b7ab0e19714..d90b0cf3792 100644 --- a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java +++ b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java @@ -230,5 +230,13 @@ public class DruidClusterBridgeTest Assert.assertEquals(118, announced.getMaxSize()); EasyMock.verify(batchServerInventoryView); + + announcer.stop(); + bridge.stop(); + + remoteCf.close(); + remoteCluster.close(); + localCf.close(); + localCluster.close(); } } From 8a13e34c50ed22cd9371938c652f8eb88700440c Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Jun 2014 10:31:50 -0700 Subject: [PATCH 081/270] fix broken ut --- .../java/io/druid/indexing/overlord/TaskLifecycleTest.java | 2 +- .../java/io/druid/server/bridge/DruidClusterBridgeTest.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 30325c9d398..370952edfda 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -135,7 +135,7 @@ public class TaskLifecycleTest mdc = newMockMDC(); tac = new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tl, mdc, newMockEmitter())); tb = new TaskToolboxFactory( - new TaskConfig(tmp.toString(), null, null, 50000), + new TaskConfig(tmp.toString(), null, null, 50000, null), tac, newMockEmitter(), new DataSegmentPusher() diff --git a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java index d90b0cf3792..b970f17778a 100644 --- a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java +++ b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java @@ -183,6 +183,8 @@ public class DruidClusterBridgeTest EasyMock.expectLastCall(); batchServerInventoryView.start(); EasyMock.expectLastCall(); + batchServerInventoryView.stop(); + EasyMock.expectLastCall(); EasyMock.replay(batchServerInventoryView); @@ -229,10 +231,10 @@ public class DruidClusterBridgeTest Assert.assertEquals(118, announced.getMaxSize()); + bridge.stop(); EasyMock.verify(batchServerInventoryView); announcer.stop(); - bridge.stop(); remoteCf.close(); remoteCluster.close(); From 5227b43050fc2f81ecc4b91564c4f0ffc1087f68 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Jun 2014 10:35:01 -0700 Subject: [PATCH 082/270] fix more test breakage --- .../java/io/druid/indexing/worker/WorkerTaskMonitorTest.java | 2 +- services/src/main/java/io/druid/cli/PullDependencies.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 8d4bf32b870..bad6fae4939 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -121,7 +121,7 @@ public class WorkerTaskMonitorTest workerCuratorCoordinator, new ThreadPoolTaskRunner( new TaskToolboxFactory( - new TaskConfig(tmp.toString(), null, null, 0), + new TaskConfig(tmp.toString(), null, null, 0, null), null, null, null, null, null, null, null, null, null, null, null, new SegmentLoaderFactory( new OmniSegmentLoader( ImmutableMap.of( diff --git a/services/src/main/java/io/druid/cli/PullDependencies.java b/services/src/main/java/io/druid/cli/PullDependencies.java index c6d7a84885d..c151a86ea3b 100644 --- a/services/src/main/java/io/druid/cli/PullDependencies.java +++ b/services/src/main/java/io/druid/cli/PullDependencies.java @@ -24,6 +24,7 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; import io.airlift.command.Command; import io.airlift.command.Option; +import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.HadoopIndexTask; import io.druid.initialization.Initialization; import io.druid.server.initialization.ExtensionsConfig; @@ -62,7 +63,7 @@ public class PullDependencies implements Runnable allCoordinates.addAll(coordinates); } if (!noDefaultHadoop) { - allCoordinates.add(HadoopIndexTask.DEFAULT_HADOOP_COORDINATES); + allCoordinates.addAll(TaskConfig.DEFAULT_DEFAULT_HADOOP_COORDINATES); } try { final DefaultTeslaAether aetherClient = Initialization.getAetherClient(extensionsConfig); From 3dbf4757841914ec5939f7fb215c65b0474eeda2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 17 Jun 2014 11:05:36 -0700 Subject: [PATCH 083/270] fix linux test failures --- .../server/bridge/DruidClusterBridge.java | 2 +- .../server/bridge/DruidClusterBridgeTest.java | 40 ++++++++++++++----- 2 files changed, 32 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java b/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java index f0e7355a1c6..5424a8af8ad 100644 --- a/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java +++ b/server/src/main/java/io/druid/server/bridge/DruidClusterBridge.java @@ -253,7 +253,7 @@ public class DruidClusterBridge log.warn(e, "Unable to close leaderLatch, ignoring"); } - exec.shutdownNow(); + exec.shutdown(); started = false; } diff --git a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java index b970f17778a..59f13d74363 100644 --- a/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java +++ b/server/src/test/java/io/druid/server/bridge/DruidClusterBridgeTest.java @@ -148,8 +148,12 @@ public class DruidClusterBridgeTest 0 ); DbSegmentPublisher dbSegmentPublisher = EasyMock.createMock(DbSegmentPublisher.class); + EasyMock.replay(dbSegmentPublisher); DatabaseSegmentManager databaseSegmentManager = EasyMock.createMock(DatabaseSegmentManager.class); + EasyMock.replay(databaseSegmentManager); ServerView serverView = EasyMock.createMock(ServerView.class); + EasyMock.replay(serverView); + BridgeZkCoordinator bridgeZkCoordinator = new BridgeZkCoordinator( jsonMapper, zkPathsConfig, @@ -163,9 +167,9 @@ public class DruidClusterBridgeTest Announcer announcer = new Announcer(remoteCf, Executors.newSingleThreadExecutor()); announcer.start(); announcer.announce(zkPathsConfig.getAnnouncementsPath() + "/" + me.getHost(), jsonMapper.writeValueAsBytes(me)); + BatchDataSegmentAnnouncer batchDataSegmentAnnouncer = EasyMock.createMock(BatchDataSegmentAnnouncer.class); BatchServerInventoryView batchServerInventoryView = EasyMock.createMock(BatchServerInventoryView.class); - EasyMock.expect(batchServerInventoryView.getInventory()).andReturn( Arrays.asList( new DruidServer("1", "localhost", 117, "historical", DruidServer.DEFAULT_TIER, 0), @@ -187,7 +191,6 @@ public class DruidClusterBridgeTest EasyMock.expectLastCall(); EasyMock.replay(batchServerInventoryView); - DruidClusterBridge bridge = new DruidClusterBridge( jsonMapper, config, @@ -224,21 +227,40 @@ public class DruidClusterBridgeTest retry++; } - DruidServerMetadata announced = jsonMapper.readValue( - remoteCf.getData().forPath(path), - DruidServerMetadata.class - ); + boolean verified = verifyUpdate(jsonMapper, path, remoteCf); + retry = 0; + while (!verified) { + if (retry > 5) { + throw new ISE("No updates to bridge node occurred"); + } - Assert.assertEquals(118, announced.getMaxSize()); + Thread.sleep(100); + retry++; - bridge.stop(); - EasyMock.verify(batchServerInventoryView); + verified = verifyUpdate(jsonMapper, path, remoteCf); + } announcer.stop(); + bridge.stop(); remoteCf.close(); remoteCluster.close(); localCf.close(); localCluster.close(); + + EasyMock.verify(batchServerInventoryView); + EasyMock.verify(dbSegmentPublisher); + EasyMock.verify(databaseSegmentManager); + EasyMock.verify(serverView); + } + + private boolean verifyUpdate(ObjectMapper jsonMapper, String path, CuratorFramework remoteCf) throws Exception + { + DruidServerMetadata announced = jsonMapper.readValue( + remoteCf.getData().forPath(path), + DruidServerMetadata.class + ); + + return (118 == announced.getMaxSize()); } } From 4e875b8084880a4ef03d50f758d1f71c4f9f9a60 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 13:00:54 -0700 Subject: [PATCH 084/270] [maven-release-plugin] prepare release druid-0.6.126 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 1f038e50446..7ef782ed51e 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/common/pom.xml b/common/pom.xml index aebebbb244d..936251f3e11 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/examples/pom.xml b/examples/pom.xml index 2e9efdd777a..71890b09d4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 5c86e8d920d..60887880338 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 04583e2dfc8..2927440dcf3 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 0d97db35c9a..66d9b6b5a39 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index b09d8775b5a..9182e02dcd5 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 7a8216f197c..64b5a6e640e 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/pom.xml b/pom.xml index 9f64111908c..357be82426a 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.126-SNAPSHOT + 0.6.126 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.126 diff --git a/processing/pom.xml b/processing/pom.xml index e748601bba7..32c6fd34808 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 62136155dc7..04a7fc56da3 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index cb28f81dd0b..62ef0e6b8ed 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/server/pom.xml b/server/pom.xml index 0a27559e459..51ce4d99221 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 diff --git a/services/pom.xml b/services/pom.xml index 0ba5709b0c0..f13e5cc73f5 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.126-SNAPSHOT + 0.6.126 From a658afe902cd6d2629f568aa8960591abf4aa993 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 13:01:01 -0700 Subject: [PATCH 085/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 7ef782ed51e..28b47eb07d3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 936251f3e11..2aee446a2c8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 71890b09d4a..0ce76c897c9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 60887880338..2d18f2f80b5 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 2927440dcf3..573417faa42 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 66d9b6b5a39..4890a0dd64f 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 9182e02dcd5..979f3d8f404 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 64b5a6e640e..1f4cf9d456c 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/pom.xml b/pom.xml index 357be82426a..92ce77f1b7f 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.126 + 0.6.127-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.126 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 32c6fd34808..cbd52f824e6 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 04a7fc56da3..f97ee885e13 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 62ef0e6b8ed..bc112819129 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 51ce4d99221..6ca648903ba 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f13e5cc73f5..4763586328d 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.126 + 0.6.127-SNAPSHOT From 9f16f0a955af30f3db5c5195bf6fefb62b238803 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 17:10:17 -0700 Subject: [PATCH 086/270] More flexible EC2 user data classes. --- .../scaling/EC2AutoScalingStrategy.java | 33 +++---- .../indexing/overlord/setup/EC2UserData.java | 42 +++++++++ ...xyUserData.java => GalaxyEC2UserData.java} | 34 +++++-- .../overlord/setup/StringEC2UserData.java | 90 +++++++++++++++++++ .../overlord/setup/WorkerSetupData.java | 6 +- .../io/druid/indexing/common/TestUtils.java | 32 +++++++ .../overlord/WorkerSetupDataTest.java | 62 +++++++++++++ .../scaling/EC2AutoScalingStrategyTest.java | 5 +- 8 files changed, 269 insertions(+), 35 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2UserData.java rename indexing-service/src/main/java/io/druid/indexing/overlord/setup/{GalaxyUserData.java => GalaxyEC2UserData.java} (63%) create mode 100644 indexing-service/src/main/java/io/druid/indexing/overlord/setup/StringEC2UserData.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/overlord/WorkerSetupDataTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java index 9081e7323d1..9a1a0db8bc2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java @@ -29,19 +29,15 @@ import com.amazonaws.services.ec2.model.Reservation; import com.amazonaws.services.ec2.model.RunInstancesRequest; import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; -import io.druid.guice.annotations.Json; import io.druid.indexing.overlord.setup.EC2NodeData; -import io.druid.indexing.overlord.setup.GalaxyUserData; +import io.druid.indexing.overlord.setup.EC2UserData; import io.druid.indexing.overlord.setup.WorkerSetupData; -import org.apache.commons.codec.binary.Base64; -import javax.annotation.Nullable; import java.util.List; /** @@ -50,20 +46,17 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy { private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class); - private final ObjectMapper jsonMapper; private final AmazonEC2 amazonEC2Client; private final SimpleResourceManagementConfig config; private final Supplier workerSetupDataRef; @Inject public EC2AutoScalingStrategy( - @Json ObjectMapper jsonMapper, AmazonEC2 amazonEC2Client, SimpleResourceManagementConfig config, Supplier workerSetupDataRef ) { - this.jsonMapper = jsonMapper; this.amazonEC2Client = amazonEC2Client; this.config = config; this.workerSetupDataRef = workerSetupDataRef; @@ -73,15 +66,17 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy public AutoScalingData provision() { try { - WorkerSetupData setupData = workerSetupDataRef.get(); - EC2NodeData workerConfig = setupData.getNodeData(); + final WorkerSetupData setupData = workerSetupDataRef.get(); + final EC2NodeData workerConfig = setupData.getNodeData(); + final EC2UserData userData; - GalaxyUserData userData = setupData.getUserData(); - if (config.getWorkerVersion() != null) { - userData = userData.withVersion(config.getWorkerVersion()); + if (config.getWorkerVersion() == null) { + userData = setupData.getUserData(); + } else { + userData = setupData.getUserData().withVersion(config.getWorkerVersion()); } - RunInstancesResult result = amazonEC2Client.runInstances( + final RunInstancesResult result = amazonEC2Client.runInstances( new RunInstancesRequest( workerConfig.getAmiId(), workerConfig.getMinInstances(), @@ -91,16 +86,10 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy .withSecurityGroupIds(workerConfig.getSecurityGroupIds()) .withPlacement(new Placement(setupData.getAvailabilityZone())) .withKeyName(workerConfig.getKeyName()) - .withUserData( - Base64.encodeBase64String( - jsonMapper.writeValueAsBytes( - userData - ) - ) - ) + .withUserData(userData.getUserDataBase64()) ); - List instanceIds = Lists.transform( + final List instanceIds = Lists.transform( result.getReservation().getInstances(), new Function() { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2UserData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2UserData.java new file mode 100644 index 00000000000..3568bf45b7b --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/EC2UserData.java @@ -0,0 +1,42 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.overlord.setup; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Represents any user data that may be needed to launch EC2 instances. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "impl", defaultImpl = GalaxyEC2UserData.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "galaxy", value = GalaxyEC2UserData.class), + @JsonSubTypes.Type(name = "string", value = StringEC2UserData.class) +}) +public interface EC2UserData +{ + /** + * Return a copy of this instance with a different worker version. If no changes are needed (possibly because the + * user data does not depend on the worker version) then it is OK to return "this". + */ + public EC2UserData withVersion(String version); + + public String getUserDataBase64(); +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyUserData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyEC2UserData.java similarity index 63% rename from indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyUserData.java rename to indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyEC2UserData.java index ee721e9ac8c..bfc822dd52c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyUserData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/GalaxyEC2UserData.java @@ -19,24 +19,32 @@ package io.druid.indexing.overlord.setup; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import io.druid.guice.annotations.Json; +import org.apache.commons.codec.binary.Base64; /** */ -public class GalaxyUserData +public class GalaxyEC2UserData implements EC2UserData { - public final String env; - public final String version; - public final String type; + private final ObjectMapper jsonMapper; + private final String env; + private final String version; + private final String type; @JsonCreator - public GalaxyUserData( + public GalaxyEC2UserData( + @JacksonInject @Json ObjectMapper jsonMapper, @JsonProperty("env") String env, @JsonProperty("version") String version, @JsonProperty("type") String type ) { + this.jsonMapper = jsonMapper; this.env = env; this.version = version; this.type = type; @@ -60,9 +68,21 @@ public class GalaxyUserData return type; } - public GalaxyUserData withVersion(String ver) + @Override + public GalaxyEC2UserData withVersion(String ver) { - return new GalaxyUserData(env, ver, type); + return new GalaxyEC2UserData(jsonMapper, env, ver, type); + } + + @Override + public String getUserDataBase64() + { + try { + return Base64.encodeBase64String(jsonMapper.writeValueAsBytes(this)); + } + catch (Exception e) { + throw Throwables.propagate(e); + } } @Override diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/StringEC2UserData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/StringEC2UserData.java new file mode 100644 index 00000000000..50f61fc1f67 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/StringEC2UserData.java @@ -0,0 +1,90 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.overlord.setup; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.util.Charsets; +import org.apache.commons.codec.binary.Base64; + +public class StringEC2UserData implements EC2UserData +{ + private final String data; + private final String versionReplacementString; + private final String version; + + @JsonCreator + public StringEC2UserData( + @JsonProperty("data") String data, + @JsonProperty("versionReplacementString") String versionReplacementString, + @JsonProperty("version") String version + ) + { + this.data = data; + this.versionReplacementString = versionReplacementString; + this.version = version; + } + + @JsonProperty + public String getData() + { + return data; + } + + @JsonProperty + public String getVersionReplacementString() + { + return versionReplacementString; + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @Override + public StringEC2UserData withVersion(final String _version) + { + return new StringEC2UserData(data, versionReplacementString, _version); + } + + @Override + public String getUserDataBase64() + { + final String finalData; + if (versionReplacementString != null && version != null) { + finalData = data.replace(versionReplacementString, version); + } else { + finalData = data; + } + return Base64.encodeBase64String(finalData.getBytes(Charsets.UTF_8)); + } + + @Override + public String toString() + { + return "StringEC2UserData{" + + "data='" + data + '\'' + + ", versionReplacementString='" + versionReplacementString + '\'' + + ", version='" + version + '\'' + + '}'; + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java index 47bb145eefe..e792f347aed 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java @@ -33,7 +33,7 @@ public class WorkerSetupData private final int maxNumWorkers; private final String availabilityZone; private final EC2NodeData nodeData; - private final GalaxyUserData userData; + private final EC2UserData userData; @JsonCreator public WorkerSetupData( @@ -42,7 +42,7 @@ public class WorkerSetupData @JsonProperty("maxNumWorkers") int maxNumWorkers, @JsonProperty("availabilityZone") String availabilityZone, @JsonProperty("nodeData") EC2NodeData nodeData, - @JsonProperty("userData") GalaxyUserData userData + @JsonProperty("userData") EC2UserData userData ) { this.minVersion = minVersion; @@ -84,7 +84,7 @@ public class WorkerSetupData } @JsonProperty - public GalaxyUserData getUserData() + public EC2UserData getUserData() { return userData; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index 2ec6c474e6c..b669c65e125 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -19,15 +19,47 @@ package io.druid.indexing.common; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Stopwatch; import com.metamx.common.ISE; +import io.druid.guice.ServerModule; +import io.druid.jackson.DefaultObjectMapper; +import java.util.List; import java.util.concurrent.TimeUnit; /** */ public class TestUtils { + public static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + static { + final List list = new ServerModule().getJacksonModules(); + for (Module module : list) { + MAPPER.registerModule(module); + } + MAPPER.setInjectableValues( + new InjectableValues() + { + @Override + public Object findInjectableValue( + Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance + ) + { + if (valueId.equals("com.fasterxml.jackson.databind.ObjectMapper")) { + return TestUtils.MAPPER; + } + throw new ISE("No Injectable value found"); + } + } + ); + } + public static boolean conditionValid(IndexingServiceCondition condition) { try { diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/WorkerSetupDataTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/WorkerSetupDataTest.java new file mode 100644 index 00000000000..39b35dd951a --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/WorkerSetupDataTest.java @@ -0,0 +1,62 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.overlord; + +import com.google.common.base.Charsets; +import io.druid.indexing.common.TestUtils; +import io.druid.indexing.overlord.setup.EC2UserData; +import io.druid.indexing.overlord.setup.GalaxyEC2UserData; +import io.druid.indexing.overlord.setup.StringEC2UserData; +import org.apache.commons.codec.binary.Base64; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class WorkerSetupDataTest +{ + @Test + public void testGalaxyEC2UserDataSerde() throws IOException + { + final String json = "{\"env\":\"druid\",\"version\":null,\"type\":\"typical\"}"; + final GalaxyEC2UserData userData = (GalaxyEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class); + Assert.assertEquals("druid", userData.getEnv()); + Assert.assertEquals("typical", userData.getType()); + Assert.assertNull(userData.getVersion()); + Assert.assertEquals("1234", userData.withVersion("1234").getVersion()); + } + + @Test + public void testStringEC2UserDataSerde() throws IOException + { + final String json = "{\"impl\":\"string\",\"data\":\"hey :ver:\",\"versionReplacementString\":\":ver:\",\"version\":\"1234\"}"; + final StringEC2UserData userData = (StringEC2UserData) TestUtils.MAPPER.readValue(json, EC2UserData.class); + Assert.assertEquals("hey :ver:", userData.getData()); + Assert.assertEquals("1234", userData.getVersion()); + Assert.assertEquals( + Base64.encodeBase64String("hey 1234".getBytes(Charsets.UTF_8)), + userData.getUserDataBase64() + ); + Assert.assertEquals( + Base64.encodeBase64String("hey xyz".getBytes(Charsets.UTF_8)), + userData.withVersion("xyz").getUserDataBase64() + ); + } +} diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java index 1fd3510b45a..1ccacc66df4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java @@ -30,7 +30,7 @@ import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Lists; import io.druid.common.guava.DSuppliers; import io.druid.indexing.overlord.setup.EC2NodeData; -import io.druid.indexing.overlord.setup.GalaxyUserData; +import io.druid.indexing.overlord.setup.GalaxyEC2UserData; import io.druid.indexing.overlord.setup.WorkerSetupData; import io.druid.jackson.DefaultObjectMapper; import org.easymock.EasyMock; @@ -75,7 +75,6 @@ public class EC2AutoScalingStrategyTest .withPrivateIpAddress(IP); strategy = new EC2AutoScalingStrategy( - new DefaultObjectMapper(), amazonEC2Client, new SimpleResourceManagementConfig().setWorkerPort(8080).setWorkerVersion(""), DSuppliers.of(workerSetupData) @@ -101,7 +100,7 @@ public class EC2AutoScalingStrategyTest 1, "", new EC2NodeData(AMI_ID, INSTANCE_ID, 1, 1, Lists.newArrayList(), "foo"), - new GalaxyUserData("env", "version", "type") + new GalaxyEC2UserData(new DefaultObjectMapper(), "env", "version", "type") ) ); From 65c1553a7617a0ee9b76146f946e11174cabdd92 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 17:27:41 -0700 Subject: [PATCH 087/270] Fix busted TaskConfig. --- .../java/io/druid/indexing/common/config/TaskConfig.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java index 76d5f2ed4ea..1671fb84bd3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/config/TaskConfig.java @@ -32,10 +32,19 @@ public class TaskConfig "org.apache.hadoop:hadoop-client:2.3.0" ); + @JsonProperty private final String baseDir; + + @JsonProperty private final File baseTaskDir; + + @JsonProperty private final String hadoopWorkingPath; + + @JsonProperty private final int defaultRowFlushBoundary; + + @JsonProperty private final List defaultHadoopCoordinates; @JsonCreator From e3dcbde07dbb56667eceb80f553903b4797c2957 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 17:29:27 -0700 Subject: [PATCH 088/270] [maven-release-plugin] prepare release druid-0.6.127 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 28b47eb07d3..86e1b98b29e 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/common/pom.xml b/common/pom.xml index 2aee446a2c8..1ef3ca297c1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/examples/pom.xml b/examples/pom.xml index 0ce76c897c9..6ad9ffd9569 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 2d18f2f80b5..9853d8f35bf 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 573417faa42..4bbff4e67ff 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 4890a0dd64f..69f523a3ff1 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 979f3d8f404..89336222c61 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 1f4cf9d456c..0cfed0c2ff9 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/pom.xml b/pom.xml index 92ce77f1b7f..c1d0c45f4ff 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.127-SNAPSHOT + 0.6.127 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.127 diff --git a/processing/pom.xml b/processing/pom.xml index cbd52f824e6..736a50b119a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index f97ee885e13..20c2e778a82 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index bc112819129..880b260d453 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/server/pom.xml b/server/pom.xml index 6ca648903ba..4caedc8c1ec 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 diff --git a/services/pom.xml b/services/pom.xml index 4763586328d..632cb7b47f2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.127-SNAPSHOT + 0.6.127 From 9532beb05c019d3eabcd205fea32c10ef7ef82f2 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Jun 2014 17:29:31 -0700 Subject: [PATCH 089/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86e1b98b29e..f37c196515c 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 1ef3ca297c1..40186eb00a8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 6ad9ffd9569..d75d4352dc2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 9853d8f35bf..f7467603864 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4bbff4e67ff..002584ec84a 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 69f523a3ff1..40549863f0f 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 89336222c61..9c5c327258f 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 0cfed0c2ff9..d079c5c1218 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/pom.xml b/pom.xml index c1d0c45f4ff..f0ad93db180 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.127 + 0.6.128-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.127 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 736a50b119a..49e616b3687 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 20c2e778a82..48d7837f12c 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 880b260d453..cbb0dfacc85 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 4caedc8c1ec..2089817e33e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 632cb7b47f2..b953a4804a2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.127 + 0.6.128-SNAPSHOT From e9ef1e956cdab13455011972c4e8c972e0725b47 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Jun 2014 09:50:37 -0700 Subject: [PATCH 090/270] Docs for defaultHadoopCoordinates. --- docs/content/Peons.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/Peons.md b/docs/content/Peons.md index b5d16e1df2a..87b99fad362 100644 --- a/docs/content/Peons.md +++ b/docs/content/Peons.md @@ -22,6 +22,7 @@ Additional peon configs include: |`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|/tmp/persistent/tasks| |`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|/tmp/druid-indexing| |`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|50000| +|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.3.0| |`druid.indexer.task.chathandler.type`|Choices are "noop" and "announce". Certain tasks will use service discovery to announce an HTTP endpoint that events can be posted to.|noop| If the peon is running in remote mode, there must be an overlord up and running. Running peons in remote mode require the following configurations: From 5c4b9a7d5b2d3094b162628302e1b2fee9ed0709 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Jun 2014 10:08:50 -0700 Subject: [PATCH 091/270] Indexing: Allow null userData. --- .../overlord/scaling/EC2AutoScalingStrategy.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java index 9a1a0db8bc2..421530f520d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategy.java @@ -35,7 +35,6 @@ import com.google.common.collect.Lists; import com.google.inject.Inject; import com.metamx.emitter.EmittingLogger; import io.druid.indexing.overlord.setup.EC2NodeData; -import io.druid.indexing.overlord.setup.EC2UserData; import io.druid.indexing.overlord.setup.WorkerSetupData; import java.util.List; @@ -68,12 +67,16 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy try { final WorkerSetupData setupData = workerSetupDataRef.get(); final EC2NodeData workerConfig = setupData.getNodeData(); - final EC2UserData userData; + final String userDataBase64; - if (config.getWorkerVersion() == null) { - userData = setupData.getUserData(); + if (setupData.getUserData() == null) { + userDataBase64 = null; } else { - userData = setupData.getUserData().withVersion(config.getWorkerVersion()); + if (config.getWorkerVersion() == null) { + userDataBase64 = setupData.getUserData().getUserDataBase64(); + } else { + userDataBase64 = setupData.getUserData().withVersion(config.getWorkerVersion()).getUserDataBase64(); + } } final RunInstancesResult result = amazonEC2Client.runInstances( @@ -86,7 +89,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy .withSecurityGroupIds(workerConfig.getSecurityGroupIds()) .withPlacement(new Placement(setupData.getAvailabilityZone())) .withKeyName(workerConfig.getKeyName()) - .withUserData(userData.getUserDataBase64()) + .withUserData(userDataBase64) ); final List instanceIds = Lists.transform( From eab9381ba10a1a6429952474b18161a3cd5429c6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Jun 2014 10:09:07 -0700 Subject: [PATCH 092/270] Docs for string-based EC2 userData. --- docs/content/Indexing-Service-Config.md | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/content/Indexing-Service-Config.md b/docs/content/Indexing-Service-Config.md index 0cd9e9978ea..122a6623504 100644 --- a/docs/content/Indexing-Service-Config.md +++ b/docs/content/Indexing-Service-Config.md @@ -69,10 +69,9 @@ A sample worker setup spec is shown below: "keyName":"keyName" }, "userData":{ - "classType":"galaxy", - "env":"druid", - "version":"druid_version", - "type":"sample_cluster/worker" + "impl":"string", + "data":"version=:VERSION:", + "versionReplacementString":":VERSION:" } } ``` @@ -81,8 +80,8 @@ Issuing a GET request at the same URL will return the current worker setup spec |Property|Description|Default| |--------|-----------|-------| -|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be the same as the coordinator version.|none| +|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be druid.indexer.runner.minWorkerVersion.|none| |`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0| |`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0| -|`nodeData`|A JSON object that contains metadata about new nodes to create.|none| -|`userData`|A JSON object that contains metadata about how the node should register itself on startup. This data is sent with node creation requests.|none| +|`nodeData`|A JSON object that describes how to launch new nodes. Currently, only EC2 is supported.|none; required| +|`userData`|A JSON object that describes how to configure new nodes. Currently, only EC2 is supported. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary.|none; optional| From a52a1d3929ef1caf54d5920a11e1c023a07ab18f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Jun 2014 10:53:36 -0700 Subject: [PATCH 093/270] [maven-release-plugin] prepare release druid-0.6.128 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index f37c196515c..3f4fdb15630 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/common/pom.xml b/common/pom.xml index 40186eb00a8..2d464a9960a 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/examples/pom.xml b/examples/pom.xml index d75d4352dc2..6f33eae5608 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index f7467603864..e45603280d8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 002584ec84a..5ed932a11ab 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 40549863f0f..cea5ecd9138 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 9c5c327258f..6ab965ede74 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index d079c5c1218..825f49de13d 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/pom.xml b/pom.xml index f0ad93db180..ffeea1730cc 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.128-SNAPSHOT + 0.6.128 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.128 diff --git a/processing/pom.xml b/processing/pom.xml index 49e616b3687..f56305d768b 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 48d7837f12c..e77a14a38ad 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index cbb0dfacc85..f21f3747f3a 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/server/pom.xml b/server/pom.xml index 2089817e33e..7e8bf780e92 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 diff --git a/services/pom.xml b/services/pom.xml index b953a4804a2..cb6f4eb0773 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.128-SNAPSHOT + 0.6.128 From 7c7d170e788c67d26fa06b3db59f7bcf72aac78c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 18 Jun 2014 10:53:40 -0700 Subject: [PATCH 094/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 3f4fdb15630..d129375e3b9 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 2d464a9960a..7dc9cfa1e38 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 6f33eae5608..5789834bac3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index e45603280d8..7d8308c04b7 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 5ed932a11ab..7d2a9c8de06 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index cea5ecd9138..aa45a4e2b81 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 6ab965ede74..05393555a35 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 825f49de13d..014d7af0304 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/pom.xml b/pom.xml index ffeea1730cc..c781e283164 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.128 + 0.6.129-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.128 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index f56305d768b..755d2f553f7 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e77a14a38ad..7ed54d2833d 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index f21f3747f3a..8d9e4fae39e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 7e8bf780e92..8fa5e300885 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index cb6f4eb0773..f29c5d681cc 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.128 + 0.6.129-SNAPSHOT From b3aad48f47d76126305968e7d69229282b872d3e Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 18 Jun 2014 14:43:57 -0700 Subject: [PATCH 095/270] remove snapshot dep --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9b9e6d59051..9580f7aac4f 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.5 2.5.0 - 0.2.4-SNAPSHOT + 0.2.4 From 3164228ecf6fa2d3c71bad7a34371fe6e3b7dd65 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 19 Jun 2014 12:43:31 -0700 Subject: [PATCH 096/270] Fix example middle manager configs. --- docs/content/Production-Cluster-Configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 76c481ec7bc..cb98fdf9d43 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -153,7 +153,7 @@ druid.indexer.logs.s3Bucket=#{LOGS_BUCKET} druid.indexer.logs.s3Prefix=prod/logs/v1 # Dedicate more resources to peons -druid.indexer.runner.javaOpts=-server -Xmx6g -Xms6g -XX:NewSize=256m -XX:MaxNewSize=256m -XX:+PrintGCDetails -XX:+PrintGCTimeStamps +druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps druid.indexer.runner.taskDir=/mnt/persistent/task/ druid.indexer.task.taskDir=/mnt/persistent/task/ druid.indexer.task.chathandler.type=announce From f1759b450c992ad6f43644f31da6dc0e0413a3b8 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 12:55:02 -0700 Subject: [PATCH 097/270] fix Zk setup problem --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 13 +++++++++++++ docs/content/Tutorial:-The-Druid-Cluster.md | 2 +- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 0afd5a36416..8ce5b810a77 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -69,6 +69,19 @@ You should see a bunch of files: * run_example_client.sh * LICENSE, config, examples, lib directories +#### Setting up Zookeeper + +Before we get started, we need to start Apache Zookeeper. + +```bash +curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz +tar xzf zookeeper-3.4.5.tar.gz +cd zookeeper-3.4.5 +cp conf/zoo_sample.cfg conf/zoo.cfg +./bin/zkServer.sh start +cd .. +``` + Running Example Scripts ----------------------- diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 259e4d3eb07..91fc08fc536 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -48,7 +48,7 @@ CREATE database druid; #### Setting up Zookeeper ```bash -curl http://www.motorlogy.com/apache/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz +curl http://apache.osuosl.org/zookeeper/zookeeper-3.4.5/zookeeper-3.4.5.tar.gz -o zookeeper-3.4.5.tar.gz tar xzf zookeeper-3.4.5.tar.gz cd zookeeper-3.4.5 cp conf/zoo_sample.cfg conf/zoo.cfg From 9aa018925de301662c61146ee75c4c56b7adecca Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 12:59:58 -0700 Subject: [PATCH 098/270] fix doc header --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 8ce5b810a77..f5fbbc34712 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -69,7 +69,8 @@ You should see a bunch of files: * run_example_client.sh * LICENSE, config, examples, lib directories -#### Setting up Zookeeper +Setting up Zookeeper +-------------------- Before we get started, we need to start Apache Zookeeper. From fdb39d57a34f50d925511e43e67ead24fd055e2e Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 13:10:08 -0700 Subject: [PATCH 099/270] fix another doc problem --- .../Tutorial:-Loading-Your-Data-Part-1.md | 31 ++++++++----------- examples/config/overlord/runtime.properties | 31 ++++++++----------- 2 files changed, 26 insertions(+), 36 deletions(-) diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index eb1d71c35c3..56ca0d6305f 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -85,28 +85,23 @@ config/overlord/runtime.properties The configurations for the overlord node are as follows: ```bash --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 +druid.host=localhost +druid.port=8087 +druid.service=overlord --Ddruid.host=localhost --Ddruid.port=8080 --Ddruid.service=overlord +druid.zk.service.host=localhost --Ddruid.zk.service.host=localhost +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid +druid.db.connector.user=druid +druid.db.connector.password=diurd --Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid --Ddruid.db.connector.user=druid --Ddruid.db.connector.password=diurd - --Ddruid.selectors.indexing.serviceName=overlord --Ddruid.indexer.queue.startDelay=PT0M --Ddruid.indexer.runner.javaOpts="-server -Xmx1g" --Ddruid.indexer.fork.property.druid.processing.numThreads=1 --Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000 +druid.selectors.indexing.serviceName=overlord +druid.indexer.queue.startDelay=PT0M +druid.indexer.runner.javaOpts="-server -Xmx256m" +druid.indexer.fork.property.druid.processing.numThreads=1 +druid.indexer.fork.property.druid.computation.buffer.size=100000000 ``` If you are interested in reading more about these configurations, see [here](Indexing-Service.html). diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index 33cf72d3cf4..2e4802e0f50 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -1,22 +1,17 @@ --server --Xmx256m --Duser.timezone=UTC --Dfile.encoding=UTF-8 +druid.host=localhost +druid.port=8087 +druid.service=overlord --Ddruid.host=localhost --Ddruid.port=8080 --Ddruid.service=overlord +druid.zk.service.host=localhost --Ddruid.zk.service.host=localhost +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid +druid.db.connector.user=druid +druid.db.connector.password=diurd --Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid --Ddruid.db.connector.user=druid --Ddruid.db.connector.password=diurd - --Ddruid.selectors.indexing.serviceName=overlord --Ddruid.indexer.queue.startDelay=PT0M --Ddruid.indexer.runner.javaOpts="-server -Xmx256m" --Ddruid.indexer.fork.property.druid.processing.numThreads=1 --Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000 \ No newline at end of file +druid.selectors.indexing.serviceName=overlord +druid.indexer.queue.startDelay=PT0M +druid.indexer.runner.javaOpts="-server -Xmx256m" +druid.indexer.fork.property.druid.processing.numThreads=1 +druid.indexer.fork.property.druid.computation.buffer.size=100000000 \ No newline at end of file From 276e48e5648a8e6468247b8184e725bd91ab3469 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 13:33:16 -0700 Subject: [PATCH 100/270] another doc fix --- docs/content/Tutorial:-The-Druid-Cluster.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 91fc08fc536..0657595209b 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -120,7 +120,7 @@ druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid druid.db.connector.user=druid druid.db.connector.password=diurd -druid.coordinator.startDelay=PT60s +druid.coordinator.startDelay=PT70s ``` To start the coordinator node: From be92c322e33ed5f23d51a3eabc7d5f34e4aa85bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 19 May 2014 22:23:40 -0700 Subject: [PATCH 101/270] make ObjectStrategy cacheable --- .../segment/data/CacheableObjectStrategy.java | 26 ++++++ .../io/druid/segment/data/GenericIndexed.java | 80 ++++++++++++++++--- .../io/druid/segment/data/ObjectStrategy.java | 13 ++- 3 files changed, 107 insertions(+), 12 deletions(-) create mode 100644 processing/src/main/java/io/druid/segment/data/CacheableObjectStrategy.java diff --git a/processing/src/main/java/io/druid/segment/data/CacheableObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CacheableObjectStrategy.java new file mode 100644 index 00000000000..9558d97a57d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CacheableObjectStrategy.java @@ -0,0 +1,26 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.segment.data; + +/** + * Implementing CacheableObjectStrategy instead of ObjectSrategy indicates + * that a column scan may cache the results of fromByteBuffer + */ +public interface CacheableObjectStrategy extends ObjectStrategy {} diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 4e56d86da27..a22e48597fc 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import com.google.common.base.Charsets; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.metamx.common.IAE; @@ -32,6 +33,8 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; import java.util.Arrays; import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; /** * A generic, flat storage mechanism. Use static methods fromArray() or fromIterable() to construct. If input @@ -49,6 +52,7 @@ import java.util.Iterator; public class GenericIndexed implements Indexed { private static final byte version = 0x1; + private int indexOffset; public static GenericIndexed fromArray(T[] objects, ObjectStrategy strategy) { @@ -114,11 +118,50 @@ public class GenericIndexed implements Indexed return new GenericIndexed(theBuffer.asReadOnlyBuffer(), strategy, allowReverseLookup); } + private static class SizedLRUMap extends LinkedHashMap + { + final Map sizes = Maps.newHashMap(); + int numBytes = 0; + int maxBytes = 0; + + public SizedLRUMap(int initialCapacity, int maxBytes) + { + super(initialCapacity, 0.75f, true); + this.maxBytes = maxBytes; + } + + @Override + protected boolean removeEldestEntry(Map.Entry eldest) + { + if (numBytes > maxBytes) { + numBytes -= sizes.remove(eldest.getKey()); + return true; + } + return false; + } + + public V put(K key, V value, int size) + { + numBytes += size; + sizes.put(key, size); + return super.put(key, value); + } + } + private final ByteBuffer theBuffer; private final ObjectStrategy strategy; private final boolean allowReverseLookup; private final int size; + private final boolean cacheable; + private final ThreadLocal> cachedValues = new ThreadLocal>() { + @Override + protected SizedLRUMap initialValue() + { + return new SizedLRUMap<>(16384, 1024 * 1024); + } + }; + private final int valuesOffset; GenericIndexed( @@ -132,7 +175,10 @@ public class GenericIndexed implements Indexed this.allowReverseLookup = allowReverseLookup; size = theBuffer.getInt(); + indexOffset = theBuffer.position(); valuesOffset = theBuffer.position() + (size << 2); + + this.cacheable = strategy instanceof CacheableObjectStrategy; } @Override @@ -157,24 +203,38 @@ public class GenericIndexed implements Indexed throw new IAE(String.format("Index[%s] >= size[%s]", index, size)); } - ByteBuffer myBuffer = theBuffer.asReadOnlyBuffer(); - int startOffset = 4; - int endOffset; + if(cacheable) { + final T cached = cachedValues.get().get(index); + if (cached != null) { + return cached; + } + } + + final int startOffset; + final int endOffset; if (index == 0) { - endOffset = myBuffer.getInt(); + startOffset = 4; + endOffset = theBuffer.getInt(indexOffset); } else { - myBuffer.position(myBuffer.position() + ((index - 1) * 4)); - startOffset = myBuffer.getInt() + 4; - endOffset = myBuffer.getInt(); + final int position = indexOffset + ((index - 1) * 4); + startOffset = theBuffer.getInt(position) + 4; + endOffset = theBuffer.getInt(position + Ints.BYTES); } if (startOffset == endOffset) { return null; } - myBuffer.position(valuesOffset + startOffset); - return strategy.fromByteBuffer(myBuffer, endOffset - startOffset); + final ByteBuffer copyBuffer = this.theBuffer.asReadOnlyBuffer(); + copyBuffer.position(valuesOffset + startOffset); + final int size = endOffset - startOffset; + final T value = strategy.fromByteBuffer(copyBuffer, size); + + if(cacheable) { + cachedValues.get().put(index, value, size); + } + return value; } @Override @@ -241,7 +301,7 @@ public class GenericIndexed implements Indexed throw new IAE("Unknown version[%s]", versionFromBuffer); } - public static ObjectStrategy stringStrategy = new ObjectStrategy() + public static ObjectStrategy stringStrategy = new CacheableObjectStrategy() { @Override public Class getClazz() diff --git a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java index 7e2753523ba..0f039ab0598 100644 --- a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java @@ -22,11 +22,20 @@ package io.druid.segment.data; import java.nio.ByteBuffer; import java.util.Comparator; -/** -*/ public interface ObjectStrategy extends Comparator { public Class getClazz(); + + /** + * Convert values from their underlying byte representation. + * + * Implementations of this method must not change the given buffer mark, or limit, but may modify its position. + * Use buffer.asReadOnlyBuffer() or buffer.duplicate() if mark or limit need to be set. + * + * @param buffer buffer to read value from + * @param numBytes number of bytes used to store the value, starting at buffer.position() + * @return + */ public T fromByteBuffer(ByteBuffer buffer, int numBytes); public byte[] toBytes(T val); } From 7a351dfde3c1f7000d8f0c34dbcf8d0a241e58fd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 19 May 2014 22:55:09 -0700 Subject: [PATCH 102/270] optimize lookups by caching bytebuffer copy --- .../hyperloglog/HyperUniquesSerde.java | 5 ++-- .../data/ConciseCompressedIndexedInts.java | 5 ++-- .../io/druid/segment/data/GenericIndexed.java | 25 +++++++++++++------ .../io/druid/segment/data/IndexedRTree.java | 6 +++-- .../segment/data/IntBufferIndexedInts.java | 5 ++-- 5 files changed, 31 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java index 21803229284..b83d52f1cb6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java @@ -128,8 +128,9 @@ public class HyperUniquesSerde extends ComplexMetricSerde @Override public HyperLogLogCollector fromByteBuffer(ByteBuffer buffer, int numBytes) { - buffer.limit(buffer.position() + numBytes); - return HyperLogLogCollector.makeCollector(buffer); + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); + return HyperLogLogCollector.makeCollector(readOnlyBuffer); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java b/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java index e81e9389a32..95b33fc0d9b 100644 --- a/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/ConciseCompressedIndexedInts.java @@ -123,8 +123,9 @@ public class ConciseCompressedIndexedInts implements IndexedInts, Comparable implements Indexed private final int size; private final boolean cacheable; + private final ThreadLocal cachedBuffer; private final ThreadLocal> cachedValues = new ThreadLocal>() { @Override protected SizedLRUMap initialValue() @@ -178,6 +179,15 @@ public class GenericIndexed implements Indexed indexOffset = theBuffer.position(); valuesOffset = theBuffer.position() + (size << 2); + this.cachedBuffer = new ThreadLocal() + { + @Override + protected ByteBuffer initialValue() + { + return theBuffer.asReadOnlyBuffer(); + } + }; + this.cacheable = strategy instanceof CacheableObjectStrategy; } @@ -210,23 +220,24 @@ public class GenericIndexed implements Indexed } } + final ByteBuffer copyBuffer = this.cachedBuffer.get(); + final int startOffset; final int endOffset; if (index == 0) { startOffset = 4; - endOffset = theBuffer.getInt(indexOffset); + endOffset = copyBuffer.getInt(indexOffset); } else { - final int position = indexOffset + ((index - 1) * 4); - startOffset = theBuffer.getInt(position) + 4; - endOffset = theBuffer.getInt(position + Ints.BYTES); + copyBuffer.position(indexOffset + ((index - 1) * 4)); + startOffset = copyBuffer.getInt() + 4; + endOffset = copyBuffer.getInt(); } if (startOffset == endOffset) { return null; } - final ByteBuffer copyBuffer = this.theBuffer.asReadOnlyBuffer(); copyBuffer.position(valuesOffset + startOffset); final int size = endOffset - startOffset; final T value = strategy.fromByteBuffer(copyBuffer, size); @@ -310,9 +321,9 @@ public class GenericIndexed implements Indexed } @Override - public String fromByteBuffer(ByteBuffer buffer, int numBytes) + public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { - byte[] bytes = new byte[numBytes]; + final byte[] bytes = new byte[numBytes]; buffer.get(bytes); return new String(bytes, Charsets.UTF_8); } diff --git a/processing/src/main/java/io/druid/segment/data/IndexedRTree.java b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java index b5cb07388fd..d0785a21a12 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedRTree.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java @@ -81,8 +81,10 @@ public class IndexedRTree implements Comparable @Override public ImmutableRTree fromByteBuffer(ByteBuffer buffer, int numBytes) { - buffer.limit(buffer.position() + numBytes); - return new ImmutableRTree(buffer.asReadOnlyBuffer()); + + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); + return new ImmutableRTree(readOnlyBuffer); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java b/processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java index bb121602e85..9229d763636 100644 --- a/processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/IntBufferIndexedInts.java @@ -101,8 +101,9 @@ public class IntBufferIndexedInts implements IndexedInts, Comparable Date: Mon, 19 May 2014 23:08:16 -0700 Subject: [PATCH 103/270] optimize single value lookup --- .../segment/QueryableIndexStorageAdapter.java | 74 +++++++++++-------- .../druid/segment/data/SingleIndexedInts.java | 53 ------------- 2 files changed, 44 insertions(+), 83 deletions(-) delete mode 100644 processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index b922d80656f..2562073813b 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -21,6 +21,7 @@ package io.druid.segment; import com.google.common.base.Function; import com.google.common.base.Predicates; +import com.google.common.collect.Iterators; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.common.guava.CloseQuietly; @@ -38,12 +39,12 @@ import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.Offset; -import io.druid.segment.data.SingleIndexedInts; import org.joda.time.DateTime; import org.joda.time.Interval; import java.io.Closeable; import java.io.IOException; +import java.util.Iterator; import java.util.Map; /** @@ -313,7 +314,27 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public IndexedInts getRow() { - return new SingleIndexedInts(column.getSingleValueRow(cursorOffset.getOffset())); + // using an anonymous class is faster than creating a class that stores a copy of the value + return new IndexedInts() + { + @Override + public int size() + { + return 1; + } + + @Override + public int get(int index) + { + return column.getSingleValueRow(cursorOffset.getOffset()); + } + + @Override + public Iterator iterator() + { + return Iterators.singletonIterator(column.getSingleValueRow(cursorOffset.getOffset())); + } + }; } @Override @@ -740,7 +761,27 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public IndexedInts getRow() { - return new SingleIndexedInts(dict.getSingleValueRow(currRow)); + // using an anonymous class is faster than creating a class that stores a copy of the value + return new IndexedInts() + { + @Override + public int size() + { + return 1; + } + + @Override + public int get(int index) + { + return dict.getSingleValueRow(currRow); + } + + @Override + public Iterator iterator() + { + return Iterators.singletonIterator(dict.getSingleValueRow(currRow)); + } + }; } @Override @@ -979,31 +1020,4 @@ public class QueryableIndexStorageAdapter implements StorageAdapter ); } } - - private static class NullDimensionSelector implements DimensionSelector - { - @Override - public IndexedInts getRow() - { - return new SingleIndexedInts(0); - } - - @Override - public int getValueCardinality() - { - return 1; - } - - @Override - public String lookupName(int id) - { - return ""; - } - - @Override - public int lookupId(String name) - { - return 0; - } - } } diff --git a/processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java b/processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java deleted file mode 100644 index 796f7ef655d..00000000000 --- a/processing/src/main/java/io/druid/segment/data/SingleIndexedInts.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.segment.data; - -import com.google.common.collect.Iterators; - -import java.util.Iterator; - -/** -*/ -public class SingleIndexedInts implements IndexedInts -{ - private final int value; - - public SingleIndexedInts(int value) { - this.value = value; - } - - @Override - public int size() - { - return 1; - } - - @Override - public int get(int index) - { - return value; - } - - @Override - public Iterator iterator() - { - return Iterators.singletonIterator(value); - } -} From be25d67894c754ebf96b64f46a0eff9da991e036 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 19 May 2014 23:25:20 -0700 Subject: [PATCH 104/270] add comments --- .../src/main/java/io/druid/segment/data/GenericIndexed.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 2e3d4491a56..3e9b449773b 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -159,7 +159,7 @@ public class GenericIndexed implements Indexed @Override protected SizedLRUMap initialValue() { - return new SizedLRUMap<>(16384, 1024 * 1024); + return new SizedLRUMap<>(16384, 1024 * 1024); // 1MB cache per column, per thread } }; @@ -220,6 +220,7 @@ public class GenericIndexed implements Indexed } } + // using a cached copy of the buffer instead of making a read-only copy every time get() is called is faster final ByteBuffer copyBuffer = this.cachedBuffer.get(); final int startOffset; @@ -240,6 +241,7 @@ public class GenericIndexed implements Indexed copyBuffer.position(valuesOffset + startOffset); final int size = endOffset - startOffset; + // fromByteBuffer must not modify the buffer limit final T value = strategy.fromByteBuffer(copyBuffer, size); if(cacheable) { From 8840af59774a306b06c15df95573884a04dc366b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 22 May 2014 13:37:08 -0700 Subject: [PATCH 105/270] make GenericIndexed release cache resources on close --- .../segment/QueryableIndexStorageAdapter.java | 52 ++++++++++------ .../io/druid/segment/column/ColumnConfig.java | 5 ++ .../io/druid/segment/data/GenericIndexed.java | 62 ++++++++++++++++--- 3 files changed, 92 insertions(+), 27 deletions(-) create mode 100644 processing/src/main/java/io/druid/segment/column/ColumnConfig.java diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 2562073813b..137024c61f4 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -187,6 +187,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter { final Offset baseOffset = offset.clone(); + final Map dictionaryColumnCache = Maps.newHashMap(); final Map genericColumnCache = Maps.newHashMap(); final Map complexColumnCache = Maps.newHashMap(); final Map objectColumnCache = Maps.newHashMap(); @@ -271,12 +272,16 @@ public class QueryableIndexStorageAdapter implements StorageAdapter public DimensionSelector makeDimensionSelector(String dimension) { final String dimensionName = dimension.toLowerCase(); + + DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName); final Column columnDesc = index.getColumn(dimensionName); - if (columnDesc == null) { - return null; + + if (cachedColumn == null && columnDesc != null) { + cachedColumn = columnDesc.getDictionaryEncoding(); + dictionaryColumnCache.put(dimensionName, cachedColumn); } - final DictionaryEncodedColumn column = columnDesc.getDictionaryEncoding(); + final DictionaryEncodedColumn column = cachedColumn; if (column == null) { return null; @@ -557,6 +562,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter public void close() throws IOException { CloseQuietly.close(timestamps); + for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) { + CloseQuietly.close(column); + } for (GenericColumn column : genericColumnCache.values()) { CloseQuietly.close(column); } @@ -641,6 +649,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter */ public Sequence build() { + final Map dictionaryColumnCache = Maps.newHashMap(); final Map genericColumnCache = Maps.newHashMap(); final Map complexColumnCache = Maps.newHashMap(); final Map objectColumnCache = Maps.newHashMap(); @@ -718,41 +727,45 @@ public class QueryableIndexStorageAdapter implements StorageAdapter public DimensionSelector makeDimensionSelector(String dimension) { final String dimensionName = dimension.toLowerCase(); - final Column column = index.getColumn(dimensionName); - if (column == null) { - return null; + + DictionaryEncodedColumn cachedColumn = dictionaryColumnCache.get(dimensionName); + final Column columnDesc = index.getColumn(dimensionName); + + if (cachedColumn == null && columnDesc != null) { + cachedColumn = columnDesc.getDictionaryEncoding(); + dictionaryColumnCache.put(dimensionName, cachedColumn); } - final DictionaryEncodedColumn dict = column.getDictionaryEncoding(); + final DictionaryEncodedColumn column = cachedColumn; - if (dict == null) { + if (column == null) { return null; - } else if (column.getCapabilities().hasMultipleValues()) { + } else if (columnDesc.getCapabilities().hasMultipleValues()) { return new DimensionSelector() { @Override public IndexedInts getRow() { - return dict.getMultiValueRow(currRow); + return column.getMultiValueRow(currRow); } @Override public int getValueCardinality() { - return dict.getCardinality(); + return column.getCardinality(); } @Override public String lookupName(int id) { - final String retVal = dict.lookupName(id); + final String retVal = column.lookupName(id); return retVal == null ? "" : retVal; } @Override public int lookupId(String name) { - return dict.lookupId(name); + return column.lookupId(name); } }; } else { @@ -773,13 +786,13 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public int get(int index) { - return dict.getSingleValueRow(currRow); + return column.getSingleValueRow(currRow); } @Override public Iterator iterator() { - return Iterators.singletonIterator(dict.getSingleValueRow(currRow)); + return Iterators.singletonIterator(column.getSingleValueRow(currRow)); } }; } @@ -787,19 +800,19 @@ public class QueryableIndexStorageAdapter implements StorageAdapter @Override public int getValueCardinality() { - return dict.getCardinality(); + return column.getCardinality(); } @Override public String lookupName(int id) { - return dict.lookupName(id); + return column.lookupName(id); } @Override public int lookupId(String name) { - return dict.lookupId(name); + return column.lookupId(name); } }; } @@ -1004,6 +1017,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter public void close() throws IOException { CloseQuietly.close(timestamps); + for (DictionaryEncodedColumn column : dictionaryColumnCache.values()) { + CloseQuietly.close(column); + } for (GenericColumn column : genericColumnCache.values()) { CloseQuietly.close(column); } diff --git a/processing/src/main/java/io/druid/segment/column/ColumnConfig.java b/processing/src/main/java/io/druid/segment/column/ColumnConfig.java new file mode 100644 index 00000000000..7dc9096a64f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/ColumnConfig.java @@ -0,0 +1,5 @@ +package io.druid.segment.column; + +public interface ColumnConfig +{ +} diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 3e9b449773b..46cacfd8bac 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -49,9 +49,12 @@ import java.util.Map; * bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values * bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes for value */ -public class GenericIndexed implements Indexed +public class GenericIndexed implements Indexed, Closeable { private static final byte version = 0x1; + + public static final int INITIAL_CACHE_CAPACITY = 16384; + private int indexOffset; public static GenericIndexed fromArray(T[] objects, ObjectStrategy strategy) @@ -155,14 +158,7 @@ public class GenericIndexed implements Indexed private final boolean cacheable; private final ThreadLocal cachedBuffer; - private final ThreadLocal> cachedValues = new ThreadLocal>() { - @Override - protected SizedLRUMap initialValue() - { - return new SizedLRUMap<>(16384, 1024 * 1024); // 1MB cache per column, per thread - } - }; - + private final ThreadLocal> cachedValues; private final int valuesOffset; GenericIndexed( @@ -188,6 +184,35 @@ public class GenericIndexed implements Indexed } }; + this.cacheable = false; + this.cachedValues = new ThreadLocal<>(); + } + + /** + * Creates a copy of the given indexed with the given cache + * The resulting copy must be closed to release resources used by the cache + * + * @param other + * @param cache + */ + GenericIndexed(GenericIndexed other, final SizedLRUMap cache) + { + this.theBuffer = other.theBuffer; + this.strategy = other.strategy; + this.allowReverseLookup = other.allowReverseLookup; + this.size = other.size; + this.indexOffset = other.indexOffset; + this.valuesOffset = other.valuesOffset; + this.cachedBuffer = other.cachedBuffer; + + this.cachedValues = new ThreadLocal>() { + @Override + protected SizedLRUMap initialValue() + { + return cache; + } + }; + this.cacheable = strategy instanceof CacheableObjectStrategy; } @@ -293,6 +318,25 @@ public class GenericIndexed implements Indexed channel.write(theBuffer.asReadOnlyBuffer()); } + /** + * The returned CachedIndexed must be closed to release the underlying memory + * @param maxBytes + * @return + */ + public GenericIndexed withCache(int maxBytes) + { + return new GenericIndexed<>(this, new SizedLRUMap(INITIAL_CACHE_CAPACITY, maxBytes)); + } + + @Override + public void close() throws IOException + { + if(cacheable) { + cachedValues.get().clear(); + cachedValues.remove(); + } + } + public static GenericIndexed read(ByteBuffer buffer, ObjectStrategy strategy) { byte versionFromBuffer = buffer.get(); From 09346b0a3c18cae48dff35a179988d9e17fb22ec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 22 May 2014 13:46:25 -0700 Subject: [PATCH 106/270] make column cache configurable --- .../indexer/HadoopDruidIndexerConfig.java | 13 +++- .../io/druid/indexer/IndexGeneratorJob.java | 5 +- .../indexer/HadoopDruidIndexerConfigTest.java | 12 ++++ .../io/druid/indexing/common/TaskToolbox.java | 11 +++- .../indexing/common/TaskToolboxFactory.java | 9 ++- .../common/index/YeOldePlumberSchool.java | 10 ++- .../indexing/common/task/AppendTask.java | 8 ++- .../druid/indexing/common/task/IndexTask.java | 3 +- .../druid/indexing/common/task/MergeTask.java | 9 ++- .../common/task/RealtimeIndexTask.java | 8 ++- .../common/task/VersionConverterTask.java | 3 +- .../druid/indexing/common/TestMergeTask.java | 10 ++- .../indexing/common/TestRealtimeTask.java | 1 + .../indexing/common/task/TaskSerdeTest.java | 61 ++++++++++++++----- .../indexing/overlord/TaskLifecycleTest.java | 11 +++- .../SimpleResourceManagementStrategyTest.java | 3 +- .../indexing/worker/TaskAnnouncementTest.java | 1 + .../worker/WorkerTaskMonitorTest.java | 43 ++++++++++++- .../main/java/io/druid/segment/IndexIO.java | 25 ++++---- .../java/io/druid/segment/IndexMerger.java | 1 + .../io/druid/segment/column/ColumnConfig.java | 20 ++++++ .../segment/column/ColumnDescriptor.java | 4 +- .../column/DictionaryEncodedColumn.java | 4 +- .../column/SimpleDictionaryEncodedColumn.java | 8 +++ .../druid/segment/serde/ColumnPartSerde.java | 3 +- .../segment/serde/ComplexColumnPartSerde.java | 3 +- .../DictionaryEncodedColumnPartSerde.java | 11 ++-- .../DictionaryEncodedColumnSupplier.java | 11 +++- .../serde/FloatGenericColumnPartSerde.java | 3 +- .../serde/LongGenericColumnPartSerde.java | 3 +- .../java/io/druid/segment/EmptyIndexTest.java | 12 +++- .../io/druid/segment/IndexMergerTest.java | 26 +++++--- .../test/java/io/druid/segment/TestIndex.java | 17 +++++- .../filter/SpatialFilterBonusTest.java | 17 +++++- .../segment/filter/SpatialFilterTest.java | 18 +++++- .../io/druid/guice/StorageNodeModule.java | 3 + .../loading/MMappedQueryableIndexFactory.java | 14 ++++- .../realtime/plumber/FlushingPlumber.java | 7 ++- .../plumber/FlushingPlumberSchool.java | 8 ++- .../realtime/plumber/RealtimePlumber.java | 13 ++-- .../plumber/RealtimePlumberSchool.java | 7 ++- .../druid/server/DruidProcessingConfig.java | 11 +++- .../segment/realtime/FireDepartmentTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 11 ++++ 44 files changed, 393 insertions(+), 90 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 0a863b78c13..59b9d09d5d9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -19,6 +19,7 @@ package io.druid.indexer; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; @@ -45,6 +46,7 @@ import io.druid.guice.annotations.Self; import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.path.PathSpec; import io.druid.initialization.Initialization; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.server.DruidNode; import io.druid.timeline.DataSegment; @@ -105,7 +107,7 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromSchema(HadoopIngestionSpec schema) { - return new HadoopDruidIndexerConfig(schema); + return new HadoopDruidIndexerConfig(injector.getInstance(ColumnConfig.class), schema); } public static HadoopDruidIndexerConfig fromMap(Map argSpec) @@ -115,6 +117,7 @@ public class HadoopDruidIndexerConfig return HadoopDruidIndexerConfig.jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class); } else { return new HadoopDruidIndexerConfig( + injector.getInstance(ColumnConfig.class), HadoopDruidIndexerConfig.jsonMapper.convertValue( argSpec, HadoopIngestionSpec.class @@ -166,12 +169,15 @@ public class HadoopDruidIndexerConfig private volatile HadoopIngestionSpec schema; private volatile PathSpec pathSpec; + private volatile ColumnConfig columnConfig; @JsonCreator public HadoopDruidIndexerConfig( + @JacksonInject final ColumnConfig columnConfig, final @JsonProperty("schema") HadoopIngestionSpec schema ) { + this.columnConfig = columnConfig; this.schema = schema; this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class); } @@ -182,6 +188,11 @@ public class HadoopDruidIndexerConfig return schema; } + public ColumnConfig getColumnConfig() + { + return columnConfig; + } + public String getDataSource() { return schema.getDataSchema().getDataSource(); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 6dd97b920ba..29f9be0dad8 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -40,6 +40,7 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.timeline.DataSegment; @@ -281,6 +282,8 @@ public class IndexGeneratorJob implements Jobby //final DataRollupSpec rollupSpec = config.getRollupSpec(); final AggregatorFactory[] aggs = config.getSchema().getDataSchema().getAggregators(); + final ColumnConfig columnConfig = config.getColumnConfig(); + IncrementalIndex index = makeIncrementalIndex(bucket, aggs); File baseFlushFile = File.createTempFile("base", "flush"); @@ -371,7 +374,7 @@ public class IndexGeneratorJob implements Jobby } for (File file : toMerge) { - indexes.add(IndexIO.loadIndex(file)); + indexes.add(IndexIO.loadIndex(file, columnConfig)); } mergedBase = IndexMerger.mergeQueryableIndex( indexes, aggs, new File(baseFlushFile, "merged"), new IndexMerger.ProgressIndicator() diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index be8188fe7a0..52c63779e19 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -22,6 +22,7 @@ package io.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.column.ColumnConfig; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -45,6 +46,15 @@ public class HadoopDruidIndexerConfigTest } } + private static final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + @Test public void shouldMakeHDFSCompliantSegmentOutputPath() @@ -70,6 +80,7 @@ public class HadoopDruidIndexerConfigTest } HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( + columnConfig, schema.withTuningConfig( schema.getTuningConfig() .withVersion( @@ -110,6 +121,7 @@ public class HadoopDruidIndexerConfigTest } HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( + columnConfig, schema.withTuningConfig( schema.getTuningConfig() .withVersion( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 5eb50b622c6..db2012e8f03 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -34,6 +34,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -73,6 +74,7 @@ public class TaskToolbox private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; private final File taskWorkDir; + private final ColumnConfig columnConfig; public TaskToolbox( TaskConfig config, @@ -90,7 +92,8 @@ public class TaskToolbox MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, ObjectMapper objectMapper, - final File taskWorkDir + final File taskWorkDir, + ColumnConfig columnConfig ) { this.config = config; @@ -109,6 +112,7 @@ public class TaskToolbox this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; this.taskWorkDir = taskWorkDir; + this.columnConfig = columnConfig; } public TaskConfig getConfig() @@ -176,6 +180,11 @@ public class TaskToolbox return objectMapper; } + public ColumnConfig getColumnConfig() + { + return columnConfig; + } + public Map fetchSegments(List segments) throws SegmentLoadingException { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 89d275b9a7f..61c83a22cf4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -29,6 +29,7 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -57,6 +58,7 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; private final ObjectMapper objectMapper; + private final ColumnConfig columnConfig; @Inject public TaskToolboxFactory( @@ -73,7 +75,8 @@ public class TaskToolboxFactory @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper + ObjectMapper objectMapper, + ColumnConfig columnConfig ) { this.config = config; @@ -90,6 +93,7 @@ public class TaskToolboxFactory this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; this.objectMapper = objectMapper; + this.columnConfig = columnConfig; } public TaskToolbox build(Task task) @@ -112,7 +116,8 @@ public class TaskToolboxFactory monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), objectMapper, - taskWorkDir + taskWorkDir, + columnConfig ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 11f6bb2264d..ea3be607681 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -38,6 +38,7 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -65,6 +66,7 @@ public class YeOldePlumberSchool implements PlumberSchool private final String version; private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; + private final ColumnConfig columnConfig; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -73,13 +75,15 @@ public class YeOldePlumberSchool implements PlumberSchool @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, - @JacksonInject("tmpSegmentDir") File tmpSegmentDir + @JacksonInject("tmpSegmentDir") File tmpSegmentDir, + @JacksonInject ColumnConfig columnConfig ) { this.interval = interval; this.version = version; this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; + this.columnConfig = columnConfig; } @Override @@ -162,7 +166,7 @@ public class YeOldePlumberSchool implements PlumberSchool } else { List indexes = Lists.newArrayList(); for (final File oneSpill : spilled) { - indexes.add(IndexIO.loadIndex(oneSpill)); + indexes.add(IndexIO.loadIndex(oneSpill, columnConfig)); } fileToUpload = new File(tmpSegmentDir, "merged"); @@ -170,7 +174,7 @@ public class YeOldePlumberSchool implements PlumberSchool } // Map merged segment so we can extract dimensions - final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload); + final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload, columnConfig); final DataSegment segmentToUpload = theSink.getSegment() .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index 40807073874..aeec42cf77e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; @@ -32,6 +33,7 @@ import io.druid.segment.IndexableAdapter; import io.druid.segment.QueryableIndexIndexableAdapter; import io.druid.segment.Rowboat; import io.druid.segment.RowboatFilteringIndexAdapter; +import io.druid.segment.column.ColumnConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; @@ -46,14 +48,18 @@ import java.util.Map; */ public class AppendTask extends MergeTaskBase { + private final ColumnConfig columnConfig; + @JsonCreator public AppendTask( + @JacksonInject ColumnConfig columnConfig, @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments ) { super(id, dataSource, segments); + this.columnConfig = columnConfig; } @Override @@ -92,7 +98,7 @@ public class AppendTask extends MergeTaskBase adapters.add( new RowboatFilteringIndexAdapter( new QueryableIndexIndexableAdapter( - IndexIO.loadIndex(holder.getFile()) + IndexIO.loadIndex(holder.getFile(), columnConfig) ), new Predicate() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 182a1569903..0fb759db931 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -382,7 +382,8 @@ public class IndexTask extends AbstractFixedIntervalTask interval, version, wrappedDataSegmentPusher, - tmpDir + tmpDir, + toolbox.getColumnConfig() ).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics); // rowFlushBoundary for this job diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java index 9bca4a3eee5..8b9ef088f6b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -30,6 +31,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; +import io.druid.segment.column.ColumnConfig; import io.druid.timeline.DataSegment; import javax.annotation.Nullable; @@ -43,17 +45,20 @@ public class MergeTask extends MergeTaskBase { @JsonIgnore private final List aggregators; + private final ColumnConfig columnConfig; @JsonCreator public MergeTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators + @JsonProperty("aggregations") List aggregators, + @JacksonInject ColumnConfig columnConfig ) { super(id, dataSource, segments); this.aggregators = aggregators; + this.columnConfig = columnConfig; } @Override @@ -69,7 +74,7 @@ public class MergeTask extends MergeTaskBase public QueryableIndex apply(@Nullable File input) { try { - return IndexIO.loadIndex(input); + return IndexIO.loadIndex(input, columnConfig); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 84ef51a9c4f..6232fbe5691 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -19,6 +19,7 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -43,6 +44,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryToolChest; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -94,6 +96,7 @@ public class RealtimeIndexTask extends AbstractTask @JsonIgnore private final FireDepartment spec; + private final ColumnConfig columnConfig; @JsonIgnore private volatile Plumber plumber = null; @@ -113,7 +116,8 @@ public class RealtimeIndexTask extends AbstractTask @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("maxPendingPersists") int maxPendingPersists, @JsonProperty("segmentGranularity") Granularity segmentGranularity, - @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory + @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, + @JacksonInject ColumnConfig columnConfig ) { super( @@ -147,6 +151,7 @@ public class RealtimeIndexTask extends AbstractTask null, null, null, null ); } + this.columnConfig = columnConfig; } @Override @@ -308,6 +313,7 @@ public class RealtimeIndexTask extends AbstractTask segmentPublisher, toolbox.getNewSegmentServerView(), toolbox.getQueryExecutorService(), + columnConfig, null, null, null, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java index 75561f2408e..cbb388a0fc3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; @@ -253,7 +252,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask final File location = localSegments.get(segment); final File outLocation = new File(location, "v9_out"); - if (IndexIO.convertSegment(location, outLocation)) { + if (IndexIO.convertSegment(location, outLocation, toolbox.getColumnConfig())) { final int outVersion = IndexIO.getVersionFromDir(outLocation); // Appending to the version makes a new version that inherits most comparability parameters of the original diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java index 05f3118d3c9..4775bb407f4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java @@ -19,12 +19,14 @@ package io.druid.indexing.common; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Lists; import io.druid.indexing.common.task.MergeTask; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -54,7 +56,8 @@ public class TestMergeTask extends MergeTask 0 ) ), - Lists.newArrayList() + Lists.newArrayList(), + null ); } @@ -65,10 +68,11 @@ public class TestMergeTask extends MergeTask @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators + @JsonProperty("aggregations") List aggregators, + @JacksonInject ColumnConfig columnConfig ) { - super(id, dataSource, segments, aggregators); + super(id, dataSource, segments, aggregators, columnConfig); this.id = id; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java index 6220f447bde..5280a884a55 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java @@ -54,6 +54,7 @@ public class TestRealtimeTask extends RealtimeIndexTask null, 1, null, + null, null ); this.status = status; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 00407381afc..5fa3548d797 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -21,8 +21,15 @@ package io.druid.indexing.common.task; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; +import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector; +import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; import com.metamx.common.Granularity; import io.druid.data.input.impl.JSONDataSpec; import io.druid.data.input.impl.TimestampSpec; @@ -34,6 +41,7 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.Schema; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; @@ -48,6 +56,41 @@ import java.io.File; public class TaskSerdeTest { + private static final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final Injector injector = Guice.createInjector( + new com.google.inject.Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(ColumnConfig.class).toInstance(columnConfig); + } + } + ); + + static { + final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); + + jsonMapper.setInjectableValues(new GuiceInjectableValues(injector)); + jsonMapper.setAnnotationIntrospectors( + new AnnotationIntrospectorPair( + guiceIntrospector, jsonMapper.getSerializationConfig().getAnnotationIntrospector() + ), + new AnnotationIntrospectorPair( + guiceIntrospector, jsonMapper.getDeserializationConfig().getAnnotationIntrospector() + ) + ); + } + @Test public void testIndexTaskSerde() throws Exception { @@ -68,7 +111,6 @@ public class TaskSerdeTest -1 ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { jsonMapper.registerModule(jacksonModule); } @@ -99,10 +141,10 @@ public class TaskSerdeTest ), ImmutableList.of( new CountAggregatorFactory("cnt") - ) + ), + null ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -131,7 +173,6 @@ public class TaskSerdeTest new Interval("2010-01-01/P1D") ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -153,7 +194,6 @@ public class TaskSerdeTest DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build() ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -177,7 +217,6 @@ public class TaskSerdeTest DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build() ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -205,10 +244,10 @@ public class TaskSerdeTest new Period("PT10M"), 1, Granularity.HOUR, + null, null ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -251,7 +290,6 @@ public class TaskSerdeTest new Interval("2010-01-01/P1D") ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -269,7 +307,6 @@ public class TaskSerdeTest @Test public void testDeleteTaskFromJson() throws Exception { - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final DeleteTask task = (DeleteTask) jsonMapper.readValue( "{\"type\":\"delete\",\"dataSource\":\"foo\",\"interval\":\"2010-01-01/P1D\"}", Task.class @@ -293,6 +330,7 @@ public class TaskSerdeTest public void testAppendTaskSerde() throws Exception { final AppendTask task = new AppendTask( + columnConfig, null, "foo", ImmutableList.of( @@ -300,7 +338,6 @@ public class TaskSerdeTest ) ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -325,7 +362,6 @@ public class TaskSerdeTest new Interval("2010-01-01/P1D") ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -350,7 +386,6 @@ public class TaskSerdeTest new Interval("2010-01-01/P1D") ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -375,7 +410,6 @@ public class TaskSerdeTest ImmutableMap.of("bucket", "hey", "baseKey", "what") ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change @@ -434,7 +468,6 @@ public class TaskSerdeTest null ); - final ObjectMapper jsonMapper = new DefaultObjectMapper(); final String json = jsonMapper.writeValueAsString(task); final HadoopIndexTask task2 = (HadoopIndexTask) jsonMapper.readValue(json, Task.class); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 370952edfda..9fa2b76617f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -43,6 +43,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; @@ -205,7 +206,15 @@ public class TaskLifecycleTest } ) ), - new DefaultObjectMapper() + new DefaultObjectMapper(), + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + } ); tr = new ThreadPoolTaskRunner(tb); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 21f3277a653..78ec78c3f06 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -87,7 +87,8 @@ public class SimpleResourceManagementStrategyTest 0 ) ), - Lists.newArrayList() + Lists.newArrayList(), + null ); simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( autoScalingStrategy, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index 7aafad53e23..ec593aeee33 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -49,6 +49,7 @@ public class TaskAnnouncementTest new Period("PT10M"), 1, Granularity.HOUR, + null, null ); final TaskStatus status = TaskStatus.running(task.getId()); diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index bad6fae4939..21070ba0340 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -20,11 +20,17 @@ package io.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; +import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector; +import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.Files; +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; import io.druid.curator.PotentiallyGzippedCompressionProvider; import io.druid.indexing.common.IndexingServiceCondition; import io.druid.indexing.common.SegmentLoaderFactory; @@ -38,6 +44,7 @@ import io.druid.indexing.overlord.TestRemoteTaskRunnerConfig; import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.loading.DataSegmentPuller; import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.loading.OmniSegmentLoader; @@ -61,6 +68,40 @@ import java.util.List; public class WorkerTaskMonitorTest { private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final Injector injector = Guice.createInjector( + new com.google.inject.Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(ColumnConfig.class).toInstance( + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + } + ); + } + } + ); + + static { + final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); + + jsonMapper.setInjectableValues(new GuiceInjectableValues(injector)); + jsonMapper.setAnnotationIntrospectors( + new AnnotationIntrospectorPair( + guiceIntrospector, jsonMapper.getSerializationConfig().getAnnotationIntrospector() + ), + new AnnotationIntrospectorPair( + guiceIntrospector, jsonMapper.getDeserializationConfig().getAnnotationIntrospector() + ) + ); + } + private static final Joiner joiner = Joiner.on("/"); private static final String basePath = "/test/druid"; private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath); @@ -138,7 +179,7 @@ public class WorkerTaskMonitorTest } } ) - ), jsonMapper + ), jsonMapper, null ) ), new WorkerConfig().setCapacity(1) diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index a78adad37ac..9852a0cef24 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -44,6 +44,7 @@ import io.druid.common.utils.SerializerUtils; import io.druid.jackson.DefaultObjectMapper; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayIndexed; @@ -122,7 +123,7 @@ public class IndexIO return handler.mapDir(inDir); } - public static QueryableIndex loadIndex(File inDir) throws IOException + public static QueryableIndex loadIndex(File inDir, ColumnConfig columnConfig) throws IOException { init(); final int version = SegmentUtils.getVersionFromDir(inDir); @@ -130,7 +131,7 @@ public class IndexIO final IndexLoader loader = indexLoaders.get(version); if (loader != null) { - return loader.load(inDir); + return loader.load(inDir, columnConfig); } else { throw new ISE("Unknown index version[%s]", version); } @@ -180,7 +181,7 @@ public class IndexIO } } - public static boolean convertSegment(File toConvert, File converted) throws IOException + public static boolean convertSegment(File toConvert, File converted, ColumnConfig columnConfig) throws IOException { final int version = SegmentUtils.getVersionFromDir(toConvert); @@ -198,7 +199,7 @@ public class IndexIO case 7: log.info("Old version, re-persisting."); IndexMerger.append( - Arrays.asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), + Arrays.asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert, columnConfig))), converted ); return true; @@ -608,13 +609,13 @@ public class IndexIO static interface IndexLoader { - public QueryableIndex load(File inDir) throws IOException; + public QueryableIndex load(File inDir, ColumnConfig columnConfig) throws IOException; } static class LegacyIndexLoader implements IndexLoader { @Override - public QueryableIndex load(File inDir) throws IOException + public QueryableIndex load(File inDir, ColumnConfig columnConfig) throws IOException { MMappedIndex index = IndexIO.mapDir(inDir); @@ -626,7 +627,7 @@ public class IndexIO .setHasMultipleValues(true) .setDictionaryEncodedColumn( new DictionaryEncodedColumnSupplier( - index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension)) + index.getDimValueLookup(dimension), null, index.getDimColumn(dimension), columnConfig.columnCacheSizeBytes() ) ) .setBitmapIndex( @@ -699,7 +700,7 @@ public class IndexIO static class V9IndexLoader implements IndexLoader { @Override - public QueryableIndex load(File inDir) throws IOException + public QueryableIndex load(File inDir, ColumnConfig columnConfig) throws IOException { log.debug("Mapping v9 index[%s]", inDir); long startTime = System.currentTimeMillis(); @@ -721,11 +722,11 @@ public class IndexIO ObjectMapper mapper = new DefaultObjectMapper(); for (String columnName : cols) { - columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); + columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName), columnConfig)); } final QueryableIndex index = new SimpleQueryableIndex( - dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns, smooshedFiles + dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time"), columnConfig), columns, smooshedFiles ); log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); @@ -733,12 +734,12 @@ public class IndexIO return index; } - private Column deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer) throws IOException + private Column deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer, ColumnConfig columnConfig) throws IOException { ColumnDescriptor serde = mapper.readValue( serializerUtils.readString(byteBuffer), ColumnDescriptor.class ); - return serde.read(byteBuffer); + return serde.read(byteBuffer, columnConfig); } } diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index cee858283ad..56b4d925538 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -51,6 +51,7 @@ import io.druid.common.utils.JodaUtils; import io.druid.common.utils.SerializerUtils; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.ToLowerCaseAggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.ConciseCompressedIndexedInts; diff --git a/processing/src/main/java/io/druid/segment/column/ColumnConfig.java b/processing/src/main/java/io/druid/segment/column/ColumnConfig.java index 7dc9096a64f..94f990f1ec2 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnConfig.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnConfig.java @@ -1,5 +1,25 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.segment.column; public interface ColumnConfig { + public int columnCacheSizeBytes(); } diff --git a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java index e352d1d161e..af791e24525 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java @@ -92,14 +92,14 @@ public class ColumnDescriptor } } - public Column read(ByteBuffer buffer) + public Column read(ByteBuffer buffer, ColumnConfig columnConfig) { final ColumnBuilder builder = new ColumnBuilder() .setType(valueType) .setHasMultipleValues(hasMultipleValues); for (ColumnPartSerde part : parts) { - part.read(buffer, builder); + part.read(buffer, builder, columnConfig); } return builder.build(); diff --git a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java index 4925df0f62d..3eb22232b3a 100644 --- a/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DictionaryEncodedColumn.java @@ -21,9 +21,11 @@ package io.druid.segment.column; import io.druid.segment.data.IndexedInts; +import java.io.Closeable; + /** */ -public interface DictionaryEncodedColumn +public interface DictionaryEncodedColumn extends Closeable { public int length(); public boolean hasMultipleValues(); diff --git a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index 5e3bb3d0b9f..79463217035 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -24,6 +24,8 @@ import io.druid.segment.data.IndexedInts; import io.druid.segment.data.VSizeIndexed; import io.druid.segment.data.VSizeIndexedInts; +import java.io.IOException; + /** */ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn @@ -84,4 +86,10 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn { return lookups.size(); } + + @Override + public void close() throws IOException + { + lookups.close(); + } } diff --git a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java index c22685ddc65..21718ba9935 100644 --- a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -22,6 +22,7 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import java.io.IOException; import java.nio.ByteBuffer; @@ -40,5 +41,5 @@ public interface ColumnPartSerde { public long numBytes(); public void write(WritableByteChannel channel) throws IOException; - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder); + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); } diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java index 080978e77ab..f45adb57d47 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java @@ -22,6 +22,7 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.data.GenericIndexed; import java.io.IOException; @@ -71,7 +72,7 @@ public class ComplexColumnPartSerde implements ColumnPartSerde } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) { return serde == null ? this : serde.deserializeColumn(buffer, builder); } diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 391bc62ffdb..ccd08ddac62 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -25,6 +25,7 @@ import com.google.common.primitives.Ints; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; import io.druid.segment.data.ByteBufferSerializer; import io.druid.segment.data.ConciseCompressedIndexedInts; @@ -43,7 +44,9 @@ import java.nio.channels.WritableByteChannel; public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde { @JsonCreator - public static DictionaryEncodedColumnPartSerde createDeserializer(boolean singleValued) + public static DictionaryEncodedColumnPartSerde createDeserializer( + boolean singleValued + ) { return new DictionaryEncodedColumnPartSerde(); } @@ -125,7 +128,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) { final boolean isSingleValued = buffer.get() == 0x0; final GenericIndexed dictionary = GenericIndexed.read(buffer, GenericIndexed.stringStrategy); @@ -138,12 +141,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde singleValuedColumn = VSizeIndexedInts.readFromByteBuffer(buffer); multiValuedColumn = null; builder.setHasMultipleValues(false) - .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null)); + .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, singleValuedColumn, null, columnConfig.columnCacheSizeBytes())); } else { singleValuedColumn = null; multiValuedColumn = VSizeIndexed.readFromByteBuffer(buffer); builder.setHasMultipleValues(true) - .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn)); + .setDictionaryEncodedColumn(new DictionaryEncodedColumnSupplier(dictionary, null, multiValuedColumn, columnConfig.columnCacheSizeBytes())); } GenericIndexed bitmaps = GenericIndexed.read( diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java index d2a5ee82a85..a50d557c7f0 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnSupplier.java @@ -33,21 +33,28 @@ public class DictionaryEncodedColumnSupplier implements Supplier dictionary; private final VSizeIndexedInts singleValuedColumn; private final VSizeIndexed multiValuedColumn; + private final int lookupCacheSize; public DictionaryEncodedColumnSupplier( GenericIndexed dictionary, VSizeIndexedInts singleValuedColumn, - VSizeIndexed multiValuedColumn + VSizeIndexed multiValuedColumn, + int lookupCacheSize ) { this.dictionary = dictionary; this.singleValuedColumn = singleValuedColumn; this.multiValuedColumn = multiValuedColumn; + this.lookupCacheSize = lookupCacheSize; } @Override public DictionaryEncodedColumn get() { - return new SimpleDictionaryEncodedColumn(singleValuedColumn, multiValuedColumn, dictionary); + return new SimpleDictionaryEncodedColumn( + singleValuedColumn, + multiValuedColumn, + lookupCacheSize > 0 ? dictionary.withCache(lookupCacheSize) : dictionary + ); } } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index 0a0951af793..e72800cdb2c 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -22,6 +22,7 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedFloatsIndexedSupplier; @@ -70,7 +71,7 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) { final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, byteOrder); diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index 92e25b17e79..1954ca6c360 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -22,6 +22,7 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedLongsIndexedSupplier; @@ -70,7 +71,7 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder) + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) { final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, byteOrder); diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index d4e835566a9..e559cec4cc6 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import org.joda.time.Interval; @@ -33,6 +34,15 @@ import java.io.File; public class EmptyIndexTest { + final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + @Test public void testEmptyIndex() throws Exception { @@ -49,7 +59,7 @@ public class EmptyIndexTest IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex); IndexMerger.merge(Lists.newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir); - QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir); + QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir, columnConfig); Assert.assertEquals("getAvailableDimensions", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); Assert.assertEquals("getAvailableMetrics", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index a21eb92c718..cf2ae431def 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -26,6 +26,7 @@ import com.google.common.io.Files; import io.druid.data.input.MapBasedInputRow; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; import junit.framework.Assert; @@ -39,6 +40,15 @@ import java.util.Arrays; */ public class IndexMergerTest { + private final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + @Test public void testPersistCaseInsensitive() throws Exception { @@ -48,7 +58,7 @@ public class IndexMergerTest final File tempDir = Files.createTempDir(); try { - QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir)); + QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir), columnConfig); Assert.assertEquals(2, index.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); @@ -87,20 +97,21 @@ public class IndexMergerTest final File tempDir2 = Files.createTempDir(); final File mergedDir = Files.createTempDir(); try { - QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1)); + QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1), columnConfig); Assert.assertEquals(2, index1.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); Assert.assertEquals(2, index1.getColumnNames().size()); - QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2)); + QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2), columnConfig); Assert.assertEquals(2, index2.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); Assert.assertEquals(2, index2.getColumnNames().size()); QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir) + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir), + columnConfig ); Assert.assertEquals(3, merged.getTimeColumn().getLength()); @@ -140,10 +151,11 @@ public class IndexMergerTest ) ); - final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1)); - final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2)); + final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1), columnConfig); + final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2), columnConfig); final QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3), + columnConfig ); Assert.assertEquals(1, index1.getTimeColumn().getLength()); diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index d3073bcd6ac..6debfd2e64c 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -35,6 +35,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; @@ -51,6 +52,15 @@ import java.util.concurrent.atomic.AtomicLong; */ public class TestIndex { + private static final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + private static final Logger log = new Logger(TestIndex.class); private static IncrementalIndex realtimeIndex = null; @@ -135,10 +145,11 @@ public class TestIndex mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeQueryableIndex( - Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), + Arrays.asList(IndexIO.loadIndex(topFile, columnConfig), IndexIO.loadIndex(bottomFile, columnConfig)), METRIC_AGGS, mergedFile - ) + ), + columnConfig ); return mergedRealtime; @@ -229,7 +240,7 @@ public class TestIndex someTmpFile.deleteOnExit(); IndexMerger.persist(index, someTmpFile); - return IndexIO.loadIndex(someTmpFile); + return IndexIO.loadIndex(someTmpFile, columnConfig); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index c8155526a89..d1f01d6539a 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -49,6 +49,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestHelper; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import org.joda.time.DateTime; @@ -69,6 +70,15 @@ import java.util.Random; @RunWith(Parameterized.class) public class SpatialFilterBonusTest { + private static final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ @@ -220,7 +230,7 @@ public class SpatialFilterBonusTest tmpFile.deleteOnExit(); IndexMerger.persist(theIndex, tmpFile); - return IndexIO.loadIndex(tmpFile); + return IndexIO.loadIndex(tmpFile, columnConfig); } private static QueryableIndex makeMergedQueryableIndex() @@ -385,10 +395,11 @@ public class SpatialFilterBonusTest QueryableIndex mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeQueryableIndex( - Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), + Arrays.asList(IndexIO.loadIndex(firstFile, columnConfig), IndexIO.loadIndex(secondFile, columnConfig), IndexIO.loadIndex(thirdFile, columnConfig)), METRIC_AGGS, mergedFile - ) + ), + columnConfig ); return mergedRealtime; diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 84df58a260d..8ef4369ebff 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -49,6 +49,7 @@ import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestHelper; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import org.joda.time.DateTime; @@ -69,6 +70,16 @@ import java.util.Random; @RunWith(Parameterized.class) public class SpatialFilterTest { + + private static ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ @@ -235,7 +246,7 @@ public class SpatialFilterTest tmpFile.deleteOnExit(); IndexMerger.persist(theIndex, tmpFile); - return IndexIO.loadIndex(tmpFile); + return IndexIO.loadIndex(tmpFile, columnConfig); } private static QueryableIndex makeMergedQueryableIndex() @@ -415,10 +426,11 @@ public class SpatialFilterTest QueryableIndex mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeQueryableIndex( - Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), + Arrays.asList(IndexIO.loadIndex(firstFile, columnConfig), IndexIO.loadIndex(secondFile, columnConfig), IndexIO.loadIndex(thirdFile, columnConfig)), METRIC_AGGS, mergedFile - ) + ), + columnConfig ); return mergedRealtime; diff --git a/server/src/main/java/io/druid/guice/StorageNodeModule.java b/server/src/main/java/io/druid/guice/StorageNodeModule.java index 01f9563c7d5..3ce1020154c 100644 --- a/server/src/main/java/io/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/io/druid/guice/StorageNodeModule.java @@ -28,10 +28,12 @@ import io.druid.client.DruidServerConfig; import io.druid.guice.annotations.Self; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.loading.MMappedQueryableIndexFactory; import io.druid.segment.loading.QueryableIndexFactory; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.server.DruidNode; +import io.druid.server.DruidProcessingConfig; import io.druid.server.coordination.DruidServerMetadata; import javax.annotation.Nullable; @@ -48,6 +50,7 @@ public class StorageNodeModule implements Module binder.bind(NodeTypeConfig.class).toProvider(Providers.of(null)); binder.bind(QueryableIndexFactory.class).to(MMappedQueryableIndexFactory.class).in(LazySingleton.class); + binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); binder.bind(QueryRunnerFactoryConglomerate.class) .to(DefaultQueryRunnerFactoryConglomerate.class) diff --git a/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java index 717b0b513a0..7adef194779 100644 --- a/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java @@ -19,9 +19,11 @@ package io.druid.segment.loading; +import com.google.inject.Inject; import com.metamx.common.logger.Logger; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; +import io.druid.segment.column.ColumnConfig; import org.apache.commons.io.FileUtils; import java.io.File; @@ -33,11 +35,21 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory { private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); + private final ColumnConfig columnConfig; + + @Inject + public MMappedQueryableIndexFactory( + ColumnConfig columnConfig + ) + { + this.columnConfig = columnConfig; + } + @Override public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { - return IndexIO.loadIndex(parentDir); + return IndexIO.loadIndex(parentDir, columnConfig); } catch (IOException e) { log.warn(e, "Got exception!!!! Going to delete parentDir[%s]", parentDir); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 083c290c4cc..0e6a8786df3 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -27,6 +27,7 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import io.druid.common.guava.ThreadRenamingCallable; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireDepartmentMetrics; @@ -62,7 +63,8 @@ public class FlushingPlumber extends RealtimePlumber ServiceEmitter emitter, QueryRunnerFactoryConglomerate conglomerate, DataSegmentAnnouncer segmentAnnouncer, - ExecutorService queryExecutorService + ExecutorService queryExecutorService, + ColumnConfig columnConfig ) { super( @@ -75,7 +77,8 @@ public class FlushingPlumber extends RealtimePlumber queryExecutorService, null, null, - null + null, + columnConfig ); this.flushDuration = flushDuration; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 026af6f43c8..2c15a7c02f3 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -27,6 +27,7 @@ import com.metamx.common.Granularity; import com.metamx.emitter.service.ServiceEmitter; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.realtime.FireDepartmentMetrics; @@ -51,6 +52,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool private final QueryRunnerFactoryConglomerate conglomerate; private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; + private final ColumnConfig columnConfig; @JsonCreator public FlushingPlumberSchool( @@ -59,6 +61,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject @Processing ExecutorService queryExecutorService, + @JacksonInject ColumnConfig columnConfig, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -76,6 +79,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool null, null, queryExecutorService, + columnConfig, windowPeriod, basePersistDirectory, segmentGranularity, @@ -89,6 +93,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool this.conglomerate = conglomerate; this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; + this.columnConfig = columnConfig; } @Override @@ -108,7 +113,8 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool emitter, conglomerate, segmentAnnouncer, - queryExecutorService + queryExecutorService, + columnConfig ); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 306350aaaa3..96d835c4bef 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -37,6 +37,8 @@ import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -90,6 +92,7 @@ public class RealtimePlumber implements Plumber private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( String.CASE_INSENSITIVE_ORDER ); + private final ColumnConfig columnConfig; private volatile boolean shuttingDown = false; private volatile boolean stopped = false; private volatile ExecutorService persistExecutor = null; @@ -106,7 +109,8 @@ public class RealtimePlumber implements Plumber ExecutorService queryExecutorService, DataSegmentPusher dataSegmentPusher, SegmentPublisher segmentPublisher, - FilteredServerView serverView + FilteredServerView serverView, + ColumnConfig columnConfig ) { this.schema = schema; @@ -120,6 +124,7 @@ public class RealtimePlumber implements Plumber this.dataSegmentPusher = dataSegmentPusher; this.segmentPublisher = segmentPublisher; this.serverView = serverView; + this.columnConfig = columnConfig; log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); } @@ -342,7 +347,7 @@ public class RealtimePlumber implements Plumber mergedTarget ); - QueryableIndex index = IndexIO.loadIndex(mergedFile); + QueryableIndex index = IndexIO.loadIndex(mergedFile, columnConfig); DataSegment segment = dataSegmentPusher.push( mergedFile, @@ -528,7 +533,7 @@ public class RealtimePlumber implements Plumber versioningPolicy.getVersion(sinkInterval), config.getShardSpec() ), - IndexIO.loadIndex(segmentDir) + IndexIO.loadIndex(segmentDir, columnConfig) ), Integer.parseInt(segmentDir.getName()) ) @@ -709,7 +714,7 @@ public class RealtimePlumber implements Plumber indexToPersist.swapSegment( new QueryableIndexSegment( indexToPersist.getSegment().getIdentifier(), - IndexIO.loadIndex(persistedFile) + IndexIO.loadIndex(persistedFile, columnConfig) ) ); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index eb52a30ba31..c3480a682bd 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -29,6 +29,7 @@ import io.druid.client.FilteredServerView; import io.druid.client.ServerView; import io.druid.guice.annotations.Processing; import io.druid.query.QueryRunnerFactoryConglomerate; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -51,6 +52,7 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final FilteredServerView serverView; private final ExecutorService queryExecutorService; + private final ColumnConfig columnConfig; // Backwards compatible private final Period windowPeriod; @@ -69,6 +71,7 @@ public class RealtimePlumberSchool implements PlumberSchool @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject FilteredServerView serverView, @JacksonInject @Processing ExecutorService executorService, + @JacksonInject ColumnConfig columnConfig, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -85,6 +88,7 @@ public class RealtimePlumberSchool implements PlumberSchool this.segmentPublisher = segmentPublisher; this.serverView = serverView; this.queryExecutorService = executorService; + this.columnConfig = columnConfig; this.windowPeriod = windowPeriod; this.basePersistDirectory = basePersistDirectory; this.segmentGranularity = segmentGranularity; @@ -149,7 +153,8 @@ public class RealtimePlumberSchool implements PlumberSchool queryExecutorService, dataSegmentPusher, segmentPublisher, - serverView + serverView, + columnConfig ); } diff --git a/server/src/main/java/io/druid/server/DruidProcessingConfig.java b/server/src/main/java/io/druid/server/DruidProcessingConfig.java index af596f2da02..806e11e01da 100644 --- a/server/src/main/java/io/druid/server/DruidProcessingConfig.java +++ b/server/src/main/java/io/druid/server/DruidProcessingConfig.java @@ -20,11 +20,10 @@ package io.druid.server; import com.metamx.common.concurrent.ExecutorServiceConfig; +import io.druid.segment.column.ColumnConfig; import org.skife.config.Config; -/** - */ -public abstract class DruidProcessingConfig extends ExecutorServiceConfig +public abstract class DruidProcessingConfig extends ExecutorServiceConfig implements ColumnConfig { @Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"}) public int intermediateComputeSizeBytes() @@ -39,4 +38,10 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig final int processors = Runtime.getRuntime().availableProcessors(); return processors > 1 ? processors - 1 : processors; } + + @Config(value = "${base_path}.columnCache.sizeBytes") + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } } diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 19c104a4ff0..0113ee3dbc1 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -73,7 +73,7 @@ public class FireDepartmentTest new RealtimeIOConfig( null, new RealtimePlumberSchool( - null, null, null, null, null, null, null, null, null, null, null, null, 0 + null, null, null, null, null, null, null, null, null, null, null, null, null, 0 ) ), new RealtimeTuningConfig( diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 6b0a9e45f64..8f88d093efa 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -41,6 +41,7 @@ import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -64,6 +65,15 @@ import java.util.concurrent.TimeUnit; */ public class RealtimePlumberSchoolTest { + private final ColumnConfig columnConfig = new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 1024 * 1024; + } + }; + private Plumber plumber; private DataSegmentAnnouncer announcer; @@ -143,6 +153,7 @@ public class RealtimePlumberSchoolTest segmentPublisher, serverView, MoreExecutors.sameThreadExecutor(), + columnConfig, new Period("PT10m"), tmpDir, Granularity.HOUR, From c2e2391e3a5bad23affbfd06001e2088e8e91c64 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 29 May 2014 22:24:26 -0700 Subject: [PATCH 107/270] fix thread local initial value --- .../java/io/druid/segment/data/GenericIndexed.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 46cacfd8bac..551f5a29f23 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -189,13 +189,10 @@ public class GenericIndexed implements Indexed, Closeable } /** - * Creates a copy of the given indexed with the given cache + * Creates a copy of the given indexed with the given cache size * The resulting copy must be closed to release resources used by the cache - * - * @param other - * @param cache */ - GenericIndexed(GenericIndexed other, final SizedLRUMap cache) + GenericIndexed(GenericIndexed other, final int maxBytes) { this.theBuffer = other.theBuffer; this.strategy = other.strategy; @@ -209,7 +206,7 @@ public class GenericIndexed implements Indexed, Closeable @Override protected SizedLRUMap initialValue() { - return cache; + return new SizedLRUMap<>(INITIAL_CACHE_CAPACITY, maxBytes); } }; @@ -319,13 +316,13 @@ public class GenericIndexed implements Indexed, Closeable } /** - * The returned CachedIndexed must be closed to release the underlying memory + * The returned GenericIndexed must be closed to release the underlying memory * @param maxBytes * @return */ public GenericIndexed withCache(int maxBytes) { - return new GenericIndexed<>(this, new SizedLRUMap(INITIAL_CACHE_CAPACITY, maxBytes)); + return new GenericIndexed<>(this, maxBytes); } @Override From a870fe5cbe545c81991634956c25db5b402a0fb7 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 18 Jun 2014 13:17:48 -0700 Subject: [PATCH 108/270] inject column config --- .../indexer/HadoopDruidIndexerConfig.java | 8 +- .../io/druid/indexer/IndexGeneratorJob.java | 5 +- .../indexer/HadoopDruidIndexerConfigTest.java | 13 --- .../io/druid/indexing/common/TaskToolbox.java | 15 +-- .../indexing/common/TaskToolboxFactory.java | 8 +- .../common/index/YeOldePlumberSchool.java | 10 +- .../indexing/common/task/AppendTask.java | 8 +- .../indexing/common/task/HadoopIndexTask.java | 5 +- .../druid/indexing/common/task/IndexTask.java | 3 +- .../druid/indexing/common/task/MergeTask.java | 9 +- .../common/task/RealtimeIndexTask.java | 6 +- .../common/task/VersionConverterTask.java | 2 +- .../druid/indexing/common/TestMergeTask.java | 10 +- .../indexing/common/TestRealtimeTask.java | 1 - .../indexing/common/task/TaskSerdeTest.java | 46 +-------- .../indexing/overlord/TaskLifecycleTest.java | 10 +- .../SimpleResourceManagementStrategyTest.java | 3 +- .../indexing/worker/TaskAnnouncementTest.java | 1 - .../worker/WorkerTaskMonitorTest.java | 2 +- .../java/io/druid/guice}/ConfigModule.java | 2 +- .../io/druid/guice}/ExtensionsConfig.java | 2 +- .../io/druid/guice}/PropertiesModule.java | 2 +- .../druid/guice/annotations/Processing.java | 0 .../druid/initialization/GuiceInjectors.java | 86 +++++++++++++++++ .../io/druid/initialization/ModuleList.java | 93 +++++++++++++++++++ .../druid/query}/DruidProcessingConfig.java | 2 +- .../main/java/io/druid/query}/VMUtils.java | 2 +- .../main/java/io/druid/segment/IndexIO.java | 62 ++++++++++--- .../java/io/druid/segment/EmptyIndexTest.java | 11 +-- .../io/druid/segment/IndexMergerTest.java | 25 ++--- .../test/java/io/druid/segment/TestIndex.java | 16 +--- .../filter/SpatialFilterBonusTest.java | 16 +--- .../segment/filter/SpatialFilterTest.java | 17 +--- .../io/druid/guice/DruidProcessingModule.java | 4 +- .../io/druid/guice/StorageNodeModule.java | 2 +- .../druid/initialization/Initialization.java | 28 +----- .../loading/MMappedQueryableIndexFactory.java | 14 +-- .../realtime/plumber/FlushingPlumber.java | 6 +- .../plumber/FlushingPlumberSchool.java | 7 +- .../realtime/plumber/RealtimePlumber.java | 13 +-- .../plumber/RealtimePlumberSchool.java | 6 +- .../initialization/InitializationTest.java | 8 +- .../segment/realtime/FireDepartmentTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 10 -- .../server/initialization/JettyTest.java | 3 +- .../java/io/druid/cli/CliHadoopIndexer.java | 2 +- services/src/main/java/io/druid/cli/Main.java | 8 +- .../java/io/druid/cli/PullDependencies.java | 2 +- 48 files changed, 302 insertions(+), 314 deletions(-) rename {server/src/main/java/io/druid/server/initialization => processing/src/main/java/io/druid/guice}/ConfigModule.java (97%) rename {server/src/main/java/io/druid/server/initialization => processing/src/main/java/io/druid/guice}/ExtensionsConfig.java (98%) rename {server/src/main/java/io/druid/server/initialization => processing/src/main/java/io/druid/guice}/PropertiesModule.java (98%) rename {server => processing}/src/main/java/io/druid/guice/annotations/Processing.java (100%) create mode 100644 processing/src/main/java/io/druid/initialization/GuiceInjectors.java create mode 100644 processing/src/main/java/io/druid/initialization/ModuleList.java rename {server/src/main/java/io/druid/server => processing/src/main/java/io/druid/query}/DruidProcessingConfig.java (98%) rename {server/src/main/java/io/druid/server => processing/src/main/java/io/druid/query}/VMUtils.java (98%) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 59b9d09d5d9..d338506e512 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -19,7 +19,6 @@ package io.druid.indexer; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; @@ -45,6 +44,7 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.path.PathSpec; +import io.druid.initialization.GuiceInjectors; import io.druid.initialization.Initialization; import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.GranularitySpec; @@ -83,7 +83,7 @@ public class HadoopDruidIndexerConfig static { injector = Initialization.makeInjectorWithModules( - Initialization.makeStartupInjector(), + GuiceInjectors.makeStartupInjector(), ImmutableList.of( new Module() { @@ -107,7 +107,7 @@ public class HadoopDruidIndexerConfig public static HadoopDruidIndexerConfig fromSchema(HadoopIngestionSpec schema) { - return new HadoopDruidIndexerConfig(injector.getInstance(ColumnConfig.class), schema); + return new HadoopDruidIndexerConfig(schema); } public static HadoopDruidIndexerConfig fromMap(Map argSpec) @@ -117,7 +117,6 @@ public class HadoopDruidIndexerConfig return HadoopDruidIndexerConfig.jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class); } else { return new HadoopDruidIndexerConfig( - injector.getInstance(ColumnConfig.class), HadoopDruidIndexerConfig.jsonMapper.convertValue( argSpec, HadoopIngestionSpec.class @@ -173,7 +172,6 @@ public class HadoopDruidIndexerConfig @JsonCreator public HadoopDruidIndexerConfig( - @JacksonInject final ColumnConfig columnConfig, final @JsonProperty("schema") HadoopIngestionSpec schema ) { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 29f9be0dad8..6dd97b920ba 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -40,7 +40,6 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.timeline.DataSegment; @@ -282,8 +281,6 @@ public class IndexGeneratorJob implements Jobby //final DataRollupSpec rollupSpec = config.getRollupSpec(); final AggregatorFactory[] aggs = config.getSchema().getDataSchema().getAggregators(); - final ColumnConfig columnConfig = config.getColumnConfig(); - IncrementalIndex index = makeIncrementalIndex(bucket, aggs); File baseFlushFile = File.createTempFile("base", "flush"); @@ -374,7 +371,7 @@ public class IndexGeneratorJob implements Jobby } for (File file : toMerge) { - indexes.add(IndexIO.loadIndex(file, columnConfig)); + indexes.add(IndexIO.loadIndex(file)); } mergedBase = IndexMerger.mergeQueryableIndex( indexes, aggs, new File(baseFlushFile, "merged"), new IndexMerger.ProgressIndicator() diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index 52c63779e19..ee8fa9315c5 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -22,7 +22,6 @@ package io.druid.indexer; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import io.druid.jackson.DefaultObjectMapper; -import io.druid.segment.column.ColumnConfig; import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; @@ -46,16 +45,6 @@ public class HadoopDruidIndexerConfigTest } } - private static final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - - @Test public void shouldMakeHDFSCompliantSegmentOutputPath() { @@ -80,7 +69,6 @@ public class HadoopDruidIndexerConfigTest } HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( - columnConfig, schema.withTuningConfig( schema.getTuningConfig() .withVersion( @@ -121,7 +109,6 @@ public class HadoopDruidIndexerConfigTest } HadoopDruidIndexerConfig cfg = new HadoopDruidIndexerConfig( - columnConfig, schema.withTuningConfig( schema.getTuningConfig() .withVersion( diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index db2012e8f03..cfd19a275dd 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -34,7 +34,6 @@ import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -50,7 +49,6 @@ import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutorService; /** @@ -74,7 +72,6 @@ public class TaskToolbox private final SegmentLoader segmentLoader; private final ObjectMapper objectMapper; private final File taskWorkDir; - private final ColumnConfig columnConfig; public TaskToolbox( TaskConfig config, @@ -92,8 +89,7 @@ public class TaskToolbox MonitorScheduler monitorScheduler, SegmentLoader segmentLoader, ObjectMapper objectMapper, - final File taskWorkDir, - ColumnConfig columnConfig + final File taskWorkDir ) { this.config = config; @@ -112,7 +108,6 @@ public class TaskToolbox this.segmentLoader = segmentLoader; this.objectMapper = objectMapper; this.taskWorkDir = taskWorkDir; - this.columnConfig = columnConfig; } public TaskConfig getConfig() @@ -180,11 +175,6 @@ public class TaskToolbox return objectMapper; } - public ColumnConfig getColumnConfig() - { - return columnConfig; - } - public Map fetchSegments(List segments) throws SegmentLoadingException { @@ -196,7 +186,8 @@ public class TaskToolbox return retVal; } - public void pushSegments(Iterable segments) throws IOException { + public void pushSegments(Iterable segments) throws IOException + { // Request segment pushes for each set final Multimap segmentMultimap = Multimaps.index( segments, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 61c83a22cf4..41181d6d44d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -58,7 +58,6 @@ public class TaskToolboxFactory private final MonitorScheduler monitorScheduler; private final SegmentLoaderFactory segmentLoaderFactory; private final ObjectMapper objectMapper; - private final ColumnConfig columnConfig; @Inject public TaskToolboxFactory( @@ -75,8 +74,7 @@ public class TaskToolboxFactory @Processing ExecutorService queryExecutorService, MonitorScheduler monitorScheduler, SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper, - ColumnConfig columnConfig + ObjectMapper objectMapper ) { this.config = config; @@ -93,7 +91,6 @@ public class TaskToolboxFactory this.monitorScheduler = monitorScheduler; this.segmentLoaderFactory = segmentLoaderFactory; this.objectMapper = objectMapper; - this.columnConfig = columnConfig; } public TaskToolbox build(Task task) @@ -116,8 +113,7 @@ public class TaskToolboxFactory monitorScheduler, segmentLoaderFactory.manufacturate(taskWorkDir), objectMapper, - taskWorkDir, - columnConfig + taskWorkDir ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index ea3be607681..11f6bb2264d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -38,7 +38,6 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.SegmentUtils; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -66,7 +65,6 @@ public class YeOldePlumberSchool implements PlumberSchool private final String version; private final DataSegmentPusher dataSegmentPusher; private final File tmpSegmentDir; - private final ColumnConfig columnConfig; private static final Logger log = new Logger(YeOldePlumberSchool.class); @@ -75,15 +73,13 @@ public class YeOldePlumberSchool implements PlumberSchool @JsonProperty("interval") Interval interval, @JsonProperty("version") String version, @JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher, - @JacksonInject("tmpSegmentDir") File tmpSegmentDir, - @JacksonInject ColumnConfig columnConfig + @JacksonInject("tmpSegmentDir") File tmpSegmentDir ) { this.interval = interval; this.version = version; this.dataSegmentPusher = dataSegmentPusher; this.tmpSegmentDir = tmpSegmentDir; - this.columnConfig = columnConfig; } @Override @@ -166,7 +162,7 @@ public class YeOldePlumberSchool implements PlumberSchool } else { List indexes = Lists.newArrayList(); for (final File oneSpill : spilled) { - indexes.add(IndexIO.loadIndex(oneSpill, columnConfig)); + indexes.add(IndexIO.loadIndex(oneSpill)); } fileToUpload = new File(tmpSegmentDir, "merged"); @@ -174,7 +170,7 @@ public class YeOldePlumberSchool implements PlumberSchool } // Map merged segment so we can extract dimensions - final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload, columnConfig); + final QueryableIndex mappedSegment = IndexIO.loadIndex(fileToUpload); final DataSegment segmentToUpload = theSink.getSegment() .withDimensions(ImmutableList.copyOf(mappedSegment.getAvailableDimensions())) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index aeec42cf77e..40807073874 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -19,7 +19,6 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; @@ -33,7 +32,6 @@ import io.druid.segment.IndexableAdapter; import io.druid.segment.QueryableIndexIndexableAdapter; import io.druid.segment.Rowboat; import io.druid.segment.RowboatFilteringIndexAdapter; -import io.druid.segment.column.ColumnConfig; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; @@ -48,18 +46,14 @@ import java.util.Map; */ public class AppendTask extends MergeTaskBase { - private final ColumnConfig columnConfig; - @JsonCreator public AppendTask( - @JacksonInject ColumnConfig columnConfig, @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments ) { super(id, dataSource, segments); - this.columnConfig = columnConfig; } @Override @@ -98,7 +92,7 @@ public class AppendTask extends MergeTaskBase adapters.add( new RowboatFilteringIndexAdapter( new QueryableIndexIndexableAdapter( - IndexIO.loadIndex(holder.getFile(), columnConfig) + IndexIO.loadIndex(holder.getFile()) ), new Predicate() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index ce6324e32e8..9e5a67c9ec5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -42,8 +42,9 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.initialization.GuiceInjectors; import io.druid.initialization.Initialization; -import io.druid.server.initialization.ExtensionsConfig; +import io.druid.guice.ExtensionsConfig; import io.druid.timeline.DataSegment; import io.tesla.aether.internal.DefaultTeslaAether; import org.joda.time.DateTime; @@ -63,7 +64,7 @@ public class HadoopIndexTask extends AbstractTask private static final ExtensionsConfig extensionsConfig; static { - extensionsConfig = Initialization.makeStartupInjector().getInstance(ExtensionsConfig.class); + extensionsConfig = GuiceInjectors.makeStartupInjector().getInstance(ExtensionsConfig.class); } private static String getTheDataSource(HadoopIngestionSpec spec, HadoopIngestionSpec config) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 0fb759db931..182a1569903 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -382,8 +382,7 @@ public class IndexTask extends AbstractFixedIntervalTask interval, version, wrappedDataSegmentPusher, - tmpDir, - toolbox.getColumnConfig() + tmpDir ).findPlumber(schema, new RealtimeTuningConfig(null, null, null, null, null, null, null, shardSpec), metrics); // rowFlushBoundary for this job diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java index 8b9ef088f6b..9bca4a3eee5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java @@ -19,7 +19,6 @@ package io.druid.indexing.common.task; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -31,7 +30,6 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; -import io.druid.segment.column.ColumnConfig; import io.druid.timeline.DataSegment; import javax.annotation.Nullable; @@ -45,20 +43,17 @@ public class MergeTask extends MergeTaskBase { @JsonIgnore private final List aggregators; - private final ColumnConfig columnConfig; @JsonCreator public MergeTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators, - @JacksonInject ColumnConfig columnConfig + @JsonProperty("aggregations") List aggregators ) { super(id, dataSource, segments); this.aggregators = aggregators; - this.columnConfig = columnConfig; } @Override @@ -74,7 +69,7 @@ public class MergeTask extends MergeTaskBase public QueryableIndex apply(@Nullable File input) { try { - return IndexIO.loadIndex(input, columnConfig); + return IndexIO.loadIndex(input); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 6232fbe5691..57b6c0ddaa1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -96,7 +96,6 @@ public class RealtimeIndexTask extends AbstractTask @JsonIgnore private final FireDepartment spec; - private final ColumnConfig columnConfig; @JsonIgnore private volatile Plumber plumber = null; @@ -116,8 +115,7 @@ public class RealtimeIndexTask extends AbstractTask @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("maxPendingPersists") int maxPendingPersists, @JsonProperty("segmentGranularity") Granularity segmentGranularity, - @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, - @JacksonInject ColumnConfig columnConfig + @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory ) { super( @@ -151,7 +149,6 @@ public class RealtimeIndexTask extends AbstractTask null, null, null, null ); } - this.columnConfig = columnConfig; } @Override @@ -313,7 +310,6 @@ public class RealtimeIndexTask extends AbstractTask segmentPublisher, toolbox.getNewSegmentServerView(), toolbox.getQueryExecutorService(), - columnConfig, null, null, null, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java index cbb388a0fc3..d10ea9948c9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/VersionConverterTask.java @@ -252,7 +252,7 @@ public class VersionConverterTask extends AbstractFixedIntervalTask final File location = localSegments.get(segment); final File outLocation = new File(location, "v9_out"); - if (IndexIO.convertSegment(location, outLocation, toolbox.getColumnConfig())) { + if (IndexIO.convertSegment(location, outLocation)) { final int outVersion = IndexIO.getVersionFromDir(outLocation); // Appending to the version makes a new version that inherits most comparability parameters of the original diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java index 4775bb407f4..05f3118d3c9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestMergeTask.java @@ -19,14 +19,12 @@ package io.druid.indexing.common; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Lists; import io.druid.indexing.common.task.MergeTask; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.segment.column.ColumnConfig; import io.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -56,8 +54,7 @@ public class TestMergeTask extends MergeTask 0 ) ), - Lists.newArrayList(), - null + Lists.newArrayList() ); } @@ -68,11 +65,10 @@ public class TestMergeTask extends MergeTask @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators, - @JacksonInject ColumnConfig columnConfig + @JsonProperty("aggregations") List aggregators ) { - super(id, dataSource, segments, aggregators, columnConfig); + super(id, dataSource, segments, aggregators); this.id = id; } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java index 5280a884a55..6220f447bde 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java @@ -54,7 +54,6 @@ public class TestRealtimeTask extends RealtimeIndexTask null, 1, null, - null, null ); this.status = status; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 5fa3548d797..0cd0fde9326 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -21,15 +21,8 @@ package io.druid.indexing.common.task; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; -import com.fasterxml.jackson.databind.introspect.GuiceAnnotationIntrospector; -import com.fasterxml.jackson.databind.introspect.GuiceInjectableValues; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; import com.metamx.common.Granularity; import io.druid.data.input.impl.JSONDataSpec; import io.druid.data.input.impl.TimestampSpec; @@ -41,7 +34,6 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.realtime.Schema; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; @@ -56,40 +48,7 @@ import java.io.File; public class TaskSerdeTest { - private static final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); - private static final Injector injector = Guice.createInjector( - new com.google.inject.Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(ColumnConfig.class).toInstance(columnConfig); - } - } - ); - - static { - final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector(); - - jsonMapper.setInjectableValues(new GuiceInjectableValues(injector)); - jsonMapper.setAnnotationIntrospectors( - new AnnotationIntrospectorPair( - guiceIntrospector, jsonMapper.getSerializationConfig().getAnnotationIntrospector() - ), - new AnnotationIntrospectorPair( - guiceIntrospector, jsonMapper.getDeserializationConfig().getAnnotationIntrospector() - ) - ); - } @Test public void testIndexTaskSerde() throws Exception @@ -141,8 +100,7 @@ public class TaskSerdeTest ), ImmutableList.of( new CountAggregatorFactory("cnt") - ), - null + ) ); final String json = jsonMapper.writeValueAsString(task); @@ -244,7 +202,6 @@ public class TaskSerdeTest new Period("PT10M"), 1, Granularity.HOUR, - null, null ); @@ -330,7 +287,6 @@ public class TaskSerdeTest public void testAppendTaskSerde() throws Exception { final AppendTask task = new AppendTask( - columnConfig, null, "foo", ImmutableList.of( diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 9fa2b76617f..55c00a58d9c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -206,15 +206,7 @@ public class TaskLifecycleTest } ) ), - new DefaultObjectMapper(), - new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - } + new DefaultObjectMapper() ); tr = new ThreadPoolTaskRunner(tb); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 78ec78c3f06..21f3277a653 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -87,8 +87,7 @@ public class SimpleResourceManagementStrategyTest 0 ) ), - Lists.newArrayList(), - null + Lists.newArrayList() ); simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( autoScalingStrategy, diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index ec593aeee33..7aafad53e23 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -49,7 +49,6 @@ public class TaskAnnouncementTest new Period("PT10M"), 1, Granularity.HOUR, - null, null ); final TaskStatus status = TaskStatus.running(task.getId()); diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index 21070ba0340..11756824c34 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -179,7 +179,7 @@ public class WorkerTaskMonitorTest } } ) - ), jsonMapper, null + ), jsonMapper ) ), new WorkerConfig().setCapacity(1) diff --git a/server/src/main/java/io/druid/server/initialization/ConfigModule.java b/processing/src/main/java/io/druid/guice/ConfigModule.java similarity index 97% rename from server/src/main/java/io/druid/server/initialization/ConfigModule.java rename to processing/src/main/java/io/druid/guice/ConfigModule.java index 62644146a05..71cb1e31f8b 100644 --- a/server/src/main/java/io/druid/server/initialization/ConfigModule.java +++ b/processing/src/main/java/io/druid/guice/ConfigModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization; +package io.druid.guice; import com.google.inject.Binder; import com.google.inject.Module; diff --git a/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java b/processing/src/main/java/io/druid/guice/ExtensionsConfig.java similarity index 98% rename from server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java rename to processing/src/main/java/io/druid/guice/ExtensionsConfig.java index b4d2699a2a4..b904b421045 100644 --- a/server/src/main/java/io/druid/server/initialization/ExtensionsConfig.java +++ b/processing/src/main/java/io/druid/guice/ExtensionsConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization; +package io.druid.guice; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; diff --git a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java b/processing/src/main/java/io/druid/guice/PropertiesModule.java similarity index 98% rename from server/src/main/java/io/druid/server/initialization/PropertiesModule.java rename to processing/src/main/java/io/druid/guice/PropertiesModule.java index 56773ee5a7d..bd0868e2414 100644 --- a/server/src/main/java/io/druid/server/initialization/PropertiesModule.java +++ b/processing/src/main/java/io/druid/guice/PropertiesModule.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server.initialization; +package io.druid.guice; import com.google.common.base.Charsets; import com.google.common.base.Throwables; diff --git a/server/src/main/java/io/druid/guice/annotations/Processing.java b/processing/src/main/java/io/druid/guice/annotations/Processing.java similarity index 100% rename from server/src/main/java/io/druid/guice/annotations/Processing.java rename to processing/src/main/java/io/druid/guice/annotations/Processing.java diff --git a/processing/src/main/java/io/druid/initialization/GuiceInjectors.java b/processing/src/main/java/io/druid/initialization/GuiceInjectors.java new file mode 100644 index 00000000000..74dd122cd55 --- /dev/null +++ b/processing/src/main/java/io/druid/initialization/GuiceInjectors.java @@ -0,0 +1,86 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.initialization; + +import com.google.common.collect.Lists; +import com.google.inject.Binder; +import com.google.inject.Guice; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.util.Modules; +import io.druid.guice.ConfigModule; +import io.druid.guice.DruidGuiceExtensions; +import io.druid.guice.DruidSecondaryModule; +import io.druid.guice.ExtensionsConfig; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.PropertiesModule; +import io.druid.jackson.JacksonModule; + +import java.util.List; + +/** + */ +public class GuiceInjectors +{ + public static Injector makeStartupInjector() + { + return Guice.createInjector( + new DruidGuiceExtensions(), + new JacksonModule(), + new PropertiesModule("runtime.properties"), + new ConfigModule(), + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(DruidSecondaryModule.class); + JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); + } + } + ); + } + + public static Injector makeStartupInjectorWithModules(Iterable modules) + { + List theModules = Lists.newArrayList(); + theModules.add(new DruidGuiceExtensions()); + theModules.add(new JacksonModule()); + theModules.add(new PropertiesModule("runtime.properties")); + theModules.add(new ConfigModule()); + theModules.add( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(DruidSecondaryModule.class); + JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); + } + } + ); + for (Module theModule : modules) { + theModules.add(theModule); + } + + + return Guice.createInjector(theModules); + } +} diff --git a/processing/src/main/java/io/druid/initialization/ModuleList.java b/processing/src/main/java/io/druid/initialization/ModuleList.java new file mode 100644 index 00000000000..113e61a5982 --- /dev/null +++ b/processing/src/main/java/io/druid/initialization/ModuleList.java @@ -0,0 +1,93 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.initialization; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.metamx.common.ISE; +import io.druid.guice.annotations.Json; +import io.druid.guice.annotations.Smile; + +import java.util.Collections; +import java.util.List; + +/** + */ +public class ModuleList +{ + private final Injector baseInjector; + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final List modules; + + public ModuleList(Injector baseInjector) + { + this.baseInjector = baseInjector; + this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class)); + this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class)); + this.modules = Lists.newArrayList(); + } + + public List getModules() + { + return Collections.unmodifiableList(modules); + } + + public void addModule(Object input) + { + if (input instanceof DruidModule) { + baseInjector.injectMembers(input); + modules.add(registerJacksonModules(((DruidModule) input))); + } else if (input instanceof Module) { + baseInjector.injectMembers(input); + modules.add((Module) input); + } else if (input instanceof Class) { + if (DruidModule.class.isAssignableFrom((Class) input)) { + modules.add(registerJacksonModules(baseInjector.getInstance((Class) input))); + } else if (Module.class.isAssignableFrom((Class) input)) { + modules.add(baseInjector.getInstance((Class) input)); + return; + } else { + throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class); + } + } else { + throw new ISE("Unknown module type[%s]", input.getClass()); + } + } + + public void addModules(Object... object) + { + for (Object o : object) { + addModule(o); + } + } + + private DruidModule registerJacksonModules(DruidModule module) + { + for (com.fasterxml.jackson.databind.Module jacksonModule : module.getJacksonModules()) { + jsonMapper.registerModule(jacksonModule); + smileMapper.registerModule(jacksonModule); + } + return module; + } +} diff --git a/server/src/main/java/io/druid/server/DruidProcessingConfig.java b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java similarity index 98% rename from server/src/main/java/io/druid/server/DruidProcessingConfig.java rename to processing/src/main/java/io/druid/query/DruidProcessingConfig.java index 806e11e01da..208cd345805 100644 --- a/server/src/main/java/io/druid/server/DruidProcessingConfig.java +++ b/processing/src/main/java/io/druid/query/DruidProcessingConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server; +package io.druid.query; import com.metamx.common.concurrent.ExecutorServiceConfig; import io.druid.segment.column.ColumnConfig; diff --git a/server/src/main/java/io/druid/server/VMUtils.java b/processing/src/main/java/io/druid/query/VMUtils.java similarity index 98% rename from server/src/main/java/io/druid/server/VMUtils.java rename to processing/src/main/java/io/druid/query/VMUtils.java index bc694f59355..db1213cbe7e 100644 --- a/server/src/main/java/io/druid/server/VMUtils.java +++ b/processing/src/main/java/io/druid/query/VMUtils.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.server; +package io.druid.query; import java.lang.reflect.InvocationTargetException; diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 9852a0cef24..20624b60ed9 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -22,6 +22,7 @@ package io.druid.segment; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -31,6 +32,9 @@ import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; import com.google.common.io.Files; import com.google.common.primitives.Ints; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Module; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; import com.metamx.common.ISE; @@ -41,7 +45,10 @@ import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import io.druid.common.utils.SerializerUtils; +import io.druid.guice.ConfigProvider; +import io.druid.initialization.GuiceInjectors; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.DruidProcessingConfig; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; @@ -91,6 +98,9 @@ public class IndexIO { public static final byte V8_VERSION = 0x8; public static final byte V9_VERSION = 0x9; + public static final int CURRENT_VERSION_ID = V9_VERSION; + + public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); private static final Map indexLoaders = ImmutableMap.builder() @@ -108,13 +118,32 @@ public class IndexIO private static final EmittingLogger log = new EmittingLogger(IndexIO.class); private static final SerializerUtils serializerUtils = new SerializerUtils(); - public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); - // This should really be provided by DI, should be changed once we switch around to using a DI framework - private static final ObjectMapper mapper = new DefaultObjectMapper(); + private static final ObjectMapper mapper; + private static final ColumnConfig columnConfig; + + static { + final Injector injector = GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + ConfigProvider.bind( + binder, + DruidProcessingConfig.class, + ImmutableMap.of("base_path", "druid.processing") + ); + } + } + ) + ); + mapper = injector.getInstance(ObjectMapper.class); + columnConfig = injector.getInstance(DruidProcessingConfig.class); + } private static volatile IndexIOHandler handler = null; - public static final int CURRENT_VERSION_ID = V9_VERSION; @Deprecated public static MMappedIndex mapDir(final File inDir) throws IOException @@ -123,7 +152,7 @@ public class IndexIO return handler.mapDir(inDir); } - public static QueryableIndex loadIndex(File inDir, ColumnConfig columnConfig) throws IOException + public static QueryableIndex loadIndex(File inDir) throws IOException { init(); final int version = SegmentUtils.getVersionFromDir(inDir); @@ -131,7 +160,7 @@ public class IndexIO final IndexLoader loader = indexLoaders.get(version); if (loader != null) { - return loader.load(inDir, columnConfig); + return loader.load(inDir); } else { throw new ISE("Unknown index version[%s]", version); } @@ -181,7 +210,7 @@ public class IndexIO } } - public static boolean convertSegment(File toConvert, File converted, ColumnConfig columnConfig) throws IOException + public static boolean convertSegment(File toConvert, File converted) throws IOException { final int version = SegmentUtils.getVersionFromDir(toConvert); @@ -199,7 +228,7 @@ public class IndexIO case 7: log.info("Old version, re-persisting."); IndexMerger.append( - Arrays.asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert, columnConfig))), + Arrays.asList(new QueryableIndexIndexableAdapter(loadIndex(toConvert))), converted ); return true; @@ -609,13 +638,13 @@ public class IndexIO static interface IndexLoader { - public QueryableIndex load(File inDir, ColumnConfig columnConfig) throws IOException; + public QueryableIndex load(File inDir) throws IOException; } static class LegacyIndexLoader implements IndexLoader { @Override - public QueryableIndex load(File inDir, ColumnConfig columnConfig) throws IOException + public QueryableIndex load(File inDir) throws IOException { MMappedIndex index = IndexIO.mapDir(inDir); @@ -627,7 +656,10 @@ public class IndexIO .setHasMultipleValues(true) .setDictionaryEncodedColumn( new DictionaryEncodedColumnSupplier( - index.getDimValueLookup(dimension), null, index.getDimColumn(dimension), columnConfig.columnCacheSizeBytes() + index.getDimValueLookup(dimension), + null, + index.getDimColumn(dimension), + columnConfig.columnCacheSizeBytes() ) ) .setBitmapIndex( @@ -700,7 +732,7 @@ public class IndexIO static class V9IndexLoader implements IndexLoader { @Override - public QueryableIndex load(File inDir, ColumnConfig columnConfig) throws IOException + public QueryableIndex load(File inDir) throws IOException { log.debug("Mapping v9 index[%s]", inDir); long startTime = System.currentTimeMillis(); @@ -722,11 +754,11 @@ public class IndexIO ObjectMapper mapper = new DefaultObjectMapper(); for (String columnName : cols) { - columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName), columnConfig)); + columns.put(columnName, deserializeColumn(mapper, smooshedFiles.mapFile(columnName))); } final QueryableIndex index = new SimpleQueryableIndex( - dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time"), columnConfig), columns, smooshedFiles + dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns, smooshedFiles ); log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); @@ -734,7 +766,7 @@ public class IndexIO return index; } - private Column deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer, ColumnConfig columnConfig) throws IOException + private Column deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer) throws IOException { ColumnDescriptor serde = mapper.readValue( serializerUtils.readString(byteBuffer), ColumnDescriptor.class diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index e559cec4cc6..fe57fd1ddac 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -34,15 +34,6 @@ import java.io.File; public class EmptyIndexTest { - final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - @Test public void testEmptyIndex() throws Exception { @@ -59,7 +50,7 @@ public class EmptyIndexTest IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter(new Interval("2012-08-01/P3D"), emptyIndex); IndexMerger.merge(Lists.newArrayList(emptyIndexAdapter), new AggregatorFactory[0], tmpDir); - QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir, columnConfig); + QueryableIndex emptyQueryableIndex = IndexIO.loadIndex(tmpDir); Assert.assertEquals("getAvailableDimensions", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); Assert.assertEquals("getAvailableMetrics", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index cf2ae431def..b3d36852364 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -40,15 +40,6 @@ import java.util.Arrays; */ public class IndexMergerTest { - private final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - @Test public void testPersistCaseInsensitive() throws Exception { @@ -58,7 +49,7 @@ public class IndexMergerTest final File tempDir = Files.createTempDir(); try { - QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir), columnConfig); + QueryableIndex index = IndexIO.loadIndex(IndexMerger.persist(toPersist, tempDir)); Assert.assertEquals(2, index.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); @@ -97,21 +88,20 @@ public class IndexMergerTest final File tempDir2 = Files.createTempDir(); final File mergedDir = Files.createTempDir(); try { - QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1), columnConfig); + QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tempDir1)); Assert.assertEquals(2, index1.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); Assert.assertEquals(2, index1.getColumnNames().size()); - QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2), columnConfig); + QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist2, tempDir2)); Assert.assertEquals(2, index2.getTimeColumn().getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); Assert.assertEquals(2, index2.getColumnNames().size()); QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir), - columnConfig + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, mergedDir) ); Assert.assertEquals(3, merged.getTimeColumn().getLength()); @@ -151,11 +141,10 @@ public class IndexMergerTest ) ); - final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1), columnConfig); - final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2), columnConfig); + final QueryableIndex index1 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir1)); + final QueryableIndex index2 = IndexIO.loadIndex(IndexMerger.persist(toPersist1, tmpDir2)); final QueryableIndex merged = IndexIO.loadIndex( - IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3), - columnConfig + IndexMerger.mergeQueryableIndex(Arrays.asList(index1, index2), new AggregatorFactory[]{}, tmpDir3) ); Assert.assertEquals(1, index1.getTimeColumn().getLength()); diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 6debfd2e64c..3d37545027c 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -52,15 +52,6 @@ import java.util.concurrent.atomic.AtomicLong; */ public class TestIndex { - private static final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - private static final Logger log = new Logger(TestIndex.class); private static IncrementalIndex realtimeIndex = null; @@ -145,11 +136,10 @@ public class TestIndex mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeQueryableIndex( - Arrays.asList(IndexIO.loadIndex(topFile, columnConfig), IndexIO.loadIndex(bottomFile, columnConfig)), + Arrays.asList(IndexIO.loadIndex(topFile), IndexIO.loadIndex(bottomFile)), METRIC_AGGS, mergedFile - ), - columnConfig + ) ); return mergedRealtime; @@ -240,7 +230,7 @@ public class TestIndex someTmpFile.deleteOnExit(); IndexMerger.persist(index, someTmpFile); - return IndexIO.loadIndex(someTmpFile, columnConfig); + return IndexIO.loadIndex(someTmpFile); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index d1f01d6539a..8083cde3399 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -70,15 +70,6 @@ import java.util.Random; @RunWith(Parameterized.class) public class SpatialFilterBonusTest { - private static final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ @@ -230,7 +221,7 @@ public class SpatialFilterBonusTest tmpFile.deleteOnExit(); IndexMerger.persist(theIndex, tmpFile); - return IndexIO.loadIndex(tmpFile, columnConfig); + return IndexIO.loadIndex(tmpFile); } private static QueryableIndex makeMergedQueryableIndex() @@ -395,11 +386,10 @@ public class SpatialFilterBonusTest QueryableIndex mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeQueryableIndex( - Arrays.asList(IndexIO.loadIndex(firstFile, columnConfig), IndexIO.loadIndex(secondFile, columnConfig), IndexIO.loadIndex(thirdFile, columnConfig)), + Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), METRIC_AGGS, mergedFile - ), - columnConfig + ) ); return mergedRealtime; diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 8ef4369ebff..1f317c221b5 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -70,16 +70,6 @@ import java.util.Random; @RunWith(Parameterized.class) public class SpatialFilterTest { - - private static ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ @@ -246,7 +236,7 @@ public class SpatialFilterTest tmpFile.deleteOnExit(); IndexMerger.persist(theIndex, tmpFile); - return IndexIO.loadIndex(tmpFile, columnConfig); + return IndexIO.loadIndex(tmpFile); } private static QueryableIndex makeMergedQueryableIndex() @@ -426,11 +416,10 @@ public class SpatialFilterTest QueryableIndex mergedRealtime = IndexIO.loadIndex( IndexMerger.mergeQueryableIndex( - Arrays.asList(IndexIO.loadIndex(firstFile, columnConfig), IndexIO.loadIndex(secondFile, columnConfig), IndexIO.loadIndex(thirdFile, columnConfig)), + Arrays.asList(IndexIO.loadIndex(firstFile), IndexIO.loadIndex(secondFile), IndexIO.loadIndex(thirdFile)), METRIC_AGGS, mergedFile - ), - columnConfig + ) ); return mergedRealtime; diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 402aded1f74..0a67b869624 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -33,10 +33,10 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.collections.StupidPool; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Processing; +import io.druid.query.DruidProcessingConfig; import io.druid.query.MetricsEmittingExecutorService; import io.druid.query.PrioritizedExecutorService; -import io.druid.server.DruidProcessingConfig; -import io.druid.server.VMUtils; +import io.druid.query.VMUtils; import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; diff --git a/server/src/main/java/io/druid/guice/StorageNodeModule.java b/server/src/main/java/io/druid/guice/StorageNodeModule.java index 3ce1020154c..bfc8d28a2df 100644 --- a/server/src/main/java/io/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/io/druid/guice/StorageNodeModule.java @@ -33,7 +33,7 @@ import io.druid.segment.loading.MMappedQueryableIndexFactory; import io.druid.segment.loading.QueryableIndexFactory; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.server.DruidNode; -import io.druid.server.DruidProcessingConfig; +import io.druid.query.DruidProcessingConfig; import io.druid.server.coordination.DruidServerMetadata; import javax.annotation.Nullable; diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index ed4674aa5d8..4a5dd8ab20b 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -24,7 +24,6 @@ import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import com.google.inject.Binder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; @@ -37,14 +36,13 @@ import io.druid.curator.discovery.DiscoveryModule; import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; import io.druid.guice.DbConnectorModule; -import io.druid.guice.DruidGuiceExtensions; import io.druid.guice.DruidProcessingModule; import io.druid.guice.DruidSecondaryModule; +import io.druid.guice.ExtensionsConfig; import io.druid.guice.FirehoseModule; import io.druid.guice.HttpClientModule; import io.druid.guice.IndexingServiceDiscoveryModule; import io.druid.guice.JacksonConfigManagerModule; -import io.druid.guice.JsonConfigProvider; import io.druid.guice.LifecycleModule; import io.druid.guice.LocalDataStorageDruidModule; import io.druid.guice.ParsersModule; @@ -56,12 +54,8 @@ import io.druid.guice.StorageNodeModule; import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; -import io.druid.jackson.JacksonModule; -import io.druid.server.initialization.ConfigModule; import io.druid.server.initialization.EmitterModule; -import io.druid.server.initialization.ExtensionsConfig; import io.druid.server.initialization.JettyServerModule; -import io.druid.server.initialization.PropertiesModule; import io.druid.server.metrics.MetricsModule; import io.tesla.aether.Repository; import io.tesla.aether.TeslaAether; @@ -85,6 +79,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.net.URL; import java.net.URLClassLoader; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -354,25 +349,6 @@ public class Initialization return Guice.createInjector(Modules.override(defaultModules.getModules()).with(actualModules.getModules())); } - public static Injector makeStartupInjector() - { - return Guice.createInjector( - new DruidGuiceExtensions(), - new JacksonModule(), - new PropertiesModule("runtime.properties"), - new ConfigModule(), - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(DruidSecondaryModule.class); - JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class); - } - } - ); - } - private static class ModuleList { private final Injector baseInjector; diff --git a/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java index 7adef194779..717b0b513a0 100644 --- a/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/io/druid/segment/loading/MMappedQueryableIndexFactory.java @@ -19,11 +19,9 @@ package io.druid.segment.loading; -import com.google.inject.Inject; import com.metamx.common.logger.Logger; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndex; -import io.druid.segment.column.ColumnConfig; import org.apache.commons.io.FileUtils; import java.io.File; @@ -35,21 +33,11 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory { private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); - private final ColumnConfig columnConfig; - - @Inject - public MMappedQueryableIndexFactory( - ColumnConfig columnConfig - ) - { - this.columnConfig = columnConfig; - } - @Override public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { - return IndexIO.loadIndex(parentDir, columnConfig); + return IndexIO.loadIndex(parentDir); } catch (IOException e) { log.warn(e, "Got exception!!!! Going to delete parentDir[%s]", parentDir); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 0e6a8786df3..4f978626c69 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -63,8 +63,7 @@ public class FlushingPlumber extends RealtimePlumber ServiceEmitter emitter, QueryRunnerFactoryConglomerate conglomerate, DataSegmentAnnouncer segmentAnnouncer, - ExecutorService queryExecutorService, - ColumnConfig columnConfig + ExecutorService queryExecutorService ) { super( @@ -77,8 +76,7 @@ public class FlushingPlumber extends RealtimePlumber queryExecutorService, null, null, - null, - columnConfig + null ); this.flushDuration = flushDuration; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 2c15a7c02f3..46c3d91c96a 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -52,7 +52,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool private final QueryRunnerFactoryConglomerate conglomerate; private final DataSegmentAnnouncer segmentAnnouncer; private final ExecutorService queryExecutorService; - private final ColumnConfig columnConfig; @JsonCreator public FlushingPlumberSchool( @@ -61,7 +60,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JacksonInject QueryRunnerFactoryConglomerate conglomerate, @JacksonInject DataSegmentAnnouncer segmentAnnouncer, @JacksonInject @Processing ExecutorService queryExecutorService, - @JacksonInject ColumnConfig columnConfig, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -79,7 +77,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool null, null, queryExecutorService, - columnConfig, windowPeriod, basePersistDirectory, segmentGranularity, @@ -93,7 +90,6 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool this.conglomerate = conglomerate; this.segmentAnnouncer = segmentAnnouncer; this.queryExecutorService = queryExecutorService; - this.columnConfig = columnConfig; } @Override @@ -113,8 +109,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool emitter, conglomerate, segmentAnnouncer, - queryExecutorService, - columnConfig + queryExecutorService ); } diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 96d835c4bef..306350aaaa3 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -37,8 +37,6 @@ import io.druid.segment.IndexMerger; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; -import io.druid.segment.column.Column; -import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.loading.DataSegmentPusher; @@ -92,7 +90,6 @@ public class RealtimePlumber implements Plumber private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( String.CASE_INSENSITIVE_ORDER ); - private final ColumnConfig columnConfig; private volatile boolean shuttingDown = false; private volatile boolean stopped = false; private volatile ExecutorService persistExecutor = null; @@ -109,8 +106,7 @@ public class RealtimePlumber implements Plumber ExecutorService queryExecutorService, DataSegmentPusher dataSegmentPusher, SegmentPublisher segmentPublisher, - FilteredServerView serverView, - ColumnConfig columnConfig + FilteredServerView serverView ) { this.schema = schema; @@ -124,7 +120,6 @@ public class RealtimePlumber implements Plumber this.dataSegmentPusher = dataSegmentPusher; this.segmentPublisher = segmentPublisher; this.serverView = serverView; - this.columnConfig = columnConfig; log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); } @@ -347,7 +342,7 @@ public class RealtimePlumber implements Plumber mergedTarget ); - QueryableIndex index = IndexIO.loadIndex(mergedFile, columnConfig); + QueryableIndex index = IndexIO.loadIndex(mergedFile); DataSegment segment = dataSegmentPusher.push( mergedFile, @@ -533,7 +528,7 @@ public class RealtimePlumber implements Plumber versioningPolicy.getVersion(sinkInterval), config.getShardSpec() ), - IndexIO.loadIndex(segmentDir, columnConfig) + IndexIO.loadIndex(segmentDir) ), Integer.parseInt(segmentDir.getName()) ) @@ -714,7 +709,7 @@ public class RealtimePlumber implements Plumber indexToPersist.swapSegment( new QueryableIndexSegment( indexToPersist.getSegment().getIdentifier(), - IndexIO.loadIndex(persistedFile, columnConfig) + IndexIO.loadIndex(persistedFile) ) ); diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index c3480a682bd..cd000651709 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -52,7 +52,6 @@ public class RealtimePlumberSchool implements PlumberSchool private final SegmentPublisher segmentPublisher; private final FilteredServerView serverView; private final ExecutorService queryExecutorService; - private final ColumnConfig columnConfig; // Backwards compatible private final Period windowPeriod; @@ -71,7 +70,6 @@ public class RealtimePlumberSchool implements PlumberSchool @JacksonInject SegmentPublisher segmentPublisher, @JacksonInject FilteredServerView serverView, @JacksonInject @Processing ExecutorService executorService, - @JacksonInject ColumnConfig columnConfig, // Backwards compatible @JsonProperty("windowPeriod") Period windowPeriod, @JsonProperty("basePersistDirectory") File basePersistDirectory, @@ -88,7 +86,6 @@ public class RealtimePlumberSchool implements PlumberSchool this.segmentPublisher = segmentPublisher; this.serverView = serverView; this.queryExecutorService = executorService; - this.columnConfig = columnConfig; this.windowPeriod = windowPeriod; this.basePersistDirectory = basePersistDirectory; this.segmentGranularity = segmentGranularity; @@ -153,8 +150,7 @@ public class RealtimePlumberSchool implements PlumberSchool queryExecutorService, dataSegmentPusher, segmentPublisher, - serverView, - columnConfig + serverView ); } diff --git a/server/src/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java index 8e93336d967..cf294087453 100644 --- a/server/src/test/java/io/druid/initialization/InitializationTest.java +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -27,10 +27,10 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; +import io.druid.guice.ExtensionsConfig; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; -import io.druid.server.initialization.ExtensionsConfig; import org.junit.Assert; import org.junit.FixMethodOrder; import org.junit.Test; @@ -58,7 +58,7 @@ public class InitializationTest @Test public void test02MakeStartupInjector() throws Exception { - Injector startupInjector = Initialization.makeStartupInjector(); + Injector startupInjector = GuiceInjectors.makeStartupInjector(); Assert.assertNotNull(startupInjector); Assert.assertNotNull(startupInjector.getInstance(ObjectMapper.class)); } @@ -66,7 +66,7 @@ public class InitializationTest @Test public void test03ClassLoaderExtensionsLoading() { - Injector startupInjector = Initialization.makeStartupInjector(); + Injector startupInjector = GuiceInjectors.makeStartupInjector(); Function fnClassName = new Function() { @@ -99,7 +99,7 @@ public class InitializationTest @Test public void test04MakeInjectorWithModules() throws Exception { - Injector startupInjector = Initialization.makeStartupInjector(); + Injector startupInjector = GuiceInjectors.makeStartupInjector(); Injector injector = Initialization.makeInjectorWithModules( startupInjector, ImmutableList.of( new com.google.inject.Module() diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 0113ee3dbc1..19c104a4ff0 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -73,7 +73,7 @@ public class FireDepartmentTest new RealtimeIOConfig( null, new RealtimePlumberSchool( - null, null, null, null, null, null, null, null, null, null, null, null, null, 0 + null, null, null, null, null, null, null, null, null, null, null, null, 0 ) ), new RealtimeTuningConfig( diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 8f88d093efa..db5ae44be8f 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -65,15 +65,6 @@ import java.util.concurrent.TimeUnit; */ public class RealtimePlumberSchoolTest { - private final ColumnConfig columnConfig = new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 1024 * 1024; - } - }; - private Plumber plumber; private DataSegmentAnnouncer announcer; @@ -153,7 +144,6 @@ public class RealtimePlumberSchoolTest segmentPublisher, serverView, MoreExecutors.sameThreadExecutor(), - columnConfig, new Period("PT10m"), tmpDir, Granularity.HOUR, diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index 02276653245..bbb7a4c6f9d 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -35,6 +35,7 @@ import com.metamx.http.client.response.StatusResponseHolder; import io.druid.guice.Jerseys; import io.druid.guice.LazySingleton; import io.druid.guice.annotations.Global; +import io.druid.initialization.GuiceInjectors; import io.druid.initialization.Initialization; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Handler; @@ -90,7 +91,7 @@ public class JettyTest { setProperties(); Injector injector = Initialization.makeInjectorWithModules( - Initialization.makeStartupInjector(), Lists.newArrayList( + GuiceInjectors.makeStartupInjector(), Lists.newArrayList( new Module() { @Override diff --git a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java index cf81cc7aca5..25d8a28a935 100644 --- a/services/src/main/java/io/druid/cli/CliHadoopIndexer.java +++ b/services/src/main/java/io/druid/cli/CliHadoopIndexer.java @@ -27,7 +27,7 @@ import io.airlift.command.Arguments; import io.airlift.command.Command; import io.airlift.command.Option; import io.druid.initialization.Initialization; -import io.druid.server.initialization.ExtensionsConfig; +import io.druid.guice.ExtensionsConfig; import io.tesla.aether.internal.DefaultTeslaAether; import java.io.File; diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 450aa36afe9..18a1be65134 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -25,13 +25,11 @@ import io.airlift.command.Help; import io.airlift.command.ParseException; import io.druid.cli.convert.ConvertProperties; import io.druid.cli.validate.DruidJsonValidator; +import io.druid.guice.ExtensionsConfig; +import io.druid.initialization.GuiceInjectors; import io.druid.initialization.Initialization; -import io.druid.server.initialization.ExtensionsConfig; -import org.apache.log4j.Level; -import org.apache.log4j.Logger; import java.util.Collection; -import java.util.List; /** */ @@ -75,7 +73,7 @@ public class Main .withDefaultCommand(Help.class) .withCommands(CliPeon.class, CliInternalHadoopIndexer.class); - final Injector injector = Initialization.makeStartupInjector(); + final Injector injector = GuiceInjectors.makeStartupInjector(); final ExtensionsConfig config = injector.getInstance(ExtensionsConfig.class); final Collection extensionCommands = Initialization.getFromExtensions(config, CliCommandCreator.class); diff --git a/services/src/main/java/io/druid/cli/PullDependencies.java b/services/src/main/java/io/druid/cli/PullDependencies.java index c151a86ea3b..f12c167a0d5 100644 --- a/services/src/main/java/io/druid/cli/PullDependencies.java +++ b/services/src/main/java/io/druid/cli/PullDependencies.java @@ -27,7 +27,7 @@ import io.airlift.command.Option; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.HadoopIndexTask; import io.druid.initialization.Initialization; -import io.druid.server.initialization.ExtensionsConfig; +import io.druid.guice.ExtensionsConfig; import io.tesla.aether.internal.DefaultTeslaAether; import java.util.List; From 4c40e71e544cdfd31bc672dacd9d22597d6d50da Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 10:45:54 -0700 Subject: [PATCH 109/270] address cr --- .../src/main/java/io/druid/common/utils}/VMUtils.java | 2 +- .../main/java/io/druid/indexer/HadoopDruidIndexerConfig.java | 2 +- .../java/io/druid/indexing/common/task/HadoopIndexTask.java | 2 +- .../io/druid/{initialization => guice}/GuiceInjectors.java | 2 +- .../java/io/druid/{initialization => guice}/ModuleList.java | 3 ++- processing/src/main/java/io/druid/segment/IndexIO.java | 2 +- server/src/main/java/io/druid/guice/DruidProcessingModule.java | 2 +- .../test/java/io/druid/initialization/InitializationTest.java | 1 + .../test/java/io/druid/server/initialization/JettyTest.java | 2 +- services/src/main/java/io/druid/cli/Main.java | 2 +- 10 files changed, 11 insertions(+), 9 deletions(-) rename {processing/src/main/java/io/druid/query => common/src/main/java/io/druid/common/utils}/VMUtils.java (98%) rename processing/src/main/java/io/druid/{initialization => guice}/GuiceInjectors.java (98%) rename processing/src/main/java/io/druid/{initialization => guice}/ModuleList.java (97%) diff --git a/processing/src/main/java/io/druid/query/VMUtils.java b/common/src/main/java/io/druid/common/utils/VMUtils.java similarity index 98% rename from processing/src/main/java/io/druid/query/VMUtils.java rename to common/src/main/java/io/druid/common/utils/VMUtils.java index db1213cbe7e..60fa4a01f8f 100644 --- a/processing/src/main/java/io/druid/query/VMUtils.java +++ b/common/src/main/java/io/druid/common/utils/VMUtils.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.query; +package io.druid.common.utils; import java.lang.reflect.InvocationTargetException; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index d338506e512..8f0f2d43c34 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -44,7 +44,7 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.path.PathSpec; -import io.druid.initialization.GuiceInjectors; +import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.GranularitySpec; diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java index 9e5a67c9ec5..69e5c667551 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopIndexTask.java @@ -42,7 +42,7 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.LockAcquireAction; import io.druid.indexing.common.actions.LockTryAcquireAction; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.initialization.GuiceInjectors; +import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; import io.druid.guice.ExtensionsConfig; import io.druid.timeline.DataSegment; diff --git a/processing/src/main/java/io/druid/initialization/GuiceInjectors.java b/processing/src/main/java/io/druid/guice/GuiceInjectors.java similarity index 98% rename from processing/src/main/java/io/druid/initialization/GuiceInjectors.java rename to processing/src/main/java/io/druid/guice/GuiceInjectors.java index 74dd122cd55..38b6b2260be 100644 --- a/processing/src/main/java/io/druid/initialization/GuiceInjectors.java +++ b/processing/src/main/java/io/druid/guice/GuiceInjectors.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.initialization; +package io.druid.guice; import com.google.common.collect.Lists; import com.google.inject.Binder; diff --git a/processing/src/main/java/io/druid/initialization/ModuleList.java b/processing/src/main/java/io/druid/guice/ModuleList.java similarity index 97% rename from processing/src/main/java/io/druid/initialization/ModuleList.java rename to processing/src/main/java/io/druid/guice/ModuleList.java index 113e61a5982..56ccc85e7cd 100644 --- a/processing/src/main/java/io/druid/initialization/ModuleList.java +++ b/processing/src/main/java/io/druid/guice/ModuleList.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.initialization; +package io.druid.guice; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; @@ -27,6 +27,7 @@ import com.google.inject.Module; import com.metamx.common.ISE; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; +import io.druid.initialization.DruidModule; import java.util.Collections; import java.util.List; diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 20624b60ed9..3c32cb346cb 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -46,7 +46,7 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import io.druid.common.utils.SerializerUtils; import io.druid.guice.ConfigProvider; -import io.druid.initialization.GuiceInjectors; +import io.druid.guice.GuiceInjectors; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.DruidProcessingConfig; import io.druid.segment.column.Column; diff --git a/server/src/main/java/io/druid/guice/DruidProcessingModule.java b/server/src/main/java/io/druid/guice/DruidProcessingModule.java index 0a67b869624..8613e0dc4c4 100644 --- a/server/src/main/java/io/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/io/druid/guice/DruidProcessingModule.java @@ -36,7 +36,7 @@ import io.druid.guice.annotations.Processing; import io.druid.query.DruidProcessingConfig; import io.druid.query.MetricsEmittingExecutorService; import io.druid.query.PrioritizedExecutorService; -import io.druid.query.VMUtils; +import io.druid.common.utils.VMUtils; import java.nio.ByteBuffer; import java.util.concurrent.ExecutorService; diff --git a/server/src/test/java/io/druid/initialization/InitializationTest.java b/server/src/test/java/io/druid/initialization/InitializationTest.java index cf294087453..f4bcc3708d2 100644 --- a/server/src/test/java/io/druid/initialization/InitializationTest.java +++ b/server/src/test/java/io/druid/initialization/InitializationTest.java @@ -28,6 +28,7 @@ import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; import io.druid.guice.ExtensionsConfig; +import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; import io.druid.server.DruidNode; diff --git a/server/src/test/java/io/druid/server/initialization/JettyTest.java b/server/src/test/java/io/druid/server/initialization/JettyTest.java index bbb7a4c6f9d..3d03bf8a22a 100644 --- a/server/src/test/java/io/druid/server/initialization/JettyTest.java +++ b/server/src/test/java/io/druid/server/initialization/JettyTest.java @@ -35,7 +35,7 @@ import com.metamx.http.client.response.StatusResponseHolder; import io.druid.guice.Jerseys; import io.druid.guice.LazySingleton; import io.druid.guice.annotations.Global; -import io.druid.initialization.GuiceInjectors; +import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; import org.apache.commons.io.IOUtils; import org.eclipse.jetty.server.Handler; diff --git a/services/src/main/java/io/druid/cli/Main.java b/services/src/main/java/io/druid/cli/Main.java index 18a1be65134..0e23a0e81e7 100644 --- a/services/src/main/java/io/druid/cli/Main.java +++ b/services/src/main/java/io/druid/cli/Main.java @@ -26,7 +26,7 @@ import io.airlift.command.ParseException; import io.druid.cli.convert.ConvertProperties; import io.druid.cli.validate.DruidJsonValidator; import io.druid.guice.ExtensionsConfig; -import io.druid.initialization.GuiceInjectors; +import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; import java.util.Collection; From 0602a45200efabe2211da796ae03a21a97232067 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 19 Jun 2014 13:35:00 -0700 Subject: [PATCH 110/270] add column config test (disabled by default) --- .../main/java/io/druid/segment/IndexIO.java | 5 +-- .../java/io/druid/segment/IndexIOTest.java | 34 +++++++++++++++++++ 2 files changed, 37 insertions(+), 2 deletions(-) create mode 100644 processing/src/test/java/io/druid/segment/IndexIOTest.java diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 3c32cb346cb..b532c3c5729 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -120,7 +120,7 @@ public class IndexIO private static final SerializerUtils serializerUtils = new SerializerUtils(); private static final ObjectMapper mapper; - private static final ColumnConfig columnConfig; + protected static final ColumnConfig columnConfig; static { final Injector injector = GuiceInjectors.makeStartupInjectorWithModules( @@ -135,12 +135,13 @@ public class IndexIO DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing") ); + binder.bind(ColumnConfig.class).to(DruidProcessingConfig.class); } } ) ); mapper = injector.getInstance(ObjectMapper.class); - columnConfig = injector.getInstance(DruidProcessingConfig.class); + columnConfig = injector.getInstance(ColumnConfig.class); } private static volatile IndexIOHandler handler = null; diff --git a/processing/src/test/java/io/druid/segment/IndexIOTest.java b/processing/src/test/java/io/druid/segment/IndexIOTest.java new file mode 100644 index 00000000000..dd7cb2ef21c --- /dev/null +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -0,0 +1,34 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.segment; + +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +public class IndexIOTest +{ + @Test @Ignore // this test depends on static fields, so it has to be tested independently + public void testInjector() throws Exception + { + System.setProperty("druid.processing.columnCache.sizeBytes", "1234"); + Assert.assertEquals(1234, IndexIO.columnConfig.columnCacheSizeBytes()); + } +} From b313601a3b603527dbe43f62226de18d1f1c2229 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 16:32:38 -0700 Subject: [PATCH 111/270] add an optional delay for how long it takes to drop a segment, fixes all manners of inconsistent issues --- .../druid/client/CachingClusteredClient.java | 2 +- .../segment/loading/SegmentLoaderConfig.java | 15 +++++++-- .../server/coordination/ZkCoordinator.java | 32 ++++++++++++++++--- .../coordination/ZkCoordinatorTest.java | 5 ++- 4 files changed, 45 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index cf5f09228f6..fe38811f332 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -228,7 +228,7 @@ public class CachingClusteredClient implements QueryRunner final QueryableDruidServer queryableDruidServer = segment.lhs.pick(); if (queryableDruidServer == null) { - log.error("No servers found for %s?! How can this be?!", segment.rhs); + log.makeAlert("No servers found for %s?! How can this be?!", segment.rhs).emit(); } else { final DruidServer server = queryableDruidServer.getServer(); List descriptors = serverSegments.get(server); diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index a1339790a20..78fc18fb92d 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -37,6 +37,9 @@ public class SegmentLoaderConfig @JsonProperty("deleteOnRemove") private boolean deleteOnRemove = true; + @JsonProperty("dropSegmentDelayMillis") + private int dropSegmentDelayMillis = 0; + @JsonProperty private File infoDir = null; @@ -50,6 +53,11 @@ public class SegmentLoaderConfig return deleteOnRemove; } + public int getDropSegmentDelayMillis() + { + return dropSegmentDelayMillis; + } + public File getInfoDir() { if (infoDir == null) { @@ -72,9 +80,10 @@ public class SegmentLoaderConfig public String toString() { return "SegmentLoaderConfig{" + - "locations=" + getLocations() + - ", deleteOnRemove=" + isDeleteOnRemove() + - ", infoDir=" + getInfoDir() + + "locations=" + locations + + ", deleteOnRemove=" + deleteOnRemove + + ", dropSegmentDelayMillis=" + dropSegmentDelayMillis + + ", infoDir=" + infoDir + '}'; } } diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index 28edb8ddca2..b2ab3ec1151 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -20,8 +20,10 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.inject.Inject; +import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.emitter.EmittingLogger; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoadingException; @@ -32,6 +34,8 @@ import org.apache.curator.framework.CuratorFramework; import java.io.File; import java.io.IOException; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; /** */ @@ -43,6 +47,7 @@ public class ZkCoordinator extends BaseZkCoordinator private final SegmentLoaderConfig config; private final DataSegmentAnnouncer announcer; private final ServerManager serverManager; + private final ScheduledExecutorService exec; @Inject public ZkCoordinator( @@ -52,7 +57,8 @@ public class ZkCoordinator extends BaseZkCoordinator DruidServerMetadata me, DataSegmentAnnouncer announcer, CuratorFramework curator, - ServerManager serverManager + ServerManager serverManager, + ScheduledExecutorFactory factory ) { super(jsonMapper, zkPaths, me, curator); @@ -61,6 +67,8 @@ public class ZkCoordinator extends BaseZkCoordinator this.config = config; this.announcer = announcer; this.serverManager = serverManager; + + this.exec = factory.create(1, "ZkCoordinator-Exec--%d"); } @Override @@ -225,17 +233,33 @@ public class ZkCoordinator extends BaseZkCoordinator @Override - public void removeSegment(DataSegment segment, DataSegmentChangeCallback callback) + public void removeSegment(final DataSegment segment, DataSegmentChangeCallback callback) { try { - serverManager.dropSegment(segment); - File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } announcer.unannounceSegment(segment); + + exec.schedule( + new Runnable() + { + @Override + public void run() + { + try { + serverManager.dropSegment(segment); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + }, + config.getDropSegmentDelayMillis(), + TimeUnit.MILLISECONDS + ); } catch (Exception e) { log.makeAlert(e, "Failed to remove segment") diff --git a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java index d0a7b62f4be..f50ae3e16fc 100644 --- a/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordination/ZkCoordinatorTest.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.concurrent.ScheduledExecutors; +import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.LocalCacheProvider; @@ -117,7 +119,8 @@ public class ZkCoordinatorTest extends CuratorTestBase me, announcer, curator, - serverManager + serverManager, + ScheduledExecutors.createFactory(new Lifecycle()) ); } From 2aab2a0cd9a2afad78f10cd0c1740629e589c9a5 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 16:48:43 -0700 Subject: [PATCH 112/270] address cr --- .../java/io/druid/segment/loading/SegmentLoaderConfig.java | 2 +- .../java/io/druid/server/coordination/ZkCoordinator.java | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index 78fc18fb92d..cefb275e946 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -38,7 +38,7 @@ public class SegmentLoaderConfig private boolean deleteOnRemove = true; @JsonProperty("dropSegmentDelayMillis") - private int dropSegmentDelayMillis = 0; + private int dropSegmentDelayMillis = 5 * 60 * 1000; // 5 mins @JsonProperty private File infoDir = null; diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index b2ab3ec1151..f2d3a2fd854 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -233,7 +233,7 @@ public class ZkCoordinator extends BaseZkCoordinator @Override - public void removeSegment(final DataSegment segment, DataSegmentChangeCallback callback) + public void removeSegment(final DataSegment segment, final DataSegmentChangeCallback callback) { try { File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); @@ -243,6 +243,7 @@ public class ZkCoordinator extends BaseZkCoordinator announcer.unannounceSegment(segment); + log.info("Completely removing [%s] in [%,d] millis", segment.getIdentifier(), config.getDropSegmentDelayMillis()); exec.schedule( new Runnable() { @@ -253,6 +254,9 @@ public class ZkCoordinator extends BaseZkCoordinator serverManager.dropSegment(segment); } catch (Exception e) { + log.makeAlert(e, "Failed to remove segment! Possible resource leak!") + .addData("segment", segment) + .emit(); throw Throwables.propagate(e); } } From fc36bfbc6db5b2e6a42dd43a585e9505871277a5 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 16:49:50 -0700 Subject: [PATCH 113/270] remove pointless exception --- .../main/java/io/druid/server/coordination/ZkCoordinator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index f2d3a2fd854..7a91d542b96 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -257,7 +257,6 @@ public class ZkCoordinator extends BaseZkCoordinator log.makeAlert(e, "Failed to remove segment! Possible resource leak!") .addData("segment", segment) .emit(); - throw Throwables.propagate(e); } } }, From 5afcb9cbd3a3efc287f430f6c405ce024427da8d Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 19 Jun 2014 16:55:02 -0700 Subject: [PATCH 114/270] add docs --- docs/content/Configuration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/Configuration.md b/docs/content/Configuration.md index 35269241c72..264efb91ecd 100644 --- a/docs/content/Configuration.md +++ b/docs/content/Configuration.md @@ -154,6 +154,7 @@ Druid storage nodes maintain information about segments they have already downlo |--------|-----------|-------| |`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. | none (no caching) | |`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true| +|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|5 minutes| |`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| ### Jetty Server Module From a4003966b23c5c1f6b0f48e6b4f23ab68d290118 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 20 Jun 2014 21:45:30 +0530 Subject: [PATCH 115/270] push sinks after bootstrap. fix #570 --- .../realtime/plumber/RealtimePlumber.java | 69 +++++++++++-------- 1 file changed, 40 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index 306350aaaa3..4f3b3c17587 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -152,6 +152,8 @@ public class RealtimePlumber implements Plumber bootstrapSinksFromDisk(); registerServerViewCallback(); startPersistThread(); + // Push pending sinks bootstrapped from previous run + mergeAndPush(); } @Override @@ -595,35 +597,7 @@ public class RealtimePlumber implements Plumber return ScheduledExecutors.Signal.STOP; } - log.info("Starting merge and push."); - - DateTime minTimestampAsDate = segmentGranularity.truncate( - rejectionPolicy.getCurrMaxTime().minus(windowMillis) - ); - long minTimestamp = minTimestampAsDate.getMillis(); - - log.info("Found [%,d] sinks. minTimestamp [%s]", sinks.size(), minTimestampAsDate); - - List> sinksToPush = Lists.newArrayList(); - for (Map.Entry entry : sinks.entrySet()) { - final Long intervalStart = entry.getKey(); - if (intervalStart < minTimestamp) { - log.info("Adding entry[%s] for merge and push.", entry); - sinksToPush.add(entry); - } else { - log.warn( - "[%s] < [%s] Skipping persist and merge.", - new DateTime(intervalStart), - minTimestampAsDate - ); - } - } - - log.info("Found [%,d] sinks to persist and merge", sinksToPush.size()); - - for (final Map.Entry entry : sinksToPush) { - persistAndMerge(entry.getKey(), entry.getValue()); - } + mergeAndPush(); if (stopped) { log.info("Stopping merge-n-push overseer thread"); @@ -636,6 +610,43 @@ public class RealtimePlumber implements Plumber ); } + private void mergeAndPush() + { + final Granularity segmentGranularity = schema.getGranularitySpec().getSegmentGranularity(); + final Period windowPeriod = config.getWindowPeriod(); + + final long windowMillis = windowPeriod.toStandardDuration().getMillis(); + log.info("Starting merge and push."); + + DateTime minTimestampAsDate = segmentGranularity.truncate( + rejectionPolicy.getCurrMaxTime().minus(windowMillis) + ); + long minTimestamp = minTimestampAsDate.getMillis(); + + log.info("Found [%,d] sinks. minTimestamp [%s]", sinks.size(), minTimestampAsDate); + + List> sinksToPush = Lists.newArrayList(); + for (Map.Entry entry : sinks.entrySet()) { + final Long intervalStart = entry.getKey(); + if (intervalStart < minTimestamp) { + log.info("Adding entry[%s] for merge and push.", entry); + sinksToPush.add(entry); + } else { + log.warn( + "[%s] < [%s] Skipping persist and merge.", + new DateTime(intervalStart), + minTimestampAsDate + ); + } + } + + log.info("Found [%,d] sinks to persist and merge", sinksToPush.size()); + + for (final Map.Entry entry : sinksToPush) { + persistAndMerge(entry.getKey(), entry.getValue()); + } + } + /** * Unannounces a given sink and removes all local references to it. */ From b978eb33f1444f72e2206aef1da6a0602aca084f Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 20 Jun 2014 10:42:03 -0700 Subject: [PATCH 116/270] minor fix for cleanup of historical cache file --- .../io/druid/server/coordination/ZkCoordinator.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java index 7a91d542b96..b0611077aa3 100644 --- a/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/ZkCoordinator.java @@ -20,7 +20,6 @@ package io.druid.server.coordination; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.inject.Inject; import com.metamx.common.concurrent.ScheduledExecutorFactory; @@ -236,11 +235,6 @@ public class ZkCoordinator extends BaseZkCoordinator public void removeSegment(final DataSegment segment, final DataSegmentChangeCallback callback) { try { - File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); - if (!segmentInfoCacheFile.delete()) { - log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); - } - announcer.unannounceSegment(segment); log.info("Completely removing [%s] in [%,d] millis", segment.getIdentifier(), config.getDropSegmentDelayMillis()); @@ -252,6 +246,11 @@ public class ZkCoordinator extends BaseZkCoordinator { try { serverManager.dropSegment(segment); + + File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); + if (!segmentInfoCacheFile.delete()) { + log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); + } } catch (Exception e) { log.makeAlert(e, "Failed to remove segment! Possible resource leak!") From 262d43ae2e20e65a9666fcce1f0a567b397c0aaf Mon Sep 17 00:00:00 2001 From: Igal Levy Date: Fri, 20 Jun 2014 17:03:21 -0700 Subject: [PATCH 117/270] removed build section that didn't work (direct from whirr repo); added info on resource files needed to run test; tweaked whirr config section to match current requirements; changed cluster-start section to reflect fact that we're starting multiple instances; other minor edits --- docs/content/Booting-a-production-cluster.md | 39 ++++++++++---------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 77c22ba5f63..235df64a2ad 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -17,10 +17,7 @@ You can provision individual servers, loading Druid onto each machine (or buildi You'll need an AWS account, S3 Bucket and an EC2 key pair from that account so that Whirr can connect to the cloud via the EC2 API. If you haven't generated a key pair, see the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html) or see this [Whirr FAQ](http://whirr.apache.org/faq.html#how-do-i-find-my-cloud-credentials). -### Installing Whirr -You must use a version of Whirr that includes and supports a Druid recipe. You can do it so in one of two ways: - -#### Build the Following Version of Whirr +### Install Whirr Clone the code from [https://github.com/druid-io/whirr](https://github.com/druid-io/whirr) and build Whirr: git clone git@github.com:druid-io/whirr.git @@ -28,23 +25,23 @@ Clone the code from [https://github.com/druid-io/whirr](https://github.com/druid git checkout trunk mvn clean install -Dmaven.test.failure.ignore=true -#### Build the Latest Version of Whirr -Clone the code from the Whirr repository: +In order to run the test below, you'll also need two files that available only from a [standard install of Druid](http://druid.io/downloads.html) or the [Druid repo](https://github.com/metamx/druid/tree/master/examples/bin/examples): + +* `druid/examples/bin/examples/wikipedia/wikipedia_realtime.spec` +* `druid/examples/bin/examples/indexing/wikipedia_realtime_task.json` - git clone git://git.apache.org/whirr.git - -Then run `mvn install` from the root directory. ### Configure Whirr -The Whirr recipe for Druid is the configuration file `$WHIRR_HOME/recipies/druid.properties`. You can edit this file to suit your needs -- it is annotated and self-explanatory. Here are some hints about that file: +The Whirr recipe for Druid is the configuration file `$WHIRR_HOME/recipies/druid.properties`. You can edit this file to suit your needs; it is annotated and self-explanatory. Here are some hints about that file: -* Set `whirr.location-id` to a specific AWS region (e.g., us-east-1) if desired, else one will be chosen for you. +* Set `whirr.location-id` to a specific AWS region if desired. If this is left blank, a region is chosen for you. The default value is `us-east-1`. * You can choose the hardware used with `whirr.hardware-id` to a specific instance type (e.g., m1.large). By default druid.properties, m3.2xlarge (broker, historical, middle manager), m1.xlarge (coordinator, overlord), and m1.small (zookeeper, mysql) are used. * If you don't choose an image via `whirr.image-id` (image must be compatible with hardware), you'll get plain vanilla Linux. Default druid.properties uses ami-018c9568 (Ubuntu 12.04). * SSH keys (not password protected) must exist for the local user. If they are in the default locations, `${sys:user.home}/.ssh/id_rsa` and `${sys:user.home}/.ssh/id_rsa.pub`, Whirr will find them. Otherwise, you'll have to specify them with `whirr.private-key-file` and `whirr.public-key-file`. -* Be sure to specify the absolute path of the Druid realtime spec file `realtime.spec` in `whirr.druid.realtime.spec.path`. -* Also make sure to specify the correct S3 bucket. Otherwise the cluster won't be able to process tasks. * Two Druid cluster templates (see `whirr.instance-templates`) are provided: a small cluster running on a single EC2 instance, and a larger cluster running on multiple instances. +* You must specify the path to an S3 bucket. Otherwise the cluster won't be able to process tasks. +* To successfully submit the test task below, you'll need to specify the location of the `wikipedia_realtime.spec` in the property `whirr.druid.realtime.spec.path`. +* Specify Druid version only if [Druid extenions](Modules.html) are being used. The following AWS information must be set in `druid.properties`, as environment variables, or in the file `$WHIRR_HOME/conf/credentials`: @@ -54,7 +51,7 @@ The following AWS information must be set in `druid.properties`, as environment How to get the IDENTITY and CREDENTIAL keys is discussed above. -In order to configure each node, you can edit `services/druid/src/main/resources/functions/start_druid.sh` for JVM configuration and `services/druid/src/main/resources/functions/configure_[NODE_NAME].sh` for specific node configuration. For more information on configuration, read the Druid documentations about it (http://druid.io/docs/0.6.116/Configuration.html). +In order to configure each node, you can edit `services/druid/src/main/resources/functions/start_druid.sh` for JVM configuration and `services/druid/src/main/resources/functions/configure_[NODE_NAME].sh` for specific node configuration. For more information on configuration, see the [Druid configuration documentation](Configuration.html). ### Start a Test Cluster With Whirr Run the following command: @@ -66,14 +63,14 @@ If Whirr starts without any errors, you should see the following message: Running on provider aws-ec2 using identity -You can then use the EC2 dashboard to locate the instance and confirm that it has started up. +You can then use the EC2 dashboard to locate the instances and confirm that they have started up. -If both the instance and the Druid cluster launch successfully, a few minutes later other messages to STDOUT should follow with information returned from EC2, including the instance ID: +If both the instances and the Druid cluster launch successfully, a few minutes later other messages to STDOUT should follow with information returned from EC2, including the instance ID: - Started cluster of 1 instances + Started cluster of 8 instances Cluster{instances=[Instance{roles=[zookeeper, druid-mysql, druid-coordinator, druid-broker, druid-historical, druid-realtime], publicIp= ... -The final message will contain login information for the instance. +The final message will contain login information for the instances. Note that Whirr will return an exception if any of the nodes fail to launch, and the cluster will be destroyed. To destroy the cluster manually, run the following command: @@ -85,9 +82,11 @@ Note that Whirr will return an exception if any of the nodes fail to launch, and Now you can run an indexing task and a simple query to see if all the nodes have launched correctly. We are going to use a Wikipedia example again. For a realtime indexing task, run the following command: ```bash -curl -X 'POST' -H 'Content-Type:application/json' -d @#{YOUR_DRUID_DIRECTORY}/examples/indexing/wikipedia_realtime_task.json #{OVERLORD_PUBLIC_IP_ADDR}:#{PORT}/druid/indexer/v1/task +curl -X 'POST' -H 'Content-Type:application/json' -d @#{PATH_TO}/wikipedia_realtime_task.json #{OVERLORD_PUBLIC_IP_ADDR}:#{PORT}/druid/indexer/v1/task ``` -Issuing the request should return a task ID. +where OVERLORD_PUBLIC_IP_ADDR should be available from the EC2 information logged to STDOUT, the Overlord port is 8080 by default, and `wikipedia_realtime_task.json` is discussed above. + +Issuing this request should return a task ID. To check the state of the overlord, open up your browser and go to `#{OVERLORD_PUBLIC_IP_ADDR}:#{PORT}/console.html`. From 261d6a16b2525db2ba614d7f50817f81bc5578c1 Mon Sep 17 00:00:00 2001 From: realfun Date: Fri, 20 Jun 2014 23:55:05 -0700 Subject: [PATCH 118/270] Add HdfsTaskLogs --- .../storage/hdfs/HdfsStorageDruidModule.java | 6 + .../storage/hdfs/tasklog/HdfsTaskLogs.java | 104 ++++++++++++++++++ .../hdfs/tasklog/HdfsTaskLogsConfig.java | 41 +++++++ 3 files changed, 151 insertions(+) create mode 100644 hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java create mode 100644 hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java index 0ebb477a593..10e86973f22 100644 --- a/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/HdfsStorageDruidModule.java @@ -27,6 +27,8 @@ import io.druid.guice.Binders; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.initialization.DruidModule; +import io.druid.storage.hdfs.tasklog.HdfsTaskLogs; +import io.druid.storage.hdfs.tasklog.HdfsTaskLogsConfig; import org.apache.hadoop.conf.Configuration; import java.util.List; @@ -68,5 +70,9 @@ public class HdfsStorageDruidModule implements DruidModule binder.bind(Configuration.class).toInstance(conf); JsonConfigProvider.bind(binder, "druid.storage", HdfsDataSegmentPusherConfig.class); + + Binders.taskLogsBinder(binder).addBinding("hdfs").to(HdfsTaskLogs.class); + JsonConfigProvider.bind(binder, "druid.indexer.logs", HdfsTaskLogsConfig.class); + binder.bind(HdfsTaskLogs.class).in(LazySingleton.class); } } diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java new file mode 100644 index 00000000000..b5db10a2a80 --- /dev/null +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogs.java @@ -0,0 +1,104 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.hdfs.tasklog; + +import com.google.common.base.Optional; +import com.google.common.io.ByteStreams; +import com.google.common.io.InputSupplier; +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import io.druid.tasklogs.TaskLogs; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; + +/** + * Indexer hdfs task logs, to support storing hdfs tasks to hdfs + * + * Created by Frank Ren on 6/20/14. + */ +public class HdfsTaskLogs implements TaskLogs +{ + private static final Logger log = new Logger(HdfsTaskLogs.class); + + private final HdfsTaskLogsConfig config; + + @Inject + public HdfsTaskLogs(HdfsTaskLogsConfig config) + { + this.config = config; + } + + @Override + public void pushTaskLog(String taskId, File logFile) throws IOException + { + final Path path = getTaskLogFileFromId(taskId); + log.info("writing task log to: %s", path); + Configuration conf = new Configuration(); + final FileSystem fs = FileSystem.get(conf); + FileUtil.copy(logFile, fs, path, false, conf); + log.info("wrote task log to: %s", path); + } + + @Override + public Optional> streamTaskLog(final String taskId, final long offset) throws IOException + { + final Path path = getTaskLogFileFromId(taskId); + final FileSystem fs = FileSystem.get(new Configuration()); + if (fs.exists(path)) { + return Optional.>of( + new InputSupplier() { + @Override + public InputStream getInput() throws IOException + { + log.info("reading task log from: %s", path); + final InputStream inputStream = fs.open(path); + ByteStreams.skipFully(inputStream, offset); + log.info("read task log from: %s", path); + return inputStream; + } + } + ); + } else { + return Optional.absent(); + } + } + + /** + * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in + * path names. So we format paths differently for HDFS. + */ + private Path getTaskLogFileFromId(String taskId) + { + return new Path(mergePaths(config.getDirectory(), taskId.replaceAll(":", "_"))); + } + + // some hadoop version Path.mergePaths does not exist + private static String mergePaths(String path1, String path2) + { + return path1 + (path1.endsWith(Path.SEPARATOR) ? "" : Path.SEPARATOR) + path2; + } +} + + diff --git a/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java b/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java new file mode 100644 index 00000000000..447dff7c52a --- /dev/null +++ b/hdfs-storage/src/main/java/io/druid/storage/hdfs/tasklog/HdfsTaskLogsConfig.java @@ -0,0 +1,41 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.hdfs.tasklog; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.validation.constraints.NotNull; + +/** + * Indexer hdfs task logs configuration + * + * Created by Frank Ren on 6/20/14. + */ +public class HdfsTaskLogsConfig +{ + @JsonProperty + @NotNull + private String directory; + + public String getDirectory() + { + return directory; + } +} + From 1499ee6458e83158a2c4874f1db666e0ae2d2778 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 17 Jun 2014 13:35:12 -0700 Subject: [PATCH 119/270] add toString method for JavascriptDimExtractionFn --- .../druid/query/extraction/JavascriptDimExtractionFn.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java b/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java index d61d3b26b41..3935e07c293 100644 --- a/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java +++ b/processing/src/main/java/io/druid/query/extraction/JavascriptDimExtractionFn.java @@ -98,4 +98,12 @@ public class JavascriptDimExtractionFn implements DimExtractionFn { return false; } + + @Override + public String toString() + { + return "JavascriptDimExtractionFn{" + + "function='" + function + '\'' + + '}'; + } } From a7ef019b222bbee9114ffe6612bf74ad1d326a47 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 25 Jun 2014 20:14:11 -0700 Subject: [PATCH 120/270] Document batch ingestion jobProperties. --- docs/content/Batch-ingestion.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index 861bc540fb3..c7866327206 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -109,6 +109,7 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim |leaveIntermediate|leave behind files in the workingPath when job completes or fails (debugging tool).|no| |partitionsSpec|a specification of how to partition each time bucket into segments, absence of this property means no partitioning will occur.|no| |metadataUpdateSpec|a specification of how to update the metadata for the druid cluster these segments belong to.|yes| +|jobProperties|a map of properties to add to the Hadoop job configuration.|no| ### Path specification From 991ff34f38f66128cac05d0920ce2484e43cbd29 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Wed, 25 Jun 2014 20:19:03 -0700 Subject: [PATCH 121/270] More explicit example of jobProperties. --- docs/content/Batch-ingestion.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index c7866327206..057927e5596 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -89,6 +89,9 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim "user": "username", "password": "passmeup", "segmentTable": "segments" + }, + "jobProperties": { + "mapreduce.job.queuename": "default" } } ``` From afa73a2d3a913de88ce09cfe68e1e86ad9d3d172 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 30 Jun 2014 17:29:19 -0700 Subject: [PATCH 122/270] fix one of the TODOs and add test case --- .../druid/query/search/SearchQueryRunner.java | 12 +------- .../java/io/druid/segment/IndexMerger.java | 6 ---- .../java/io/druid/segment/AppendTest.java | 30 +++++++++++++------ 3 files changed, 22 insertions(+), 26 deletions(-) diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java index 53fe74e104f..e74729a0f60 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -19,7 +19,6 @@ package io.druid.query.search; -import com.google.common.base.Function; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -49,10 +48,8 @@ import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; -import it.uniroma3.mat.extendedset.intset.ConciseSet; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; import java.util.TreeSet; @@ -97,14 +94,7 @@ public class SearchQueryRunner implements QueryRunner> final ImmutableConciseSet baseFilter; if (filter == null) { - // Accept all, and work around https://github.com/metamx/extendedset/issues/1 - if (index.getNumRows() == 1) { - ConciseSet set = new ConciseSet(); - set.add(0); - baseFilter = ImmutableConciseSet.newImmutableFromMutable(set); - } else { - baseFilter = ImmutableConciseSet.complement(new ImmutableConciseSet(), index.getNumRows()); - } + baseFilter = ImmutableConciseSet.complement(new ImmutableConciseSet(), index.getNumRows()); } else { baseFilter = filter.goConcise(new ColumnSelectorBitmapIndexSelector(index)); } diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 672cec724b5..3ca90d0d54b 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -322,12 +322,6 @@ public class IndexMerger throw new ISE("Couldn't make outdir[%s].", outDir); } -/* - if (indexes.size() < 2) { - throw new ISE("Too few indexes provided for append [%d].", indexes.size()); - } -*/ - final List mergedDimensions = mergeIndexed( Lists.transform( indexes, diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index b98d3266df7..13a37ff2e1a 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -51,10 +51,6 @@ import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryBuilder; import io.druid.query.topn.TopNResultValue; -import io.druid.segment.QueryableIndex; -import io.druid.segment.QueryableIndexSegment; -import io.druid.segment.Segment; -import io.druid.segment.TestHelper; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Before; @@ -459,12 +455,28 @@ public class AppendTest SearchQuery query = makeSearchQuery(); QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment); - QueryRunner runner2 = TestQueryRunners.makeSearchQueryRunner(segment2); TestHelper.assertExpectedResults(expectedResults, runner.run(query)); -/* TODO: this is broken. Actually, it's a bug in the appending code that keeps dimension values around that don't - TODO: exist anymore. The bug should be fixed and this uncommented - TestHelper.assertExpectedResults(expectedResults, runner2.run(query)); -*/ + } + + @Test + public void testSearchWithOverlap() + { + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new SearchResultValue( + Arrays.asList( + new SearchHit(placementishDimension, "a"), + new SearchHit(placementDimension, "mezzanine"), + new SearchHit(providerDimension, "total_market") + ) + ) + ) + ); + + SearchQuery query = makeSearchQuery(); + QueryRunner runner = TestQueryRunners.makeSearchQueryRunner(segment2); + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); } @Test From 70ae6e1fcd48c09d6242da41a410215c46425c49 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 27 Jun 2014 16:51:01 -0700 Subject: [PATCH 123/270] Approximate Histogram Module --- docs/content/Aggregations.md | 27 +- docs/content/Post-aggregations.md | 72 +- histogram/pom.xml | 71 + .../histogram/ApproximateHistogram.java | 1680 +++++++++++++++++ .../ApproximateHistogramAggregator.java | 103 + ...ApproximateHistogramAggregatorFactory.java | 253 +++ .../ApproximateHistogramBufferAggregator.java | 95 + .../ApproximateHistogramDruidModule.java | 60 + ...ApproximateHistogramFoldingAggregator.java | 101 + ...mateHistogramFoldingAggregatorFactory.java | 164 ++ ...imateHistogramFoldingBufferAggregator.java | 99 + .../ApproximateHistogramFoldingSerde.java | 133 ++ .../ApproximateHistogramPostAggregator.java | 68 + .../aggregation/histogram/ArrayUtils.java | 58 + .../histogram/BucketsPostAggregator.java | 91 + .../aggregation/histogram/BufferUtils.java | 68 + .../CustomBucketsPostAggregator.java | 77 + .../histogram/EqualBucketsPostAggregator.java | 80 + .../aggregation/histogram/Histogram.java | 88 + .../histogram/MaxPostAggregator.java | 81 + .../histogram/MinPostAggregator.java | 81 + .../histogram/QuantilePostAggregator.java | 96 + .../aggregation/histogram/Quantiles.java | 111 ++ .../histogram/QuantilesPostAggregator.java | 92 + .../io.druid.initialization.DruidModule | 1 + .../ApproximateHistogramAggregatorTest.java | 76 + .../ApproximateHistogramErrorBenchmark.java | 191 ++ ...pproximateHistogramPostAggregatorTest.java | 65 + .../ApproximateHistogramQueryTest.java | 247 +++ .../histogram/ApproximateHistogramTest.java | 588 ++++++ .../aggregation/histogram/QuantilesTest.java | 81 + indexing-hadoop/README | 0 pom.xml | 1 + 33 files changed, 5094 insertions(+), 5 deletions(-) create mode 100644 histogram/pom.xml create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/ArrayUtils.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/BucketsPostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/BufferUtils.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/MaxPostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/MinPostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/Quantiles.java create mode 100644 histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilesPostAggregator.java create mode 100644 histogram/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java create mode 100644 histogram/src/test/java/io/druid/query/aggregation/histogram/QuantilesTest.java delete mode 100644 indexing-hadoop/README diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 1c28e1dc10f..cdf33ed2b9b 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -155,8 +155,33 @@ Determine the number of distinct are assigned to. ### HyperUnique aggregator -Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension that has been aggregated as a hyperUnique metric at indexing time. +Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to compute the estimated cardinality of a dimension that has been aggregated as a "hyperUnique" metric at indexing time. ```json { "type" : "hyperUnique", "name" : , "fieldName" : } ``` + +### ApproxHistogram aggregator + +This aggregator is based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) to compute approximate histograms. + +To use this feature, an "approxHistogram" aggregator must be included at indexing time. The ingestion aggregator can only apply to numeric values. To query for results, an "approxHistogramFold" aggregator must be included in the query. + +```json +{ + "type" : "approxHistogram(ingestion), approxHistogramFold(query)", + "name" : , + "fieldName" : , + "resolution" : , + "numBuckets" : , + "lowerLimit" : , + "upperLimit" : +} +``` + +|Property|Description|Default| +|--------|-----------|-------| +|`resolution`|Number of centroids (data points) to store. The higher the resolution, the more accurate results are, but the slower computation will be.|50| +|`numBuckets`|Number of output buckets for the resulting histogram.|7| +|`lowerLimit`/`upperLimit`|Restrict the approximation to the given range. The values outside this range will be aggregated into two centroids. Counts of values outside this range are still maintained. |-INF/+INF| + diff --git a/docs/content/Post-aggregations.md b/docs/content/Post-aggregations.md index 4dce46ceff1..e4ce2211f67 100644 --- a/docs/content/Post-aggregations.md +++ b/docs/content/Post-aggregations.md @@ -64,7 +64,7 @@ Example JavaScript aggregator: "function": "function(delta, total) { return 100 * Math.abs(delta) / total; }" } ``` -### `hyperUniqueCardinality` post-aggregator +### HyperUnique Cardinality post-aggregator The hyperUniqueCardinality post aggregator is used to wrap a hyperUnique object such that it can be used in post aggregations. @@ -90,8 +90,7 @@ It can be used in a sample calculation as so: } ``` - -### Example Usage +#### Example Usage In this example, let’s calculate a simple percentage using post aggregators. Let’s imagine our data set has a metric called "total". @@ -122,5 +121,70 @@ The format of the query JSON is as follows: } ... } +``` -``` \ No newline at end of file +### Approximate Histogram post-aggregators + +Post-aggregators used to transform opaque approximate histogram objects +into actual histogram representations, and to compute various distribution metrics. + +#### equal buckets post-aggregator + +Computes a visual representation of the approximate histogram with a given number of equal-sized bins + +```json +{ "type" : "equalBuckets", "name" : , "fieldName" : , + "numBuckets" : } +``` + +#### buckets post-aggregator + +Computes a visual representation given an initial breakpoint, offset, and a bucket size. + +```json +{ "type" : "buckets", "name" : , "fieldName" : , + "bucketSize" : , "offset" : } +``` + +#### custom buckets post-aggregator + +Computes a visual representation of the approximate histogram with bins laid out according to the given breaks + +```json +{ "type" : "customBuckets", "name" : , "fieldName" : , + "breaks" : [ , , ... ] } +``` + +#### min post-aggregator + +Returns the minimum value of the underlying approximate histogram aggregator + +```json +{ "type" : "min", "name" : , "fieldName" : } +``` + +#### max post-aggregator + +Returns the maximum value of the underlying approximate histogram aggregator + +```json +{ "type" : "max", "name" : , "fieldName" : } +``` + +#### quantile post-aggregator + +Computes a single quantile based on the underlying approximate histogram aggregator + +```json +{ "type" : "quantile", "name" : , "fieldName" : , + "probability" : } +``` + +#### quantiles post-aggregator + +Computes an array of quantiles based on the underlying approximate histogram aggregator + +```json +{ "type" : "quantiles", "name" : , "fieldName" : , + "probabilities" : [ , , ... ] } +``` diff --git a/histogram/pom.xml b/histogram/pom.xml new file mode 100644 index 00000000000..f3d9c31f617 --- /dev/null +++ b/histogram/pom.xml @@ -0,0 +1,71 @@ + + + + 4.0.0 + io.druid.extensions + druid-histogram + druid-histogram + druid-histogram + + + io.druid + druid + 0.6.129-SNAPSHOT + + + + + io.druid + druid-processing + ${project.parent.version} + + + + + io.druid + druid-processing + ${project.parent.version} + test + test-jar + + + junit + junit + test + + + + + + + maven-jar-plugin + + + + true + true + + + + + + + diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java new file mode 100644 index 00000000000..fc06968f870 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogram.java @@ -0,0 +1,1680 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.primitives.Floats; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.google.common.primitives.Shorts; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class ApproximateHistogram +{ + public static final int DEFAULT_HISTOGRAM_SIZE = 50; + public static final int DEFAULT_BUCKET_SIZE = 7; + + // max size of the histogram (number of bincount/position pairs) + int size; + + public float[] positions; + public long[] bins; + + // used bincount + int binCount; + // min value that's been put into histogram + float min; + float max; + // total number of values that have been put into histogram + transient long count; + + // lower limit to maintain resolution + // cutoff above which we merge bins is the difference of the limits / (size - 3) + // so we'll set size = 203, lower limit = 0, upper limit = 10.00 if we don't want + // to merge differences < 0.05 + transient float lowerLimit; + transient float upperLimit; + + // use sign bit to indicate approximate bin and remaining bits for bin count + private static final long APPROX_FLAG_BIT = Long.MIN_VALUE; + private static final long COUNT_BITS = Long.MAX_VALUE; + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ApproximateHistogram that = (ApproximateHistogram) o; + + if (size != that.size) { + return false; + } + if (binCount != that.binCount) { + return false; + } + if (Float.compare(that.max, max) != 0) { + return false; + } + if (Float.compare(that.min, min) != 0) { + return false; + } + for (int i = 0; i < binCount; ++i) { + if (positions[i] != that.positions[i]) { + return false; + } + } + for (int i = 0; i < binCount; ++i) { + if (bins[i] != that.bins[i]) { + return false; + } + } + return true; + } + + @Override + public int hashCode() + { + int result = size; + result = 31 * result + (positions != null ? ArrayUtils.hashCode(positions, 0, binCount) : 0); + result = 31 * result + (bins != null ? ArrayUtils.hashCode(bins, 0, binCount) : 0); + result = 31 * result + binCount; + result = 31 * result + (min != +0.0f ? Float.floatToIntBits(min) : 0); + result = 31 * result + (max != +0.0f ? Float.floatToIntBits(max) : 0); + return result; + } + + + public ApproximateHistogram( + int size, + float[] positions, + long[] bins, + int binCount, + float min, + float max, + long count, + float lowerLimit, + float upperLimit + ) + { + Preconditions.checkArgument(positions.length == bins.length, "position and bin array must have same size"); + Preconditions.checkArgument(binCount <= size, "binCount must be less or equal to size"); + + this.size = size; + this.positions = positions; + this.bins = bins; + this.binCount = binCount; + this.min = min; + this.max = max; + this.count = count; + this.lowerLimit = lowerLimit; + this.upperLimit = upperLimit; + } + + public ApproximateHistogram() + { + this(DEFAULT_HISTOGRAM_SIZE); + } + + public ApproximateHistogram(int size) + { + this( + size, //size + new float[size], //positions + new long[size], //bins + 0, //binCount + Float.POSITIVE_INFINITY, //min + Float.NEGATIVE_INFINITY, //max + 0, //count + Float.NEGATIVE_INFINITY, //lowerLimit + Float.POSITIVE_INFINITY //upperLimit + ); + } + + public ApproximateHistogram(int size, float lowerLimit, float upperLimit) + { + this( + size, //size + new float[size], //positions + new long[size], //bins + 0, //binCount + Float.POSITIVE_INFINITY, //min + Float.NEGATIVE_INFINITY, //max + 0, //count + lowerLimit, //lowerLimit + upperLimit //upperLimit + ); + } + + public ApproximateHistogram(int binCount, float[] positions, long[] bins, float min, float max) + { + this( + positions.length, //size + positions, //positions + bins, //bins + binCount, //binCount + min, //min + max, //max + sumBins(bins, binCount), //count + Float.NEGATIVE_INFINITY, //lowerLimit + Float.POSITIVE_INFINITY //upperLimit + ); + } + + public long count() { return count; } + + public float min() { return min; } + + public float max() { return max; } + + public int binCount() { return binCount; } + + public int capacity() { return size; } + + public float[] positions() { return Arrays.copyOfRange(positions, 0, binCount); } + + public long[] bins() + { + long[] counts = new long[binCount]; + for (int i = 0; i < binCount; ++i) { + counts[i] = bins[i] & COUNT_BITS; + } + return counts; + } + + @Override + public String toString() + { + return "ApproximateHistogram{" + + "size=" + size + + ", lowerLimit=" + lowerLimit + + ", upperLimit=" + upperLimit + + ", positions=" + Arrays.toString(positions()) + + ", bins=" + getBinsString() + + ", binCount=" + binCount + + ", min=" + min + + ", max=" + max + + ", count=" + count + + '}'; + } + + public long getExactCount() + { + long exactCount = 0; + for (int i = 0; i < binCount; ++i) { + if ((bins[i] & APPROX_FLAG_BIT) == 0) { + exactCount += (bins[i] & COUNT_BITS); + } + } + return exactCount; + } + + public float getMin() { return this.min;} + + public float getMax() { return this.max;} + + private static long sumBins(long[] bins, int binCount) + { + long count = 0; + for (int i = 0; i < binCount; ++i) { + count += bins[i] & COUNT_BITS; + } + return count; + } + + /** + * Returns a string representation of the actual bin counts + * + * @return + */ + protected String getBinsString() + { + StringBuilder s = new StringBuilder(); + s.append('['); + for (int i = 0; i < bins.length; ++i) { + if (i > 0) { + s.append(", "); + } + if ((bins[i] & APPROX_FLAG_BIT) != 0) { + s.append("*"); + } + s.append(bins[i] & COUNT_BITS); + } + s.append(']'); + return s.toString(); + } + + public void setLowerLimit(float lowerLimit) + { + this.lowerLimit = lowerLimit; + } + + public void setUpperLimit(float upperLimit) + { + this.upperLimit = upperLimit; + } + + /** + * Adds the given value to the histogram + * + * @param value the value to be added + */ + public void offer(float value) + { + // update min/max + if (value < min) { + min = value; + } + if (value > max) { + max = value; + } + + // initial value + if (binCount == 0) { + positions[0] = value; + bins[0] = 1; + count++; + + binCount++; + return; + } + + final int index = Arrays.binarySearch(positions, 0, binCount, value); + + if (index >= 0) { + // we have an exact match, simply increase the count, but keep the approximate flag + bins[index] = (bins[index] & APPROX_FLAG_BIT) | ((bins[index] & COUNT_BITS) + 1); + count++; + return; + } + + // otherwise merge the value into a new or existing bin at the following position + final int insertAt = -(index + 1); + + if (binCount < size) { + // we have a spare slot, put the value into a new bin + shiftRight(insertAt, binCount); + + positions[insertAt] = value; + bins[insertAt] = 1; + count++; + + binCount++; + return; + } + + // no more slots available merge the new value into and existing bin + // or merge existing bins before inserting the new one + + int minPos = minDeltaIndex(); + float minDelta = minPos >= 0 ? positions[minPos + 1] - positions[minPos] : Float.MAX_VALUE; + + // determine the distance of new value to the nearest bins + final float deltaRight = insertAt < binCount ? positions[insertAt] - value : Float.MAX_VALUE; + final float deltaLeft = insertAt > 0 ? value - positions[insertAt - 1] : Float.MAX_VALUE; + + boolean mergeValue = false; + if (deltaRight < minDelta) { + minDelta = deltaRight; + minPos = insertAt; + mergeValue = true; + } + if (deltaLeft < minDelta) { + minDelta = deltaLeft; + minPos = insertAt - 1; + mergeValue = true; + } + + if (mergeValue) { + // merge new value into an existing bin and set approximate flag + final long k = bins[minPos] & COUNT_BITS; + positions[minPos] = (positions[minPos] * k + value) / (k + 1); + bins[minPos] = (k + 1) | APPROX_FLAG_BIT; + count++; + } else { + // merge the closest bins together and insert new value as a separate bin + mergeInsert(minPos, insertAt, value, 1); + } + } + + protected int minDeltaIndex() + { + // determine minimum distance between existing bins + float minDelta = Float.MAX_VALUE; + int minPos = -1; + for (int i = 0; i < binCount - 1; ++i) { + float delta = (positions[i + 1] - positions[i]); + if (delta < minDelta) { + minDelta = delta; + minPos = i; + } + } + return minPos; + } + + /** + * Merges the bin in the given position with the next bin + * + * @param index index of the bin to merge, index must satisfy 0 <= index < binCount - 1 + */ + protected void merge(final int index) + { + mergeInsert(index, -1, 0, 0); + } + + /** + * Merges the bin in the mergeAt position with the bin in position mergeAt+1 + * and simultaneously inserts the given bin (v,c) as a new bin at position insertAt + * + * @param mergeAt index of the bin to be merged + * @param insertAt index to insert the new bin at + * @param v bin position + * @param c bin count + */ + protected void mergeInsert(final int mergeAt, int insertAt, final float v, final long c) + { + final long k0 = (bins[mergeAt] & COUNT_BITS); + final long k1 = (bins[mergeAt + 1] & COUNT_BITS); + final long sum = k0 + k1; + + // merge bin at given position with the next bin and set approximate flag + positions[mergeAt] = (float) (((double) positions[mergeAt] * k0 + (double) positions[mergeAt + 1] * k1) / sum); + bins[mergeAt] = sum | APPROX_FLAG_BIT; + + final int unusedIndex = mergeAt + 1; + + if (insertAt >= 0) { + // use unused slot to shift array left or right and make space for the new bin to insert + if (insertAt < unusedIndex) { + shiftRight(insertAt, unusedIndex); + } else if (insertAt >= unusedIndex) { + shiftLeft(unusedIndex, insertAt - 1); + insertAt--; + } + positions[insertAt] = v; + bins[insertAt] = c; + count++; + } else { + // simple merging of bins, shift everything left and free up the unused bin + shiftLeft(unusedIndex, binCount - 1); + binCount--; + } + } + + /** + * Shifts the given range the histogram bins one slot to the right + * + * @param start index of the first bin to shift + * @param end index of the rightmost bin to shift into + */ + protected void shiftRight(int start, int end) + { + float prevVal = positions[start]; + long prevCnt = bins[start]; + + for (int i = start + 1; i <= end; ++i) { + float tmpVal = positions[i]; + long tmpCnt = bins[i]; + + positions[i] = prevVal; + bins[i] = prevCnt; + + prevVal = tmpVal; + prevCnt = tmpCnt; + } + } + + /** + * Shifts the given range of histogram bins one slot to the left + * + * @param start index of the leftmost empty bin to shift into + * @param end index of the last bin to shift left + */ + protected void shiftLeft(int start, int end) + { + for (int i = start; i < end; ++i) { + positions[i] = positions[i + 1]; + bins[i] = bins[i + 1]; + } + } + + public ApproximateHistogram fold(ApproximateHistogram h) + { + return fold(h, null, null, null); + } + + public ApproximateHistogram fold(ApproximateHistogram h, float[] mergedPositions, long[] mergedBins, float[] deltas) + { + if (size == 0) { + return copy(h); + } else { + return foldMin(h, mergedPositions, mergedBins, deltas); + } + } + + public ApproximateHistogram foldFast(ApproximateHistogram h) + { + return foldFast(h, null, null); + } + + /** + * @param h histogram to be merged into the current histogram + * @param mergedPositions temporary buffer of size greater or equal to this.capacity() + * @param mergedBins temporary buffer of size greater or equal to this.capacity() + * + * @return returns this histogram with h folded into it + */ + public ApproximateHistogram foldFast(ApproximateHistogram h, float[] mergedPositions, long[] mergedBins) + { + if (size == 0) { + return copy(h); + } else { + return foldRule(h, mergedPositions, mergedBins); + } + } + + /** + * Copies histogram h into the current histogram. + * + * @param h + * + * @return + */ + public ApproximateHistogram copy(ApproximateHistogram h) + { + this.size = h.size; + this.positions = new float[size]; + this.bins = new long[size]; + + System.arraycopy(h.positions, 0, this.positions, 0, h.binCount); + System.arraycopy(h.bins, 0, this.bins, 0, h.binCount); + this.min = h.min; + this.max = h.max; + this.binCount = h.binCount; + this.count = h.count; + return this; + } + + //approximate histogram solution using min heap to store location of min deltas + protected ApproximateHistogram foldMin( + ApproximateHistogram h, + float[] mergedPositions, + long[] mergedBins, + float[] deltas + ) + { + // find common min / max + float mergedMin = this.min < h.min ? this.min : h.min; + float mergedMax = this.max > h.max ? this.max : h.max; + long mergedCount = this.count + h.count; + + int maxSize = this.binCount + h.binCount; + int[] next = new int[maxSize]; + int[] prev = new int[maxSize]; + + // use preallocated arrays if passed + if (mergedPositions == null || mergedBins == null || deltas == null) { + mergedPositions = new float[maxSize]; + mergedBins = new long[maxSize]; + deltas = new float[maxSize]; + } else { + Preconditions.checkArgument( + mergedPositions.length >= maxSize, + "temp buffer [mergedPositions] too small: length must be at least [%d], got [%d]", + maxSize, + mergedPositions.length + ); + Preconditions.checkArgument( + mergedBins.length >= maxSize, + "temp buffer [mergedBins] too small: length must be at least [%d], got [%d]", + maxSize, + mergedPositions.length + ); + Preconditions.checkArgument( + deltas.length >= maxSize, + "temp buffer [deltas] too small: length must be at least [%d], got [%d]", + maxSize, + mergedPositions.length + ); + } + + int mergedBinCount = combineBins( + this.binCount, this.positions, this.bins, h.binCount, h.positions, h.bins, + mergedPositions, mergedBins, deltas + ); + if (mergedBinCount == 0) { + return this; + } + + // determine how many bins to merge + int numMerge = mergedBinCount - this.size; + if (numMerge < 0) { + numMerge = 0; + } + + // perform the required number of merges + mergeBins(mergedBinCount, mergedPositions, mergedBins, deltas, numMerge, next, prev); + + // copy merged values + int i = 0; + int k = 0; + while (i < mergedBinCount) { + this.positions[k] = mergedPositions[i]; + this.bins[k] = mergedBins[i]; + ++k; + i = next[i]; + } + this.binCount = mergedBinCount - numMerge; + this.min = mergedMin; + this.max = mergedMax; + this.count = mergedCount; + return this; + } + + protected ApproximateHistogram foldRule(ApproximateHistogram h, float[] mergedPositions, long[] mergedBins) + { + // ruleCombine bins requires at least one bin + if (h.binCount == 0) { + return this; + } + + // find common min / max + float mergedMin = this.min < h.min ? this.min : h.min; + float mergedMax = this.max > h.max ? this.max : h.max; + long mergedCount = this.count + h.count; + this.min = mergedMin; + this.max = mergedMax; + + // use preallocated arrays if passed + if (mergedPositions == null) { + mergedPositions = new float[this.size]; + mergedBins = new long[this.size]; + } + + int mergedBinCount; + if (this.binCount + h.binCount <= this.size) { + // no need to merge bins + mergedBinCount = combineBins( + this.binCount, this.positions, this.bins, + h.binCount, h.positions, h.bins, + mergedPositions, mergedBins, null + ); + } else { + mergedBinCount = ruleCombineBins( + this.binCount, this.positions, this.bins, h.binCount, h.positions, h.bins, + mergedPositions, mergedBins + ); + } + for (int i = 0; i < mergedBinCount; ++i) { + this.positions[i] = mergedPositions[i]; + this.bins[i] = mergedBins[i]; + } + + this.binCount = mergedBinCount; + this.count = mergedCount; + + return this; + } + + protected int ruleCombineBins( + int leftBinCount, float[] leftPositions, long[] leftBins, + int rightBinCount, float[] rightPositions, long[] rightBins, + float[] mergedPositions, long[] mergedBins + ) + { + final float cutoff; + // assumes binCount is greater than one for both histograms + // if upper and lower limits are set, we use the first and last used values of the arrays + // for information below and above the limits, respectively + if (this.upperLimit != Float.POSITIVE_INFINITY && this.lowerLimit != Float.NEGATIVE_INFINITY) { + cutoff = (this.upperLimit - this.lowerLimit) / (size - 2 - 1); + } else { + if (this.upperLimit != Float.POSITIVE_INFINITY) { + cutoff = (this.upperLimit - this.min) / (size - 2); + } else if (this.lowerLimit != Float.NEGATIVE_INFINITY) { + cutoff = (this.max - this.lowerLimit) / (size - 2); + } else { + cutoff = (this.max - this.min) / (size - 1); + } + } + + float lowerPosition = 0f; + long lowerBin = 0; + float upperPosition = 0f; + long upperBin = 0; + + int j = 0; + int k = 0; + int pos = 0; + + // continuously merge the left histogram below the lower limit + while (j != leftBinCount) { + final float m1 = leftPositions[j]; + if (m1 < lowerLimit) { + final long k1 = leftBins[j] & COUNT_BITS; + float delta = (m1 - lowerPosition); + final long k0 = lowerBin & COUNT_BITS; + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + lowerPosition = -delta * w + m1; + // set approximate flag + lowerBin = sum | APPROX_FLAG_BIT; + ++j; + } else { + break; + } + } + + // continuously merge the right histogram below the lower limit + while (k != rightBinCount) { + final float m1 = rightPositions[k]; + if (m1 < lowerLimit) { + final long k1 = rightBins[k] & COUNT_BITS; + float delta = (m1 - lowerPosition); + final long k0 = lowerBin & COUNT_BITS; + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + lowerPosition = -delta * w + m1; + // set approximate flag + lowerBin = sum | APPROX_FLAG_BIT; + ++k; + } else { + break; + } + } + + // if there are values below the lower limit, store them in array position 0 + if ((lowerBin & COUNT_BITS) > 0) { + mergedPositions[0] = lowerPosition; + mergedBins[0] = lowerBin; + pos = 1; + } + + // if there are values below the lower limit, fill in array position 1 + // else array position 0 + while (j != leftBinCount || k != rightBinCount) { + if (j != leftBinCount && (k == rightBinCount || leftPositions[j] < rightPositions[k])) { + mergedPositions[pos] = leftPositions[j]; + mergedBins[pos] = leftBins[j]; + ++j; + break; + } else { + mergedPositions[pos] = rightPositions[k]; + mergedBins[pos] = rightBins[k]; + ++k; + break; + } + } + + while (j != leftBinCount || k != rightBinCount) { + if (j != leftBinCount && (k == rightBinCount || leftPositions[j] < rightPositions[k])) { + final float m1 = leftPositions[j]; + final long k1 = leftBins[j] & COUNT_BITS; + + // above the upper limit gets merged continuously in the left histogram + if (m1 > upperLimit) { + float delta = (m1 - upperPosition); + final long k0 = upperBin & COUNT_BITS; + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + upperPosition = -delta * w + m1; + // set approximate flag + upperBin = sum | APPROX_FLAG_BIT; + ++j; + continue; + } + + final float delta = (m1 - mergedPositions[pos]); + + if (delta <= cutoff) { + final long k0 = mergedBins[pos] & COUNT_BITS; + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + mergedPositions[pos] = -delta * w + m1; + // set approximate flag + mergedBins[pos] = sum | APPROX_FLAG_BIT; + } else { + ++pos; + mergedPositions[pos] = m1; + mergedBins[pos] = k1; + } + ++j; + } else { + final float m1 = rightPositions[k]; + final long k1 = rightBins[k] & COUNT_BITS; + + // above the upper limit gets merged continuously in the right histogram + if (m1 > upperLimit) { + float delta = (m1 - upperPosition); + final long k0 = upperBin & COUNT_BITS; + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + upperPosition = -delta * w + m1; + // set approximate flag + upperBin = sum | APPROX_FLAG_BIT; + ++k; + continue; + } + + final float delta = (m1 - mergedPositions[pos]); + + if (delta <= cutoff) { + final long k0 = mergedBins[pos] & COUNT_BITS; + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + mergedPositions[pos] = -delta * w + m1; + mergedBins[pos] = sum | APPROX_FLAG_BIT; + } else { + ++pos; + mergedPositions[pos] = m1; + mergedBins[pos] = k1; + } + ++k; + } + } + + if ((upperBin & COUNT_BITS) > 0) { + ++pos; + mergedPositions[pos] = upperPosition; + mergedBins[pos] = upperBin; + } + + return pos + 1; + } + + + /** + * mergeBins performs the given number of bin merge operations on the given histogram + *

+ * It repeatedly merges the two closest bins until it has performed the requested number of merge operations. + * Merges are done in-place and unused bins have unknown state + *

+ * next / prev maintains a doubly-linked list of valid bin indices into the mergedBins array. + *

+ * Fast operation is achieved by building a min-heap of the deltas as opposed to repeatedly + * scanning the array of deltas to find the minimum. A reverse index into the heap is maintained + * to allow deleting and updating of specific deltas. + *

+ * next and prev arrays are used to maintain indices to the previous / next valid bin from a given bin index + *

+ * Its effect is equivalent to running the following code: + *

+ *

+   *   ApproximateHistogram merged = new ApproximateHistogram(mergedBinCount, mergedPositions, mergedBins);
+   *
+   *   int targetSize = merged.binCount() - numMerge;
+   *   while(merged.binCount() > targetSize) {
+   *     merged.merge(merged.minDeltaIndex());
+   *   }
+   * 
+ * + * @param mergedBinCount + * @param mergedPositions + * @param mergedBins + * @param deltas + * @param numMerge + * @param next + * @param prev + * + * @return the last valid index into the mergedPositions and mergedBins arrays + */ + private static void mergeBins( + int mergedBinCount, float[] mergedPositions, + long[] mergedBins, + float[] deltas, + int numMerge, + int[] next, + int[] prev + ) + { + // repeatedly search for two closest bins, merge them and update the corresponding deltas + + // maintain index to the last valid bin + int lastValidIndex = mergedBinCount - 1; + + // initialize prev / next lookup arrays + for (int i = 0; i < mergedBinCount; ++i) { + next[i] = i + 1; + } + for (int i = 0; i < mergedBinCount; ++i) { + prev[i] = i - 1; + } + + // initialize min-heap of deltas and the reverse index into the heap + int heapSize = mergedBinCount - 1; + int[] heap = new int[heapSize]; + int[] reverseIndex = new int[heapSize]; + for (int i = 0; i < heapSize; ++i) { + heap[i] = i; + } + for (int i = 0; i < heapSize; ++i) { + reverseIndex[i] = i; + } + + heapify(heap, reverseIndex, heapSize, deltas); + + { + int i = 0; + while (i < numMerge) { + // find the smallest delta within the range used for bins + + // pick minimum delta by scanning array + //int currentIndex = minIndex(deltas, lastValidIndex); + + // pick minimum delta index using min-heap + int currentIndex = heap[0]; + + final int nextIndex = next[currentIndex]; + final int prevIndex = prev[currentIndex]; + + final long k0 = mergedBins[currentIndex] & COUNT_BITS; + final long k1 = mergedBins[nextIndex] & COUNT_BITS; + final float m0 = mergedPositions[currentIndex]; + final float m1 = mergedPositions[nextIndex]; + final float d1 = deltas[nextIndex]; + + final long sum = k0 + k1; + final float w = (float) k0 / (float) sum; + + // merge bin at given position with the next bin + final float mm0 = (m0 - m1) * w + m1; + + mergedPositions[currentIndex] = mm0; + //mergedPositions[nextIndex] = Float.MAX_VALUE; // for debugging + + mergedBins[currentIndex] = sum | APPROX_FLAG_BIT; + //mergedBins[nextIndex] = -1; // for debugging + + // update deltas and min-heap + if (nextIndex == lastValidIndex) { + // merged bin is the last => remove the current bin delta from the heap + heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[currentIndex], deltas); + + //deltas[currentIndex] = Float.MAX_VALUE; // for debugging + } else { + // merged bin is not the last => remove the merged bin delta from the heap + heapSize = heapDelete(heap, reverseIndex, heapSize, reverseIndex[nextIndex], deltas); + + // updated current delta + deltas[currentIndex] = m1 - mm0 + d1; + + // updated delta is necessarily larger than existing one, therefore we only need to push it down the heap + siftDown(heap, reverseIndex, reverseIndex[currentIndex], heapSize - 1, deltas); + } + + if (prevIndex >= 0) { + // current bin is not the first, therefore update the previous bin delta + deltas[prevIndex] = mm0 - mergedPositions[prevIndex]; + + // updated previous bin delta is necessarily larger than its existing value => push down the heap + siftDown(heap, reverseIndex, reverseIndex[prevIndex], heapSize - 1, deltas); + } + + // mark the merged bin as invalid + // deltas[nextIndex] = Float.MAX_VALUE; // for debugging + + // update last valid index if we merged the last bin + if (nextIndex == lastValidIndex) { + lastValidIndex = currentIndex; + } + + next[currentIndex] = next[nextIndex]; + if (nextIndex < lastValidIndex) { + prev[next[nextIndex]] = currentIndex; + } + + ++i; + } + } + } + + /** + * Builds a min-heap and a reverseIndex into the heap from the given array of values + * + * @param heap min-heap stored as indices into the array of values + * @param reverseIndex reverse index from the array of values into the heap + * @param count current size of the heap + * @param values values to be stored in the heap + */ + private static void heapify(int[] heap, int[] reverseIndex, int count, float[] values) + { + int start = (count - 2) / 2; + while (start >= 0) { + siftDown(heap, reverseIndex, start, count - 1, values); + start--; + } + } + + /** + * Rebalances the min-heap by pushing values from the top down and simultaneously updating the reverse index + * + * @param heap min-heap stored as indices into the array of values + * @param reverseIndex reverse index from the array of values into the heap + * @param start index to start re-balancing from + * @param end index to stop re-balancing at + * @param values values stored in the heap + */ + private static void siftDown(int[] heap, int[] reverseIndex, int start, int end, float[] values) + { + int root = start; + while (root * 2 + 1 <= end) { + int child = root * 2 + 1; + int swap = root; + if (values[heap[swap]] > values[heap[child]]) { + swap = child; + } + if (child + 1 <= end && values[heap[swap]] > values[heap[child + 1]]) { + swap = child + 1; + } + if (swap != root) { + // swap + int tmp = heap[swap]; + heap[swap] = heap[root]; + heap[root] = tmp; + + // heap index from delta index + reverseIndex[heap[swap]] = swap; + reverseIndex[heap[root]] = root; + + root = swap; + } else { + return; + } + } + } + + /** + * Deletes an item from the min-heap and updates the reverse index + * + * @param heap min-heap stored as indices into the array of values + * @param reverseIndex reverse index from the array of values into the heap + * @param count current size of the heap + * @param heapIndex index of the item to be deleted + * @param values values stored in the heap + * + * @return + */ + private static int heapDelete(int[] heap, int[] reverseIndex, int count, int heapIndex, float[] values) + { + int end = count - 1; + + reverseIndex[heap[heapIndex]] = -1; + + heap[heapIndex] = heap[end]; + reverseIndex[heap[heapIndex]] = heapIndex; + + end--; + siftDown(heap, reverseIndex, heapIndex, end, values); + return count - 1; + } + + private static int minIndex(float[] deltas, int lastValidIndex) + { + int minIndex = -1; + float min = Float.MAX_VALUE; + for (int k = 0; k < lastValidIndex; ++k) { + float value = deltas[k]; + if (value < min) { + minIndex = k; + min = value; + } + } + return minIndex; + } + + /** + * Combines two sets of histogram bins using merge-sort and computes the delta between consecutive bin positions. + * Duplicate bins are merged together. + * + * @param leftBinCount + * @param leftPositions + * @param leftBins + * @param rightBinCount + * @param rightPositions + * @param rightBins + * @param mergedPositions array to store the combined bin positions (size must be at least leftBinCount + rightBinCount) + * @param mergedBins array to store the combined bin counts (size must be at least leftBinCount + rightBinCount) + * @param deltas deltas between consecutive bin positions in the merged bins (size must be at least leftBinCount + rightBinCount) + * + * @return the number of combined bins + */ + private static int combineBins( + int leftBinCount, float[] leftPositions, long[] leftBins, + int rightBinCount, float[] rightPositions, long[] rightBins, + float[] mergedPositions, long[] mergedBins, float[] deltas + ) + { + int i = 0; + int j = 0; + int k = 0; + while (j < leftBinCount || k < rightBinCount) { + if (j < leftBinCount && (k == rightBinCount || leftPositions[j] < rightPositions[k])) { + mergedPositions[i] = leftPositions[j]; + mergedBins[i] = leftBins[j]; + ++j; + } else if (k < rightBinCount && (j == leftBinCount || leftPositions[j] > rightPositions[k])) { + mergedPositions[i] = rightPositions[k]; + mergedBins[i] = rightBins[k]; + ++k; + } else { + // combine overlapping bins + mergedPositions[i] = leftPositions[j]; + mergedBins[i] = leftBins[j] + rightBins[k]; + ++j; + ++k; + } + if (deltas != null && i > 0) { + deltas[i - 1] = mergedPositions[i] - mergedPositions[i - 1]; + } + ++i; + } + return i; + } + + /** + * Returns a byte-array representation of this ApproximateHistogram object + * + * @return + */ + @JsonValue + public byte[] toBytes() + { + ByteBuffer buf = ByteBuffer.allocate(getMinStorageSize()); + toBytes(buf); + return buf.array(); + } + + + public int getDenseStorageSize() + { + return Ints.BYTES * 2 + Floats.BYTES * size + Longs.BYTES * size + Floats.BYTES * 2; + } + + public int getSparseStorageSize() + { + return Ints.BYTES * 2 + Floats.BYTES * binCount + Longs.BYTES * binCount + Floats.BYTES * 2; + } + + public int getCompactStorageSize() + { + // ensures exactCount and (count - exactCount) can safely be cast to (int) + Preconditions.checkState(canStoreCompact(), "Approximate histogram cannot be stored in compact form"); + + final long exactCount = getExactCount(); + if (exactCount == count) { + return Shorts.BYTES + 1 + Floats.BYTES * (int) exactCount; + } else { + return Shorts.BYTES + + 1 + + Floats.BYTES * (int) exactCount + + 1 + + Floats.BYTES * (int) (count - exactCount) + + Floats.BYTES * 2; + } + } + + public int getMaxStorageSize() + { + return getDenseStorageSize(); + } + + /** + * Returns the minimum number of bytes required to store this ApproximateHistogram object + * + * @return required number of bytes + */ + public int getMinStorageSize() + { + // sparse is always small than dense, so no need to check + if (canStoreCompact() && getCompactStorageSize() < getSparseStorageSize()) { + return getCompactStorageSize(); + } else { + return getSparseStorageSize(); + } + } + + /** + * Checks whether this approximate histogram can be stored in a compact form + * + * @return true if yes, false otherwise + */ + public boolean canStoreCompact() + { + final long exactCount = getExactCount(); + return ( + size <= Short.MAX_VALUE + && exactCount <= Byte.MAX_VALUE + && (count - exactCount) <= Byte.MAX_VALUE + ); + } + + /** + * Writes the representation of this ApproximateHistogram object to the given byte-buffer + * + * @param buf + */ + public void toBytes(ByteBuffer buf) + { + if (canStoreCompact() && getCompactStorageSize() < getSparseStorageSize()) { + // store compact + toBytesCompact(buf); + } else { + // store sparse + toBytesSparse(buf); + } + } + + /** + * Writes the dense representation of this ApproximateHistogram object to the given byte-buffer + *

+ * Requires 16 + 12 * size bytes of storage + * + * @param buf + */ + public void toBytesDense(ByteBuffer buf) + { + buf.putInt(size); + buf.putInt(binCount); + + buf.asFloatBuffer().put(positions); + buf.position(buf.position() + Floats.BYTES * positions.length); + buf.asLongBuffer().put(bins); + buf.position(buf.position() + Longs.BYTES * bins.length); + + buf.putFloat(min); + buf.putFloat(max); + } + + /** + * Writes the sparse representation of this ApproximateHistogram object to the given byte-buffer + *

+ * Requires 16 + 12 * binCount bytes of storage + * + * @param buf ByteBuffer to write object to + */ + public void toBytesSparse(ByteBuffer buf) + { + buf.putInt(size); + buf.putInt(-1 * binCount); // use negative binCount to indicate sparse storage + for (int i = 0; i < binCount; ++i) { + buf.putFloat(positions[i]); + } + for (int i = 0; i < binCount; ++i) { + buf.putLong(bins[i]); + } + buf.putFloat(min); + buf.putFloat(max); + } + + /** + * Returns a compact byte-buffer representation of this ApproximateHistogram object + * storing actual values as opposed to histogram bins + *

+ * Requires 3 + 4 * count bytes of storage with count <= 127 + * + * @param buf + */ + public void toBytesCompact(ByteBuffer buf) + { + Preconditions.checkState(canStoreCompact(), "Approximate histogram cannot be stored in compact form"); + + buf.putShort((short) (-1 * size)); // use negative size to indicate compact storage + + final long exactCount = getExactCount(); + if (exactCount != count) { + // use negative count to indicate approximate bins + buf.put((byte) (-1 * (count - exactCount))); + + // store actual values instead of bins + for (int i = 0; i < binCount; ++i) { + // repeat each value bins[i] times for approximate bins + if ((bins[i] & APPROX_FLAG_BIT) != 0) { + for (int k = 0; k < (bins[i] & COUNT_BITS); ++k) { + buf.putFloat(positions[i]); + } + } + } + + // tack on min and max since they may be lost int the approximate bins + buf.putFloat(min); + buf.putFloat(max); + } + + buf.put((byte) exactCount); + // store actual values instead of bins + for (int i = 0; i < binCount; ++i) { + // repeat each value bins[i] times for exact bins + if ((bins[i] & APPROX_FLAG_BIT) == 0) { + for (int k = 0; k < (bins[i] & COUNT_BITS); ++k) { + buf.putFloat(positions[i]); + } + } + } + } + + /** + * Constructs an Approximate Histogram object from the given byte-array representation + * + * @param bytes + * + * @return + */ + public static ApproximateHistogram fromBytes(byte[] bytes) + { + ByteBuffer buf = ByteBuffer.wrap(bytes); + return fromBytes(buf); + } + + /** + * Constructs an ApproximateHistogram object from the given dense byte-buffer representation + * + * @param buf + * + * @return + */ + public static ApproximateHistogram fromBytesDense(ByteBuffer buf) + { + int size = buf.getInt(); + int binCount = buf.getInt(); + + float[] positions = new float[size]; + long[] bins = new long[size]; + + buf.asFloatBuffer().get(positions); + buf.position(buf.position() + Floats.BYTES * positions.length); + buf.asLongBuffer().get(bins); + buf.position(buf.position() + Longs.BYTES * bins.length); + + float min = buf.getFloat(); + float max = buf.getFloat(); + + return new ApproximateHistogram(binCount, positions, bins, min, max); + } + + /** + * Constructs an ApproximateHistogram object from the given dense byte-buffer representation + * + * @param buf + * + * @return + */ + public static ApproximateHistogram fromBytesSparse(ByteBuffer buf) + { + int size = buf.getInt(); + int binCount = -1 * buf.getInt(); + + float[] positions = new float[size]; + long[] bins = new long[size]; + + for (int i = 0; i < binCount; ++i) { + positions[i] = buf.getFloat(); + } + for (int i = 0; i < binCount; ++i) { + bins[i] = buf.getLong(); + } + + float min = buf.getFloat(); + float max = buf.getFloat(); + + return new ApproximateHistogram(binCount, positions, bins, min, max); + } + + /** + * Constructs an ApproximateHistogram object from the given compact byte-buffer representation + * + * @param buf + * + * @return + */ + public static ApproximateHistogram fromBytesCompact(ByteBuffer buf) + { + short size = (short) (-1 * buf.getShort()); + byte count = buf.get(); + + if (count >= 0) { + // only exact bins + ApproximateHistogram histogram = new ApproximateHistogram(size); + for (int i = 0; i < count; ++i) { + histogram.offer(buf.getFloat()); + } + return histogram; + } else { + byte approxCount = (byte) (-1 * count); + + Map approx = Maps.newHashMap(); + + for (int i = 0; i < approxCount; ++i) { + final float value = buf.getFloat(); + if (approx.containsKey(value)) { + approx.put(value, approx.get(value) + 1); + } else { + approx.put(value, 1L); + } + } + + float min = buf.getFloat(); + float max = buf.getFloat(); + + byte exactCount = buf.get(); + + Map exact = Maps.newHashMap(); + + for (int i = 0; i < exactCount; ++i) { + final float value = buf.getFloat(); + if (exact.containsKey(value)) { + exact.put(value, exact.get(value) + 1); + } else { + exact.put(value, 1L); + } + } + + int binCount = exact.size() + approx.size(); + + List pos = Lists.newArrayList(); + pos.addAll(exact.keySet()); + pos.addAll(approx.keySet()); + Collections.sort(pos); + + float[] positions = new float[size]; + long[] bins = new long[size]; + + for (int i = 0; i < pos.size(); ++i) { + positions[i] = pos.get(i); + } + + for (int i = 0; i < pos.size(); ++i) { + final float value = pos.get(i); + if (exact.containsKey(value)) { + bins[i] = exact.get(value); + } else { + bins[i] = approx.get(value) | APPROX_FLAG_BIT; + } + } + + return new ApproximateHistogram(binCount, positions, bins, min, max); + } + } + + /** + * Constructs an ApproximateHistogram object from the given byte-buffer representation + * + * @param buf + * + * @return + */ + public static ApproximateHistogram fromBytes(ByteBuffer buf) + { + ByteBuffer copy = buf.asReadOnlyBuffer(); + // negative size indicates compact representation + // this works regardless of whether we use int or short for the size since the leftmost bit is the sign bit + if (copy.getShort(buf.position()) < 0) { + return fromBytesCompact(buf); + } else { + // ignore size + copy.getInt(); + // determine if sparse or dense based on sign of binCount + if (copy.getInt() < 0) { + return fromBytesSparse(buf); + } else { + return fromBytesDense(buf); + } + } + } + + /** + * Returns the approximate number of items less than or equal to b in the histogram + * + * @param b + * + * @return the approximate number of items less than or equal to b + */ + public double sum(final float b) + { + if (b < min) { + return 0; + } + if (b >= max) { + return count; + } + + int index = Arrays.binarySearch(positions, 0, binCount, b); + boolean exactMatch = index >= 0; + index = exactMatch ? index : -(index + 1); + + // we want positions[index] <= b < positions[index+1] + if (!exactMatch) { + index--; + } + + final boolean outerLeft = index < 0; + final boolean outerRight = index >= (binCount - 1); + + final long m0 = outerLeft ? 0 : (bins[index] & COUNT_BITS); + final long m1 = outerRight ? 0 : (bins[index + 1] & COUNT_BITS); + final double p0 = outerLeft ? min : positions[index]; + final double p1 = outerRight ? max : positions[index + 1]; + final boolean exact0 = (!outerLeft && (bins[index] & APPROX_FLAG_BIT) == 0); + final boolean exact1 = (!outerRight && (bins[index + 1] & APPROX_FLAG_BIT) == 0); + + // handle case when p0 = p1, which happens if the first bin = min or the last bin = max + final double l = (p1 == p0) ? 0 : (b - p0) / (p1 - p0); + + // don't include exact counts in the trapezoid calculation + long tm0 = m0; + long tm1 = m1; + if (exact0) { + tm0 = 0; + } + if (exact1) { + tm1 = 0; + } + final double mb = tm0 + (tm1 - tm0) * l; + double s = 0.5 * (tm0 + mb) * l; + + for (int i = 0; i < index; ++i) { + s += (bins[i] & COUNT_BITS); + } + + // add full bin count if left bin count is exact + if (exact0) { + return (s + m0); + } + + // otherwise add only the left half of the bin + else { + return (s + 0.5 * m0); + } + } + + /** + * Returns the approximate quantiles corresponding to the given probabilities. + * probabilities = [.5f] returns [median] + * probabilities = [.25f, .5f, .75f] returns the quartiles, [25%ile, median, 75%ile] + * + * @param probabilities + * + * @return an array of length probabilities.length representing the the approximate sample quantiles + * corresponding to the given probabilities + */ + + public float[] getQuantiles(float[] probabilities) + { + for (float p : probabilities) { + Preconditions.checkArgument(0 < p & p < 1, "quantile probabilities must be strictly between 0 and 1"); + } + + float[] quantiles = new float[probabilities.length]; + Arrays.fill(quantiles, Float.NaN); + + if (this.count() == 0) { + return quantiles; + } + + final long[] bins = this.bins(); + + for (int j = 0; j < probabilities.length; ++j) { + final double s = probabilities[j] * this.count(); + + int i = 0; + int sum = 0; + int k = 1; + long count = 0; + while (k <= this.binCount()) { + count = bins[k - 1]; + if (sum + count > s) { + i = k - 1; + break; + } else { + sum += count; + } + ++k; + } + + if (i == 0) { + quantiles[j] = this.min(); + } else { + final double d = s - sum; + final double c = -2 * d; + final long a = bins[i] - bins[i - 1]; + final long b = 2 * bins[i - 1]; + double z = 0; + if (a == 0) { + z = -c / b; + } else { + z = (-b + Math.sqrt(b * b - 4 * a * c)) / (2 * a); + } + final double uj = this.positions[i - 1] + (this.positions[i] - this.positions[i - 1]) * z; + quantiles[j] = (float) uj; + } + } + + return quantiles; + } + + /** + * Computes a visual representation of the approximate histogram with bins laid out according to the given breaks + * + * @param breaks + * + * @return + */ + public Histogram toHistogram(final float[] breaks) + { + final double[] approximateBins = new double[breaks.length - 1]; + + double prev = sum(breaks[0]); + for (int i = 1; i < breaks.length; ++i) { + double s = sum(breaks[i]); + approximateBins[i - 1] = (float) (s - prev); + prev = s; + } + + return new Histogram(breaks, approximateBins); + } + + /** + * Computes a visual representation of the approximate histogram with a given number of equal-sized bins + * + * @param size number of equal-sized bins to divide the histogram into + * + * @return + */ + public Histogram toHistogram(int size) + { + Preconditions.checkArgument(size > 1, "histogram size must be greater than 1"); + + float[] breaks = new float[size + 1]; + float delta = (max - min) / (size - 1); + breaks[0] = min - delta; + for (int i = 1; i < breaks.length - 1; ++i) { + breaks[i] = breaks[i - 1] + delta; + } + breaks[breaks.length - 1] = max; + return toHistogram(breaks); + } + + /** + * Computes a visual representation given an initial breakpoint, offset, and a bucket size. + * + * @param bucketSize the size of each bucket + * @param offset the location of one breakpoint + * + * @return + */ + public Histogram toHistogram(final float bucketSize, final float offset) + { + final float minFloor = (float) Math.floor((min() - offset) / bucketSize) * bucketSize + offset; + final float lowerLimitFloor = (float) Math.floor((lowerLimit - offset) / bucketSize) * bucketSize + offset; + final float firstBreak = Math.max(minFloor, lowerLimitFloor); + + final float maxCeil = (float) Math.ceil((max() - offset) / bucketSize) * bucketSize + offset; + final float upperLimitCeil = (float) Math.ceil((upperLimit - offset) / bucketSize) * bucketSize + offset; + final float lastBreak = Math.min(maxCeil, upperLimitCeil); + + final float cutoff = 0.1f; + + final ArrayList breaks = new ArrayList(); + + // to deal with left inclusivity when the min is the same as a break + final float bottomBreak = minFloor - bucketSize; + if (bottomBreak != firstBreak && (sum(firstBreak) - sum(bottomBreak) > cutoff)) { + breaks.add(bottomBreak); + } + + float left = firstBreak; + boolean leftSet = false; + + //the + bucketSize / 10 is because floating point addition is always slightly incorrect and so we need to account for that + while (left + bucketSize <= lastBreak + (bucketSize / 10)) { + final float right = left + bucketSize; + + if (sum(right) - sum(left) > cutoff) { + if (!leftSet) { + breaks.add(left); + } + breaks.add(right); + leftSet = true; + } else { + leftSet = false; + } + + left = right; + } + + if (breaks.get(breaks.size() - 1) != maxCeil && (sum(maxCeil) - sum(breaks.get(breaks.size() - 1)) > cutoff)) { + breaks.add(maxCeil); + } + + return toHistogram(Floats.toArray(breaks)); + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java new file mode 100644 index 00000000000..7fcac6d7213 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java @@ -0,0 +1,103 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.google.common.primitives.Longs; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.FloatColumnSelector; + +import java.util.Comparator; + +public class ApproximateHistogramAggregator implements Aggregator +{ + public static final Comparator COMPARATOR = new Comparator() + { + @Override + public int compare(Object o, Object o1) + { + return Longs.compare(((ApproximateHistogram) o).count(), ((ApproximateHistogram) o1).count()); + } + }; + + static Object combineHistograms(Object lhs, Object rhs) + { + return ((ApproximateHistogram) lhs).foldFast((ApproximateHistogram) rhs); + } + + private final String name; + private final FloatColumnSelector selector; + private final int resolution; + private final float lowerLimit; + private final float upperLimit; + + private ApproximateHistogram histogram; + + public ApproximateHistogramAggregator( + String name, + FloatColumnSelector selector, + int resolution, + float lowerLimit, + float upperLimit + ) + { + this.name = name; + this.selector = selector; + this.resolution = resolution; + this.lowerLimit = lowerLimit; + this.upperLimit = upperLimit; + this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit); + } + + @Override + public void aggregate() + { + histogram.offer(selector.get()); + } + + @Override + public void reset() + { + this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit); + } + + @Override + public Object get() + { + return histogram; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getFloat()"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java new file mode 100644 index 00000000000..603d07765ee --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -0,0 +1,253 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Floats; +import com.google.common.primitives.Ints; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ColumnSelectorFactory; +import org.apache.commons.codec.binary.Base64; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +@JsonTypeName("approxHistogram") +public class ApproximateHistogramAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x8; + + protected final String name; + protected final String fieldName; + + protected final int resolution; + protected final int numBuckets; + + protected final float lowerLimit; + protected final float upperLimit; + + @JsonCreator + public ApproximateHistogramAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("resolution") Integer resolution, + @JsonProperty("numBuckets") Integer numBuckets, + @JsonProperty("lowerLimit") Float lowerLimit, + @JsonProperty("upperLimit") Float upperLimit + + ) + { + this.name = name; + this.fieldName = fieldName.toLowerCase(); + this.resolution = resolution == null ? ApproximateHistogram.DEFAULT_HISTOGRAM_SIZE : resolution; + this.numBuckets = numBuckets == null ? ApproximateHistogram.DEFAULT_BUCKET_SIZE : numBuckets; + this.lowerLimit = lowerLimit == null ? Float.NEGATIVE_INFINITY : lowerLimit; + this.upperLimit = upperLimit == null ? Float.POSITIVE_INFINITY : upperLimit; + + Preconditions.checkArgument(this.resolution > 0, "resolution must be greater than 1"); + Preconditions.checkArgument(this.numBuckets > 0, "numBuckets must be greater than 1"); + Preconditions.checkArgument(this.upperLimit > this.lowerLimit, "upperLimit must be greater than lowerLimit"); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new ApproximateHistogramAggregator( + name, + metricFactory.makeFloatColumnSelector(fieldName), + resolution, + lowerLimit, + upperLimit + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new ApproximateHistogramBufferAggregator( + metricFactory.makeFloatColumnSelector(fieldName), + resolution, + lowerLimit, + upperLimit + ); + } + + @Override + public Comparator getComparator() + { + return ApproximateHistogramAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return ApproximateHistogramAggregator.combineHistograms(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new ApproximateHistogramAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList( + new ApproximateHistogramAggregatorFactory( + fieldName, + fieldName, + resolution, + numBuckets, + lowerLimit, + upperLimit + ) + ); + } + + @Override + public Object deserialize(Object object) + { + if (object instanceof byte[]) { + final ApproximateHistogram ah = ApproximateHistogram.fromBytes((byte[]) object); + ah.setLowerLimit(lowerLimit); + ah.setUpperLimit(upperLimit); + + return ah; + } else if (object instanceof ByteBuffer) { + final ApproximateHistogram ah = ApproximateHistogram.fromBytes((ByteBuffer) object); + ah.setLowerLimit(lowerLimit); + ah.setUpperLimit(upperLimit); + + return ah; + } else if (object instanceof String) { + byte[] bytes = Base64.decodeBase64(((String) object).getBytes(Charsets.UTF_8)); + final ApproximateHistogram ah = ApproximateHistogram.fromBytes(bytes); + ah.setLowerLimit(lowerLimit); + ah.setUpperLimit(upperLimit); + + return ah; + } else { + return object; + } + } + + @Override + public Object finalizeComputation(Object object) + { + return ((ApproximateHistogram) object).toHistogram(numBuckets); + } + + @JsonProperty + @Override + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @JsonProperty + public int getResolution() + { + return resolution; + } + + @JsonProperty + public float getLowerLimit() + { + return lowerLimit; + } + + @JsonProperty + public float getUpperLimit() + { + return upperLimit; + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); + return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2) + .put(CACHE_TYPE_ID) + .put(fieldNameBytes) + .putInt(resolution) + .putInt(numBuckets) + .putFloat(lowerLimit) + .putFloat(upperLimit).array(); + } + + @Override + public String getTypeName() + { + return "approximateHistogram"; + } + + @Override + public int getMaxIntermediateSize() + { + return new ApproximateHistogram(resolution).getMaxStorageSize(); + } + + @Override + public Object getAggregatorStartValue() + { + return new ApproximateHistogram(resolution); + } + + @Override + public String toString() + { + return "ApproximateHistogramAggregatorFactory{" + + "name='" + name + '\'' + + ", fieldName='" + fieldName + '\'' + + ", resolution=" + resolution + + ", numBuckets=" + numBuckets + + ", lowerLimit=" + lowerLimit + + ", upperLimit=" + upperLimit + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java new file mode 100644 index 00000000000..dbd566f2693 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java @@ -0,0 +1,95 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.FloatColumnSelector; + +import java.nio.ByteBuffer; + +public class ApproximateHistogramBufferAggregator implements BufferAggregator +{ + private final FloatColumnSelector selector; + private final int resolution; + private final float lowerLimit; + private final float upperLimit; + + public ApproximateHistogramBufferAggregator(FloatColumnSelector selector, int resolution, float lowerLimit, float upperLimit) + { + this.selector = selector; + this.resolution = resolution; + this.lowerLimit = lowerLimit; + this.upperLimit = upperLimit; + } + + @Override + public void init(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + mutationBuffer.putInt(resolution); + mutationBuffer.putInt(0); //initial binCount + for (int i = 0; i < resolution; ++i) { + mutationBuffer.putFloat(0f); + } + for (int i = 0; i < resolution; ++i) { + mutationBuffer.putLong(0L); + } + + // min + mutationBuffer.putFloat(Float.POSITIVE_INFINITY); + // max + mutationBuffer.putFloat(Float.NEGATIVE_INFINITY); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + ApproximateHistogram h0 = ApproximateHistogram.fromBytesDense(mutationBuffer); + h0.offer(selector.get()); + + mutationBuffer.position(position); + h0.toBytesDense(mutationBuffer); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + return ApproximateHistogram.fromBytes(mutationBuffer); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getFloat()"); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java new file mode 100644 index 00000000000..592ca2c8d4b --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramDruidModule.java @@ -0,0 +1,60 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; +import io.druid.segment.serde.ComplexMetrics; + +import java.util.List; + +/** + */ +public class ApproximateHistogramDruidModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule().registerSubtypes( + ApproximateHistogramFoldingAggregatorFactory.class, + ApproximateHistogramAggregatorFactory.class, + EqualBucketsPostAggregator.class, + CustomBucketsPostAggregator.class, + BucketsPostAggregator.class, + QuantilesPostAggregator.class, + QuantilePostAggregator.class, + MinPostAggregator.class, + MaxPostAggregator.class + ) + ); + } + + @Override + public void configure(Binder binder) + { + if (ComplexMetrics.getSerdeForType("approximateHistogram") == null) { + ComplexMetrics.registerSerde("approximateHistogram", new ApproximateHistogramFoldingSerde()); + } + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java new file mode 100644 index 00000000000..51dc682de0a --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java @@ -0,0 +1,101 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + + +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.ObjectColumnSelector; + +public class ApproximateHistogramFoldingAggregator implements Aggregator +{ + private final String name; + private final ObjectColumnSelector selector; + private final int resolution; + private final float lowerLimit; + private final float upperLimit; + + private ApproximateHistogram histogram; + private float[] tmpBufferP; + private long[] tmpBufferB; + + public ApproximateHistogramFoldingAggregator( + String name, + ObjectColumnSelector selector, + int resolution, + float lowerLimit, + float upperLimit + ) + { + this.name = name; + this.selector = selector; + this.resolution = resolution; + this.lowerLimit = lowerLimit; + this.upperLimit = upperLimit; + this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit); + + tmpBufferP = new float[resolution]; + tmpBufferB = new long[resolution]; + } + + @Override + public void aggregate() + { + ApproximateHistogram h = selector.get(); + if (h == null) { + return; + } + + if (h.binCount() + histogram.binCount() <= tmpBufferB.length) { + histogram.foldFast(h, tmpBufferP, tmpBufferB); + } else { + histogram.foldFast(h); + } + } + + @Override + public void reset() + { + this.histogram = new ApproximateHistogram(resolution, lowerLimit, upperLimit); + } + + @Override + public Object get() + { + return histogram; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getFloat()"); + } + + @Override + public String getName() + { + return name; + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java new file mode 100644 index 00000000000..c5bb1e552c5 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java @@ -0,0 +1,164 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Charsets; +import com.google.common.primitives.Floats; +import com.google.common.primitives.Ints; +import com.metamx.common.IAE; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.ObjectColumnSelector; + +import java.nio.ByteBuffer; + +@JsonTypeName("approxHistogramFold") +public class ApproximateHistogramFoldingAggregatorFactory extends ApproximateHistogramAggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x9; + + @JsonCreator + public ApproximateHistogramFoldingAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("resolution") Integer resolution, + @JsonProperty("numBuckets") Integer numBuckets, + @JsonProperty("lowerLimit") Float lowerLimit, + @JsonProperty("upperLimit") Float upperLimit + ) + { + super(name, fieldName, resolution, numBuckets, lowerLimit, upperLimit); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + + if (selector == null) { + // gracefully handle undefined metrics + + selector = new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return ApproximateHistogram.class; + } + + @Override + public ApproximateHistogram get() + { + return new ApproximateHistogram(0); + } + }; + } + + if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) { + return new ApproximateHistogramFoldingAggregator( + name, + selector, + resolution, + lowerLimit, + upperLimit + ); + } + + throw new IAE( + "Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s", + fieldName, + selector.classOfObject() + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); + + if (selector == null) { + // gracefully handle undefined metrics + + selector = new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return ApproximateHistogram.class; + } + + @Override + public ApproximateHistogram get() + { + return new ApproximateHistogram(0); + } + }; + } + + if (ApproximateHistogram.class.isAssignableFrom(selector.classOfObject())) { + return new ApproximateHistogramFoldingBufferAggregator(selector, resolution, lowerLimit, upperLimit); + } + + throw new IAE( + "Incompatible type for metric[%s], expected a ApproximateHistogram, got a %s", + fieldName, + selector.classOfObject() + ); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new ApproximateHistogramFoldingAggregatorFactory(name, name, resolution, numBuckets, lowerLimit, upperLimit); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = fieldName.getBytes(Charsets.UTF_8); + return ByteBuffer.allocate(1 + fieldNameBytes.length + Ints.BYTES * 2 + Floats.BYTES * 2) + .put(CACHE_TYPE_ID) + .put(fieldNameBytes) + .putInt(resolution) + .putInt(numBuckets) + .putFloat(lowerLimit) + .putFloat(upperLimit) + .array(); + } + + @Override + public String toString() + { + return "ApproximateHistogramFoldingAggregatorFactory{" + + "name='" + name + '\'' + + ", fieldName='" + fieldName + '\'' + + ", resolution=" + resolution + + ", numBuckets=" + numBuckets + + ", lowerLimit=" + lowerLimit + + ", upperLimit=" + upperLimit + + '}'; + } +} + diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java new file mode 100644 index 00000000000..4190ae50a40 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java @@ -0,0 +1,99 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.ObjectColumnSelector; + +import java.nio.ByteBuffer; + +public class ApproximateHistogramFoldingBufferAggregator implements BufferAggregator +{ + private final ObjectColumnSelector selector; + private final int resolution; + private final float upperLimit; + private final float lowerLimit; + + private float[] tmpBufferP; + private long[] tmpBufferB; + + public ApproximateHistogramFoldingBufferAggregator( + ObjectColumnSelector selector, + int resolution, + float lowerLimit, + float upperLimit + ) + { + this.selector = selector; + this.resolution = resolution; + this.lowerLimit = lowerLimit; + this.upperLimit = upperLimit; + + tmpBufferP = new float[resolution]; + tmpBufferB = new long[resolution]; + } + + @Override + public void init(ByteBuffer buf, int position) + { + ApproximateHistogram h = new ApproximateHistogram(resolution, lowerLimit, upperLimit); + + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + // use dense storage for aggregation + h.toBytesDense(mutationBuffer); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + ApproximateHistogram h0 = ApproximateHistogram.fromBytesDense(mutationBuffer); + h0.setLowerLimit(lowerLimit); + h0.setUpperLimit(upperLimit); + ApproximateHistogram hNext = selector.get(); + h0.foldFast(hNext, tmpBufferP, tmpBufferB); + + mutationBuffer.position(position); + h0.toBytesDense(mutationBuffer); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.asReadOnlyBuffer(); + mutationBuffer.position(position); + return ApproximateHistogram.fromBytesDense(mutationBuffer); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getFloat()"); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java new file mode 100644 index 00000000000..196a1e50548 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java @@ -0,0 +1,133 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.google.common.collect.Ordering; +import io.druid.data.input.InputRow; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.serde.ColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSerde; +import io.druid.segment.serde.ComplexColumnPartSupplier; +import io.druid.segment.serde.ComplexMetricExtractor; +import io.druid.segment.serde.ComplexMetricSerde; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; + +public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde +{ + private static Ordering comparator = new Ordering() + { + @Override + public int compare( + ApproximateHistogram arg1, ApproximateHistogram arg2 + ) + { + return ApproximateHistogramAggregator.COMPARATOR.compare(arg1, arg2); + } + }.nullsFirst(); + + @Override + public String getTypeName() + { + return "approximateHistogram"; + } + + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + @Override + public Class extractedClass() + { + return ApproximateHistogram.class; + } + + @Override + public ApproximateHistogram extractValue(InputRow inputRow, String metricName) + { + List dimValues = inputRow.getDimension(metricName); + if (dimValues != null && dimValues.size() > 0) { + Iterator values = dimValues.iterator(); + + ApproximateHistogram h = new ApproximateHistogram(); + + while (values.hasNext()) { + float value = Float.parseFloat(values.next()); + h.offer(value); + } + return h; + } else { + return new ApproximateHistogram(0); + } + } + }; + } + + @Override + public ColumnPartSerde deserializeColumn( + ByteBuffer byteBuffer, ColumnBuilder columnBuilder + ) + { + final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy()); + + columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); + + return new ComplexColumnPartSerde(column, getTypeName()); + } + + public ObjectStrategy getObjectStrategy() + { + return new ObjectStrategy() + { + @Override + public Class getClazz() + { + return ApproximateHistogram.class; + } + + @Override + public ApproximateHistogram fromByteBuffer(ByteBuffer buffer, int numBytes) + { + buffer.limit(buffer.position() + numBytes); + return ApproximateHistogram.fromBytes(buffer); + } + + @Override + public byte[] toBytes(ApproximateHistogram h) + { + if (h == null) { + return new byte[]{}; + } + return h.toBytes(); + } + + @Override + public int compare(ApproximateHistogram o1, ApproximateHistogram o2) + { + return comparator.compare(o1, o2); + } + }; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregator.java new file mode 100644 index 00000000000..92dc4a4ccb3 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregator.java @@ -0,0 +1,68 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.aggregation.PostAggregator; + +import java.util.Comparator; +import java.util.Map; + +public abstract class ApproximateHistogramPostAggregator implements PostAggregator +{ + private static final Comparator COMPARATOR = ApproximateHistogramAggregator.COMPARATOR; + + private final String name; + private final String fieldName; + + public ApproximateHistogramPostAggregator( + String name, + String fieldName + ) + { + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public abstract Object compute(Map values); + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + public abstract String toString(); +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ArrayUtils.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ArrayUtils.java new file mode 100644 index 00000000000..9aba505ea23 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ArrayUtils.java @@ -0,0 +1,58 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +public class ArrayUtils +{ + public static int hashCode(long[] a, int fromIndex, int toIndex) + { + int hashCode = 1; + int i = fromIndex; + while (i < toIndex) { + long v = a[i]; + hashCode = 31 * hashCode + (int) (v ^ (v >>> 32)); + ++i; + } + return hashCode; + } + + public static int hashCode(float[] a, int fromIndex, int toIndex) + { + int hashCode = 1; + int i = fromIndex; + while (i < toIndex) { + hashCode = 31 * hashCode + Float.floatToIntBits(a[i]); + ++i; + } + return hashCode; + } + + public static int hashCode(double[] a, int fromIndex, int toIndex) + { + int hashCode = 1; + int i = fromIndex; + while (i < toIndex) { + long v = Double.doubleToLongBits(a[i]); + hashCode = 31 * hashCode + (int) (v ^ (v >>> 32)); + ++i; + } + return hashCode; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/BucketsPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/BucketsPostAggregator.java new file mode 100644 index 00000000000..f9e6aa9be62 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/BucketsPostAggregator.java @@ -0,0 +1,91 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; +import com.metamx.common.IAE; + +import java.util.Map; +import java.util.Set; + +@JsonTypeName("buckets") +public class BucketsPostAggregator extends ApproximateHistogramPostAggregator +{ + private final float bucketSize; + private final float offset; + + private String fieldName; + + @JsonCreator + public BucketsPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("bucketSize") float bucketSize, + @JsonProperty("offset") float offset + ) + { + super(name, fieldName); + this.bucketSize = bucketSize; + if (this.bucketSize <= 0) { + throw new IAE("Illegal bucketSize [%s], must be > 0", this.bucketSize); + } + this.offset = offset; + this.fieldName = fieldName; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + return ah.toHistogram(bucketSize, offset); + } + + @JsonProperty + public float getBucketSize() + { + return bucketSize; + } + + @JsonProperty + public float getOffset() + { + return bucketSize; + } + + @Override + public String toString() + { + return "BucketsPostAggregator{" + + "name='" + this.getName() + '\'' + + ", fieldName='" + this.getFieldName() + '\'' + + ", bucketSize=" + this.getBucketSize() + + ", offset=" + this.getOffset() + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/BufferUtils.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/BufferUtils.java new file mode 100644 index 00000000000..823e6c59e36 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/BufferUtils.java @@ -0,0 +1,68 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import java.nio.DoubleBuffer; +import java.nio.FloatBuffer; + +public class BufferUtils +{ + public static int binarySearch(DoubleBuffer buf, int minIndex, int maxIndex, double value) + { + while (minIndex < maxIndex) { + int currIndex = (minIndex + maxIndex - 1) >>> 1; + + double currValue = buf.get(currIndex); + int comparison = Double.compare(currValue, value); + if (comparison == 0) { + return currIndex; + } + + if (comparison < 0) { + minIndex = currIndex + 1; + } else { + maxIndex = currIndex; + } + } + + return -(minIndex + 1); + } + + public static int binarySearch(FloatBuffer buf, int minIndex, int maxIndex, float value) + { + while (minIndex < maxIndex) { + int currIndex = (minIndex + maxIndex - 1) >>> 1; + + float currValue = buf.get(currIndex); + int comparison = Float.compare(currValue, value); + if (comparison == 0) { + return currIndex; + } + + if (comparison < 0) { + minIndex = currIndex + 1; + } else { + maxIndex = currIndex; + } + } + + return -(minIndex + 1); + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java new file mode 100644 index 00000000000..8a34eaa2bdc --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java @@ -0,0 +1,77 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; + +import java.util.Arrays; +import java.util.Map; +import java.util.Set; + +@JsonTypeName("customBuckets") +public class CustomBucketsPostAggregator extends ApproximateHistogramPostAggregator +{ + private final float[] breaks; + private String fieldName; + + @JsonCreator + public CustomBucketsPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("breaks") float[] breaks + ) + { + super(name, fieldName); + this.breaks = breaks; + this.fieldName = fieldName; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + return ah.toHistogram(breaks); + } + + @JsonProperty + public float[] getBreaks() + { + return breaks; + } + + @Override + public String toString() + { + return "CustomBucketsPostAggregator{" + + "name='" + this.getName() + '\'' + + ", fieldName='" + this.getFieldName() + '\'' + + ", breaks=" + Arrays.toString(this.getBreaks()) + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java new file mode 100644 index 00000000000..ecdb791ea26 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java @@ -0,0 +1,80 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; +import com.metamx.common.IAE; + +import java.util.Map; +import java.util.Set; + +@JsonTypeName("equalBuckets") +public class EqualBucketsPostAggregator extends ApproximateHistogramPostAggregator +{ + private final int numBuckets; + private String fieldName; + + @JsonCreator + public EqualBucketsPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("numBuckets") int numBuckets + ) + { + super(name, fieldName); + this.numBuckets = numBuckets; + if (this.numBuckets <= 1) { + throw new IAE("Illegal number of buckets[%s], must be > 1", this.numBuckets); + } + this.fieldName = fieldName; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + return ah.toHistogram(numBuckets); + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + + @Override + public String toString() + { + return "EqualBucketsPostAggregator{" + + "name='" + this.getName() + '\'' + + ", fieldName='" + this.getFieldName() + '\'' + + ", numBuckets=" + this.getNumBuckets() + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java new file mode 100644 index 00000000000..384e6eb1fbf --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/Histogram.java @@ -0,0 +1,88 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Arrays; + +public class Histogram +{ + double[] breaks; + double[] counts; + + public Histogram(float[] breaks, double[] counts) + { + double[] retVal = new double[breaks.length]; + for (int i = 0; i < breaks.length; ++i) { + retVal[i] = (double) breaks[i]; + } + + this.breaks = retVal; + this.counts = counts; + } + + @JsonProperty + public double[] getBreaks() + { + return breaks; + } + + @JsonProperty + public double[] getCounts() + { + return counts; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Histogram that = (Histogram) o; + + if (!Arrays.equals(this.getBreaks(), that.getBreaks())) { + return false; + } + if (!Arrays.equals(this.getCounts(), that.getCounts())) { + return false; + } + return true; + } + + @Override + public int hashCode() + { + int result = (this.getBreaks() != null ? ArrayUtils.hashCode(this.getBreaks(), 0, this.getBreaks().length) : 0); + result = 31 * result + (this.getCounts() != null ? ArrayUtils.hashCode( + this.getCounts(), + 0, + this.getCounts().length + ) : 0); + return result; + } + + +} \ No newline at end of file diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/MaxPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/MaxPostAggregator.java new file mode 100644 index 00000000000..b0a0ead93d6 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/MaxPostAggregator.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; + +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +@JsonTypeName("max") +public class MaxPostAggregator extends ApproximateHistogramPostAggregator +{ + static final Comparator COMPARATOR = new Comparator() + { + @Override + public int compare(Object o, Object o1) + { + return Double.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue()); + } + }; + + private String fieldName; + + @JsonCreator + public MaxPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, fieldName); + this.fieldName = fieldName; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + return ah.getMax(); + } + + @Override + public String toString() + { + return "QuantilePostAggregator{" + + "fieldName='" + fieldName + '\'' + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/MinPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/MinPostAggregator.java new file mode 100644 index 00000000000..d986901b89f --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/MinPostAggregator.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; + +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +@JsonTypeName("min") +public class MinPostAggregator extends ApproximateHistogramPostAggregator +{ + static final Comparator COMPARATOR = new Comparator() + { + @Override + public int compare(Object o, Object o1) + { + return Double.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue()); + } + }; + + private String fieldName; + + @JsonCreator + public MinPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, fieldName); + this.fieldName = fieldName; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + return ah.getMin(); + } + + @Override + public String toString() + { + return "QuantilePostAggregator{" + + "fieldName='" + fieldName + '\'' + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java new file mode 100644 index 00000000000..7a06a906c89 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilePostAggregator.java @@ -0,0 +1,96 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; +import com.metamx.common.IAE; + +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +@JsonTypeName("quantile") +public class QuantilePostAggregator extends ApproximateHistogramPostAggregator +{ + static final Comparator COMPARATOR = new Comparator() + { + @Override + public int compare(Object o, Object o1) + { + return Double.compare(((Number) o).doubleValue(), ((Number) o1).doubleValue()); + } + }; + + private final float probability; + private String fieldName; + + @JsonCreator + public QuantilePostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("probability") float probability + ) + { + super(name, fieldName); + this.probability = probability; + this.fieldName = fieldName; + + if (probability < 0 | probability > 1) { + throw new IAE("Illegal probability[%s], must be strictly between 0 and 1", probability); + } + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + return ah.getQuantiles(new float[]{this.getProbability()})[0]; + } + + @JsonProperty + public float getProbability() + { + return probability; + } + + @Override + public String toString() + { + return "QuantilePostAggregator{" + + "probability=" + probability + + ", fieldName='" + fieldName + '\'' + + '}'; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/Quantiles.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/Quantiles.java new file mode 100644 index 00000000000..c9471bc3e0b --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/Quantiles.java @@ -0,0 +1,111 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Arrays; + +@JsonTypeName("quantiles") +public class Quantiles +{ + float[] probabilities; + float[] quantiles; + float min; + float max; + + @JsonCreator + public Quantiles( + @JsonProperty("probabilities") float[] probabilities, + @JsonProperty("quantiles") float[] quantiles, + @JsonProperty("min") float min, + @JsonProperty("max") float max + ) + { + this.probabilities = probabilities; + this.quantiles = quantiles; + this.min = min; + this.max = max; + } + + @JsonProperty + public float[] getProbabilities() + { + return probabilities; + } + + @JsonProperty + public float[] getQuantiles() + { + return quantiles; + } + + @JsonProperty + public float getMin() + { + return min; + } + + @JsonProperty + public float getMax() + { + return max; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Quantiles quantiles1 = (Quantiles) o; + + if (Float.compare(quantiles1.max, max) != 0) { + return false; + } + if (Float.compare(quantiles1.min, min) != 0) { + return false; + } + if (!Arrays.equals(probabilities, quantiles1.probabilities)) { + return false; + } + if (!Arrays.equals(quantiles, quantiles1.quantiles)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + int result = probabilities != null ? Arrays.hashCode(probabilities) : 0; + result = 31 * result + (quantiles != null ? Arrays.hashCode(quantiles) : 0); + result = 31 * result + (min != +0.0f ? Float.floatToIntBits(min) : 0); + result = 31 * result + (max != +0.0f ? Float.floatToIntBits(max) : 0); + return result; + } +} diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilesPostAggregator.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilesPostAggregator.java new file mode 100644 index 00000000000..181861e5f80 --- /dev/null +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/QuantilesPostAggregator.java @@ -0,0 +1,92 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; +import com.metamx.common.IAE; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.Set; + +@JsonTypeName("quantiles") +public class QuantilesPostAggregator extends ApproximateHistogramPostAggregator +{ + private final float[] probabilities; + private String fieldName; + + @JsonCreator + public QuantilesPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty("probabilities") float[] probabilities + ) + { + super(name, fieldName); + this.probabilities = probabilities; + this.fieldName = fieldName; + + for (float p : probabilities) { + if (p < 0 | p > 1) { + throw new IAE("Illegal probability[%s], must be strictly between 0 and 1", p); + } + } + } + + @Override + public Comparator getComparator() + { + throw new UnsupportedOperationException(); + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Object compute(Map values) + { + final ApproximateHistogram ah = (ApproximateHistogram) values.get(this.getFieldName()); + + return new Quantiles(this.getProbabilities(), ah.getQuantiles(this.getProbabilities()), ah.getMin(), ah.getMax()); + } + + @JsonProperty + public float[] getProbabilities() + { + return probabilities; + } + + @Override + public String toString() + { + return "EqualBucketsPostAggregator{" + + "name='" + this.getName() + '\'' + + ", fieldName='" + this.getFieldName() + '\'' + + ", probabilities=" + Arrays.toString(this.getProbabilities()) + + '}'; + } +} diff --git a/histogram/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/histogram/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 00000000000..d39951f0cd8 --- /dev/null +++ b/histogram/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.query.aggregation.histogram.ApproximateHistogramDruidModule \ No newline at end of file diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java new file mode 100644 index 00000000000..353f9e73ad7 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java @@ -0,0 +1,76 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.TestFloatColumnSelector; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; + +public class ApproximateHistogramAggregatorTest +{ + private void aggregateBuffer(TestFloatColumnSelector selector, BufferAggregator agg, ByteBuffer buf, int position) + { + agg.aggregate(buf, position); + selector.increment(); + } + + @Test + public void testBufferAggregate() throws Exception + { + final float[] values = {23, 19, 10, 16, 36, 2, 9, 32, 30, 45}; + final int resolution = 5; + final int numBuckets = 5; + + final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); + + ApproximateHistogramAggregatorFactory factory = new ApproximateHistogramAggregatorFactory( + "billy", "billy", resolution, numBuckets, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY + ); + ApproximateHistogramBufferAggregator agg = new ApproximateHistogramBufferAggregator(selector, resolution, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY); + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + int position = 0; + + agg.init(buf, position); + for (int i = 0; i < values.length; i++) { + aggregateBuffer(selector, agg, buf, position); + } + + ApproximateHistogram h = ((ApproximateHistogram) agg.get(buf, position)); + + Assert.assertArrayEquals( + "final bin positions don't match expected positions", + new float[]{2, 9.5f, 19.33f, 32.67f, 45f}, h.positions, 0.01f + ); + + Assert.assertArrayEquals( + "final bin counts don't match expected counts", + new long[]{1, 2, 3, 3, 1}, h.bins() + ); + + Assert.assertEquals("getMin value doesn't match expected getMin", 2, h.min(), 0); + Assert.assertEquals("getMax value doesn't match expected getMax", 45, h.max(), 0); + + Assert.assertEquals("bin count doesn't match expected bin count", 5, h.binCount()); + } +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java new file mode 100644 index 00000000000..33621dc2144 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramErrorBenchmark.java @@ -0,0 +1,191 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.google.common.primitives.Floats; +import io.druid.query.aggregation.Histogram; + +import java.util.Arrays; +import java.util.Random; + +public class ApproximateHistogramErrorBenchmark +{ + private boolean debug = true; + private int numBuckets = 20; + private int numBreaks = numBuckets + 1; + private int numPerHist = 50; + private int numHists = 10; + private int resolution = 50; + private int combinedResolution = 100; + private Random rand = new Random(2); + + public ApproximateHistogramErrorBenchmark setDebug(boolean debug) + { + this.debug = debug; + return this; + } + + public ApproximateHistogramErrorBenchmark setNumBuckets(int numBuckets) + { + this.numBuckets = numBuckets; + return this; + } + + public ApproximateHistogramErrorBenchmark setNumBreaks(int numBreaks) + { + this.numBreaks = numBreaks; + return this; + } + + public ApproximateHistogramErrorBenchmark setNumPerHist(int numPerHist) + { + this.numPerHist = numPerHist; + return this; + } + + public ApproximateHistogramErrorBenchmark setNumHists(int numHists) + { + this.numHists = numHists; + return this; + } + + public ApproximateHistogramErrorBenchmark setResolution(int resolution) + { + this.resolution = resolution; + return this; + } + + public ApproximateHistogramErrorBenchmark setCombinedResolution(int combinedResolution) + { + this.combinedResolution = combinedResolution; + return this; + } + + + public static void main(String[] args) + { + ApproximateHistogramErrorBenchmark approxHist = new ApproximateHistogramErrorBenchmark(); + System.out.println( + Arrays.toString( + approxHist.setDebug(true) + .setNumPerHist(50) + .setNumHists(10000) + .setResolution(50) + .setCombinedResolution(100) + .getErrors() + ) + ); + + + ApproximateHistogramErrorBenchmark approxHist2 = new ApproximateHistogramErrorBenchmark(); + int[] numHistsArray = new int[]{10, 100, 1000, 10000, 100000}; + float[] errs1 = new float[numHistsArray.length]; + float[] errs2 = new float[numHistsArray.length]; + for (int i = 0; i < numHistsArray.length; ++i) { + float[] tmp = approxHist2.setDebug(false).setNumHists(numHistsArray[i]).setCombinedResolution(100).getErrors(); + errs1[i] = tmp[0]; + errs2[i] = tmp[1]; + } + + System.out + .format("Number of histograms for folding : %s \n", Arrays.toString(numHistsArray)); + System.out.format("Errors for approximate histogram : %s \n", Arrays.toString(errs1)); + System.out.format("Errors for approximate histogram, ruleFold : %s \n", Arrays.toString(errs2)); + } + + private float[] getErrors() + { + final int numValues = numHists * numPerHist; + final float[] values = new float[numValues]; + + for (int i = 0; i < numValues; ++i) { + values[i] = (float) rand.nextGaussian(); + } + + float min = Floats.min(values); + min = (float) (min < 0 ? 1.02 : .98) * min; + float max = Floats.max(values); + max = (float) (max < 0 ? .98 : 1.02) * max; + final float stride = (max - min) / numBuckets; + final float[] breaks = new float[numBreaks]; + for (int i = 0; i < numBreaks; i++) { + breaks[i] = min + stride * i; + } + + Histogram h = new Histogram(breaks); + for (float v : values) { + h.offer(v); + } + double[] hcounts = h.asVisual().counts; + + ApproximateHistogram ah1 = new ApproximateHistogram(resolution); + ApproximateHistogram ah2 = new ApproximateHistogram(combinedResolution); + ApproximateHistogram tmp = new ApproximateHistogram(resolution); + for (int i = 0; i < numValues; ++i) { + tmp.offer(values[i]); + if ((i + 1) % numPerHist == 0) { + ah1.fold(tmp); + ah2.foldRule(tmp, null, null); + tmp = new ApproximateHistogram(resolution); + } + } + double[] ahcounts1 = ah1.toHistogram(breaks).getCounts(); + double[] ahcounts2 = ah2.toHistogram(breaks).getCounts(); + + float err1 = 0; + float err2 = 0; + for (int j = 0; j < hcounts.length; j++) { + err1 += Math.abs((hcounts[j] - ahcounts1[j]) / numValues); + err2 += Math.abs((hcounts[j] - ahcounts2[j]) / numValues); + } + + if (debug) { + float sum = 0; + for (double v : hcounts) { + sum += v; + } + System.out.println("Exact Histogram Sum:"); + System.out.println(sum); + sum = 0; + for (double v : ahcounts1) { + sum += v; + } + System.out.println("Approximate Histogram Sum:"); + System.out.println(sum); + sum = 0; + for (double v : ahcounts2) { + sum += v; + } + System.out.println("Approximate Histogram Rule Fold Sum:"); + System.out.println(sum); + System.out.println("Exact Histogram:"); + System.out.println(h.asVisual()); + System.out.println("Approximate Histogram:"); + System.out.println(ah1.toHistogram(breaks)); + System.out.println("Approximate Histogram Rule Fold:"); + System.out.println(ah2.toHistogram(breaks)); + System.out.format("Error for approximate histogram: %s \n", err1); + System.out.format("Error for approximate histogram, ruleFold: %s \n", err2); + System.out.format("Error ratio for AHRF: %s \n", err2 / err1); + } + return new float[]{err1, err2, err2 / err1}; + } + +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java new file mode 100644 index 00000000000..f8484494b2d --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java @@ -0,0 +1,65 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import io.druid.query.aggregation.TestFloatColumnSelector; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class ApproximateHistogramPostAggregatorTest +{ + static final float[] VALUES = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + + protected ApproximateHistogram buildHistogram(int size, float[] values) + { + ApproximateHistogram h = new ApproximateHistogram(size); + for (float v : values) { + h.offer(v); + } + return h; + } + + @Test + public void testCompute() + { + ApproximateHistogram ah = buildHistogram(10, VALUES); + final TestFloatColumnSelector selector = new TestFloatColumnSelector(VALUES); + + ApproximateHistogramAggregator agg = new ApproximateHistogramAggregator("price", selector, 10, Float.NEGATIVE_INFINITY, Float.POSITIVE_INFINITY); + for (int i = 0; i < VALUES.length; i++) { + agg.aggregate(); + selector.increment(); + } + + Map metricValues = new HashMap(); + metricValues.put(agg.getName(), agg.get()); + + ApproximateHistogramPostAggregator approximateHistogramPostAggregator = new EqualBucketsPostAggregator( + "approxHist", + "price", + 5 + ); + Assert.assertEquals(ah.toHistogram(5), approximateHistogramPostAggregator.compute(metricValues)); + } + +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java new file mode 100644 index 00000000000..dd4406c5076 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramQueryTest.java @@ -0,0 +1,247 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import io.druid.collections.StupidPool; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.Result; +import io.druid.query.TestQueryRunners; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryBuilder; +import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryQueryToolChest; +import io.druid.query.topn.TopNQueryRunnerFactory; +import io.druid.query.topn.TopNResultValue; +import io.druid.segment.TestHelper; +import org.joda.time.DateTime; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class ApproximateHistogramQueryTest +{ + + private final QueryRunner runner; + + public ApproximateHistogramQueryTest( + QueryRunner runner + ) + { + this.runner = runner; + } + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + List retVal = Lists.newArrayList(); + retVal.addAll( + QueryRunnerTestHelper.makeQueryRunners( + new TopNQueryRunnerFactory( + TestQueryRunners.getPool(), + new TopNQueryQueryToolChest(new TopNQueryConfig()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + ); + retVal.addAll( + QueryRunnerTestHelper.makeQueryRunners( + new TopNQueryRunnerFactory( + new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(2000); + } + } + ), + new TopNQueryQueryToolChest(new TopNQueryConfig()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + ); + + return retVal; + } + + @Test + public void testTopNWithApproximateHistogramAgg() + { + ApproximateHistogramAggregatorFactory factory = new ApproximateHistogramAggregatorFactory( + "apphisto", + "index", + 10, + 5, + Float.NEGATIVE_INFINITY, + Float.POSITIVE_INFINITY + ); + + TopNQuery query = new TopNQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .dimension(QueryRunnerTestHelper.providerDimension) + .metric(QueryRunnerTestHelper.dependentPostAggMetric) + .threshold(4) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonAggregators, + Lists.newArrayList( + new MaxAggregatorFactory("maxIndex", "index"), + new MinAggregatorFactory("minIndex", "index"), + factory + ) + ) + ) + ) + .postAggregators( + Arrays.asList( + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + new QuantilePostAggregator("quantile", "apphisto", 0.5f) + ) + ) + .build(); + + List> expectedResults = Arrays.asList( + new Result( + new DateTime("2011-01-12T00:00:00.000Z"), + new TopNResultValue( + Arrays.>asList( + ImmutableMap.builder() + .put(QueryRunnerTestHelper.providerDimension, "total_market") + .put("rows", 186L) + .put("index", 215679.82879638672D) + .put("addRowsIndexConstant", 215866.82879638672D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 216053.82879638672D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1743.9217529296875D) + .put("minIndex", 792.3260498046875D) + .put("quantile", 1085.6775f) + .put( + "apphisto", + new Histogram( + new float[]{ + 554.4271240234375f, + 792.3260498046875f, + 1030.2249755859375f, + 1268.1239013671875f, + 1506.0228271484375f, + 1743.9217529296875f + }, + new double[]{ + 0.0D, + 39.42073059082031D, + 103.29110717773438D, + 34.93659591674805D, + 8.351564407348633D + } + ) + ) + .build(), + ImmutableMap.builder() + .put(QueryRunnerTestHelper.providerDimension, "upfront") + .put("rows", 186L) + .put("index", 192046.1060180664D) + .put("addRowsIndexConstant", 192233.1060180664D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 192420.1060180664D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_2) + .put("maxIndex", 1870.06103515625D) + .put("minIndex", 545.9906005859375D) + .put("quantile", 880.9881f) + .put( + "apphisto", + new Histogram( + new float[]{ + 214.97299194335938f, + 545.9906005859375f, + 877.0081787109375f, + 1208.0257568359375f, + 1539.0433349609375f, + 1870.06103515625f + }, + new double[]{ + 0.0D, + 67.53287506103516D, + 72.22068786621094D, + 31.984678268432617D, + 14.261756896972656D + } + ) + ) + .build(), + ImmutableMap.builder() + .put(QueryRunnerTestHelper.providerDimension, "spot") + .put("rows", 837L) + .put("index", 95606.57232284546D) + .put("addRowsIndexConstant", 96444.57232284546D) + .put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D) + .put("uniques", QueryRunnerTestHelper.UNIQUES_9) + .put("maxIndex", 277.2735290527344D) + .put("minIndex", 59.02102279663086D) + .put("quantile", 101.78856f) + .put( + "apphisto", + new Histogram( + new float[]{ + 4.457897186279297f, + 59.02102279663086f, + 113.58415222167969f, + 168.14727783203125f, + 222.7104034423828f, + 277.2735290527344f + }, + new double[]{ + 0.0D, + 462.4309997558594D, + 357.5404968261719D, + 15.022850036621094D, + 2.0056631565093994D + } + ) + ) + .build() + ) + ) + ) + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query)); + } +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java new file mode 100644 index 00000000000..a0d8bda7ec5 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTest.java @@ -0,0 +1,588 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +public class ApproximateHistogramTest +{ + static final float[] VALUES = {23, 19, 10, 16, 36, 2, 9, 32, 30, 45}; + static final float[] VALUES2 = {23, 19, 10, 16, 36, 2, 1, 9, 32, 30, 45, 46}; + + static final float[] VALUES3 = { + 20, 16, 19, 27, 17, 20, 18, 20, 28, 14, 17, 21, 20, 21, 10, 25, 23, 17, 21, 18, + 14, 20, 18, 12, 19, 20, 23, 25, 15, 22, 14, 17, 15, 23, 23, 15, 27, 20, 17, 15 + }; + static final float[] VALUES4 = { + 27.489f, 3.085f, 3.722f, 66.875f, 30.998f, -8.193f, 5.395f, 5.109f, 10.944f, 54.75f, + 14.092f, 15.604f, 52.856f, 66.034f, 22.004f, -14.682f, -50.985f, 2.872f, 61.013f, + -21.766f, 19.172f, 62.882f, 33.537f, 21.081f, 67.115f, 44.789f, 64.1f, 20.911f, + -6.553f, 2.178f + }; + static final float[] VALUES5 = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + static final float[] VALUES6 = {1f, 1.5f, 2f, 2.5f, 3f, 3.5f, 4f, 4.5f, 5f, 5.5f, 6f, 6.5f, 7f, 7.5f, 8f, 8.5f, 9f, 9.5f, 10f}; + + protected ApproximateHistogram buildHistogram(int size, float[] values) + { + ApproximateHistogram h = new ApproximateHistogram(size); + for (float v : values) { + h.offer(v); + } + return h; + } + + protected ApproximateHistogram buildHistogram(int size, float[] values, float lowerLimit, float upperLimit) + { + ApproximateHistogram h = new ApproximateHistogram(size, lowerLimit, upperLimit); + for (float v : values) { + h.offer(v); + } + return h; + } + + @Test + public void testOffer() throws Exception + { + ApproximateHistogram h = buildHistogram(5, VALUES); + + // (2, 1), (9.5, 2), (19.33, 3), (32.67, 3), (45, 1) + Assert.assertArrayEquals( + "final bin positions match expected positions", + new float[]{2, 9.5f, 19.33f, 32.67f, 45f}, h.positions(), 0.1f + ); + + Assert.assertArrayEquals( + "final bin positions match expected positions", + new long[]{1, 2, 3, 3, 1}, h.bins() + ); + + Assert.assertEquals("min value matches expexted min", 2, h.min(), 0); + Assert.assertEquals("max value matches expexted max", 45, h.max(), 0); + + Assert.assertEquals("bin count matches expected bin count", 5, h.binCount()); + } + + @Test + public void testFold() + { + ApproximateHistogram merged = new ApproximateHistogram(0); + ApproximateHistogram mergedFast = new ApproximateHistogram(0); + ApproximateHistogram h1 = new ApproximateHistogram(5); + ApproximateHistogram h2 = new ApproximateHistogram(10); + + for (int i = 0; i < 5; ++i) { + h1.offer(VALUES[i]); + } + for (int i = 5; i < VALUES.length; ++i) { + h2.offer(VALUES[i]); + } + + merged.fold(h1); + merged.fold(h2); + mergedFast.foldFast(h1); + mergedFast.foldFast(h2); + + Assert.assertArrayEquals( + "final bin positions match expected positions", + new float[]{2, 9.5f, 19.33f, 32.67f, 45f}, merged.positions(), 0.1f + ); + Assert.assertArrayEquals( + "final bin positions match expected positions", + new float[]{11.2f, 30.25f, 45f}, mergedFast.positions(), 0.1f + ); + + Assert.assertArrayEquals( + "final bin counts match expected counts", + new long[]{1, 2, 3, 3, 1}, merged.bins() + ); + Assert.assertArrayEquals( + "final bin counts match expected counts", + new long[]{5, 4, 1}, mergedFast.bins() + ); + + Assert.assertEquals("merged max matches expected value", 45f, merged.max(), 0.1f); + Assert.assertEquals("mergedfast max matches expected value", 45f, mergedFast.max(), 0.1f); + Assert.assertEquals("merged min matches expected value", 2f, merged.min(), 0.1f); + Assert.assertEquals("mergedfast min matches expected value", 2f, mergedFast.min(), 0.1f); + + // fold where merged bincount is less than total bincount + ApproximateHistogram a = buildHistogram(10, new float[]{1, 2, 3, 4, 5, 6}); + ApproximateHistogram aFast = buildHistogram(10, new float[]{1, 2, 3, 4, 5, 6}); + ApproximateHistogram b = buildHistogram(5, new float[]{3, 4, 5, 6}); + + a.fold(b); + aFast.foldFast(b); + + Assert.assertEquals( + new ApproximateHistogram( + 6, + new float[]{1, 2, 3, 4, 5, 6, 0, 0, 0, 0}, + new long[]{1, 1, 2, 2, 2, 2, 0, 0, 0, 0}, + 1, 6 + ), a + ); + Assert.assertEquals( + new ApproximateHistogram( + 6, + new float[]{1, 2, 3, 4, 5, 6, 0, 0, 0, 0}, + new long[]{1, 1, 2, 2, 2, 2, 0, 0, 0, 0}, + 1, 6 + ), aFast + ); + + ApproximateHistogram h3 = new ApproximateHistogram(10); + ApproximateHistogram h4 = new ApproximateHistogram(10); + for (float v : VALUES3) { + h3.offer(v); + } + for (float v : VALUES4) { + h4.offer(v); + } + h3.fold(h4); + Assert.assertArrayEquals( + "final bin positions match expected positions", + new float[]{-50.98f, -21.77f, -9.81f, 3.73f, 13.72f, 20.1f, 29f, 44.79f, 53.8f, 64.67f}, + h3.positions(), 0.1f + ); + Assert.assertArrayEquals( + "final bin counts match expected counts", + new long[]{1, 1, 3, 6, 12, 32, 6, 1, 2, 6}, h3.bins() + ); + + } + + @Test + public void testFoldNothing() throws Exception + { + ApproximateHistogram h1 = new ApproximateHistogram(10); + ApproximateHistogram h2 = new ApproximateHistogram(10); + + h1.fold(h2); + h1.foldFast(h2); + } + + @Test + public void testFoldNothing2() throws Exception + { + ApproximateHistogram h1 = new ApproximateHistogram(10); + ApproximateHistogram h1Fast = new ApproximateHistogram(10); + ApproximateHistogram h2 = new ApproximateHistogram(10); + ApproximateHistogram h3 = new ApproximateHistogram(10); + ApproximateHistogram h4 = new ApproximateHistogram(10); + ApproximateHistogram h4Fast = new ApproximateHistogram(10); + for (float v : VALUES3) { + h3.offer(v); + h4.offer(v); + h4Fast.offer(v); + } + + h1.fold(h3); + h4.fold(h2); + h1Fast.foldFast(h3); + h4Fast.foldFast(h2); + + Assert.assertEquals(h3, h1); + Assert.assertEquals(h4, h3); + Assert.assertEquals(h3, h1Fast); + Assert.assertEquals(h3, h4Fast); + } + + //@Test + public void testFoldSpeed() + { + final int combinedHistSize = 200; + final int histSize = 50; + final int numRand = 10000; + ApproximateHistogram h = new ApproximateHistogram(combinedHistSize); + Random rand = new Random(0); + //for(int i = 0; i < 200; ++i) h.offer((float)(rand.nextGaussian() * 50.0)); + long tFold = 0; + int count = 5000000; + Float[] randNums = new Float[numRand]; + for (int i = 0; i < numRand; i++) { + randNums[i] = (float) rand.nextGaussian(); + } + + List randHist = Lists.newLinkedList(); + Iterator it = Iterators.cycle(randHist); + + for(int k = 0; k < numRand; ++k) { + ApproximateHistogram tmp = new ApproximateHistogram(histSize); + for (int i = 0; i < 20; ++i) { + tmp.offer((float) (rand.nextGaussian() + (double)k)); + } + randHist.add(tmp); + } + + float[] mergeBufferP = new float[combinedHistSize * 2]; + long[] mergeBufferB = new long[combinedHistSize * 2]; + float[] mergeBufferD = new float[combinedHistSize * 2]; + + for (int i = 0; i < count; ++i) { + ApproximateHistogram tmp = it.next(); + + long t0 = System.nanoTime(); + //h.fold(tmp, mergeBufferP, mergeBufferB, mergeBufferD); + h.foldFast(tmp, mergeBufferP, mergeBufferB); + tFold += System.nanoTime() - t0; + } + + System.out.println(String.format("Average folds per second : %f", (double) count / (double) tFold * 1e9)); + } + + @Test + public void testSum() + { + ApproximateHistogram h = buildHistogram(5, VALUES); + + Assert.assertEquals(0.0f, h.sum(0), 0.01); + Assert.assertEquals(1.0f, h.sum(2), 0.01); + Assert.assertEquals(1.16f, h.sum(5), 0.01); + Assert.assertEquals(3.28f, h.sum(15), 0.01); + Assert.assertEquals(VALUES.length, h.sum(45), 0.01); + Assert.assertEquals(VALUES.length, h.sum(46), 0.01); + + ApproximateHistogram h2 = buildHistogram(5, VALUES2); + + Assert.assertEquals(0.0f, h2.sum(0), 0.01); + Assert.assertEquals(0.0f, h2.sum(1f), 0.01); + Assert.assertEquals(1.0f, h2.sum(1.5f), 0.01); + Assert.assertEquals(1.125f, h2.sum(2f), 0.001); + Assert.assertEquals(2.0625f, h2.sum(5.75f), 0.001); + Assert.assertEquals(3.0f, h2.sum(9.5f), 0.01); + Assert.assertEquals(11.0f, h2.sum(45.5f), 0.01); + Assert.assertEquals(12.0f, h2.sum(46f), 0.01); + Assert.assertEquals(12.0f, h2.sum(47f), 0.01); + } + + @Test + public void testSerializeCompact() + { + ApproximateHistogram h = buildHistogram(5, VALUES); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes())); + + ApproximateHistogram h2 = new ApproximateHistogram(50).fold(h); + Assert.assertEquals(h2, ApproximateHistogram.fromBytes(h2.toBytes())); + } + + @Test + public void testSerializeDense() + { + ApproximateHistogram h = buildHistogram(5, VALUES); + ByteBuffer buf = ByteBuffer.allocate(h.getDenseStorageSize()); + h.toBytesDense(buf); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(buf.array())); + } + + @Test + public void testSerializeSparse() + { + ApproximateHistogram h = buildHistogram(5, VALUES); + ByteBuffer buf = ByteBuffer.allocate(h.getSparseStorageSize()); + h.toBytesSparse(buf); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(buf.array())); + } + + @Test + public void testSerializeCompactExact() + { + ApproximateHistogram h = buildHistogram(50, new float[]{1f, 2f, 3f, 4f, 5f}); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes())); + + h = buildHistogram(5, new float[]{1f, 2f, 3f}); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes())); + + h = new ApproximateHistogram(40).fold(h); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes())); + } + + @Test + public void testSerializeEmpty() + { + ApproximateHistogram h = new ApproximateHistogram(50); + Assert.assertEquals(h, ApproximateHistogram.fromBytes(h.toBytes())); + } + + @Test + public void testQuantileSmaller() + { + ApproximateHistogram h = buildHistogram(20, VALUES5); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{5f}, + h.getQuantiles(new float[]{.5f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{3.33f, 6.67f}, + h.getQuantiles(new float[]{.333f, .666f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{2.5f, 5f, 7.5f}, + h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{2f, 4f, 6f, 8f}, + h.getQuantiles(new float[]{.2f, .4f, .6f, .8f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{1f, 2f, 3f, 4f, 5f, 6f, 7f, 8f, 9f}, + h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f + ); + } + + @Test + public void testQuantileEqualSize() + { + ApproximateHistogram h = buildHistogram(10, VALUES5); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{5f}, + h.getQuantiles(new float[]{.5f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{3.33f, 6.67f}, + h.getQuantiles(new float[]{.333f, .666f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{2.5f, 5f, 7.5f}, + h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{2f, 4f, 6f, 8f}, + h.getQuantiles(new float[]{.2f, .4f, .6f, .8f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{1f, 2f, 3f, 4f, 5f, 6f, 7f, 8f, 9f}, + h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f + ); + } + + @Test + public void testQuantileBigger() + { + ApproximateHistogram h = buildHistogram(5, VALUES5); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{4.5f}, + h.getQuantiles(new float[]{.5f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{2.83f, 6.17f}, + h.getQuantiles(new float[]{.333f, .666f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{2f, 4.5f, 7f}, + h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{1.5f, 3.5f, 5.5f, 7.5f}, + h.getQuantiles(new float[]{.2f, .4f, .6f, .8f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{1f, 1.5f, 2.5f, 3.5f, 4.5f, 5.5f, 6.5f, 7.5f, 8.5f}, + h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f + ); + } + + @Test + public void testQuantileBigger2() + { + float[] thousand = new float[1000]; + for (int i = 1; i <= 1000; ++i) { + thousand[i - 1] = i; + } + ApproximateHistogram h = buildHistogram(100, thousand); + + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{493.5f}, + h.getQuantiles(new float[]{.5f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{327.5f, 662f}, + h.getQuantiles(new float[]{.333f, .666f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{244.5f, 493.5f, 746f}, + h.getQuantiles(new float[]{.25f, .5f, .75f}), 0.1f + ); + Assert.assertArrayEquals( + "expected quantiles match actual quantiles", + new float[]{96.5f, 196.53f, 294.5f, 395.5f, 493.5f, 597f, 696f, 795f, 895.25f}, + h.getQuantiles(new float[]{.1f, .2f, .3f, .4f, .5f, .6f, .7f, .8f, .9f}), 0.1f + ); + } + + @Test + public void testLimitSum() + { + final float lowerLimit = 0f; + final float upperLimit = 10f; + + ApproximateHistogram h = buildHistogram(15, VALUES6, lowerLimit, upperLimit); + + for (int i = 1; i <= 20; ++i) { + ApproximateHistogram hLow = new ApproximateHistogram(5); + ApproximateHistogram hHigh = new ApproximateHistogram(5); + hLow.offer(lowerLimit - i); + hHigh.offer(upperLimit + i); + h.foldFast(hLow); + h.foldFast(hHigh); + } + + Assert.assertEquals(20f, h.sum(lowerLimit), .7f); + Assert.assertEquals(VALUES6.length + 20f, h.sum(upperLimit), 0.01); + } + + @Test + public void testBuckets() + { + final float[] values = new float[]{-5f, .01f, .02f, .06f, .12f, 1f, 2f}; + ApproximateHistogram h = buildHistogram(50, values, 0f, 1f); + Histogram h2 = h.toHistogram(.05f, 0f); + + Assert.assertArrayEquals( + "expected counts match actual counts", + new double[]{1f, 2f, 1f, 1f, 0f, 1f, 1f}, + h2.getCounts(), 0.1f + ); + + Assert.assertArrayEquals( + "expected breaks match actual breaks", + new double[]{-5.05f, 0f, .05f, .1f, .15f, .95f, 1f, 2f}, + h2.getBreaks(), 0.1f + ); + } + + @Test + public void testBuckets2() + { + final float[] values = new float[]{-5f, .01f, .02f, .06f, .12f, .94f, 1f, 2f}; + ApproximateHistogram h = buildHistogram(50, values, 0f, 1f); + Histogram h2 = h.toHistogram(.05f, 0f); + + Assert.assertArrayEquals( + "expected counts match actual counts", + new double[]{1f, 2f, 1f, 1f, 0f, 1f, 1f, 1f}, + h2.getCounts(), 0.1f + ); + + Assert.assertArrayEquals( + "expected breaks match actual breaks", + new double[]{-5.05f, 0f, .05f, .1f, .15f, .9f, .95f, 1f, 2.05f}, + h2.getBreaks(), 0.1f + ); + } + + @Test + public void testBuckets3() + { + final float[] values = new float[]{0f, 0f, .02f, .06f, .12f, .94f}; + ApproximateHistogram h = buildHistogram(50, values, 0f, 1f); + Histogram h2 = h.toHistogram(1f, 0f); + + Assert.assertArrayEquals( + "expected counts match actual counts", + new double[]{2f, 4f}, + h2.getCounts(), 0.1f + ); + + Assert.assertArrayEquals( + "expected breaks match actual breaks", + new double[]{-1f, 0f, 1f}, + h2.getBreaks(), 0.1f + ); + } + + @Test + public void testBuckets4() + { + final float[] values = new float[]{0f, 0f, 0.01f, 0.51f, 0.6f,0.8f}; + ApproximateHistogram h = buildHistogram(50, values, 0.5f,1f); + Histogram h3 = h.toHistogram(0.2f,0); + + Assert.assertArrayEquals( + "Expected counts match actual counts", + new double[]{3f,2f,1f}, + h3.getCounts(), + 0.1f + ); + + Assert.assertArrayEquals( + "expected breaks match actual breaks", + new double[]{-0.2f,0.5f,0.7f,0.9f}, + h3.getBreaks(), 0.1f + ); + } + + @Test public void testBuckets5() + { + final float[] values = new float[]{0.1f,0.5f,0.6f}; + ApproximateHistogram h = buildHistogram(50, values, 0f,1f); + Histogram h4 = h.toHistogram(0.5f,0); + + Assert.assertArrayEquals( + "Expected counts match actual counts", + new double[]{2,1}, + h4.getCounts(), + 0.1f + ); + + Assert.assertArrayEquals( + "Expected breaks match actual breaks", + new double[]{0f,0.5f,1f}, + h4.getBreaks(), + 0.1f + ); + } + + @Test public void testEmptyHistogram() { + ApproximateHistogram h = new ApproximateHistogram(50); + Assert.assertArrayEquals( + new float[]{Float.NaN, Float.NaN}, + h.getQuantiles(new float[]{0.8f, 0.9f}), + 1e-9f + ); + } + + +} diff --git a/histogram/src/test/java/io/druid/query/aggregation/histogram/QuantilesTest.java b/histogram/src/test/java/io/druid/query/aggregation/histogram/QuantilesTest.java new file mode 100644 index 00000000000..14ec2eb34a7 --- /dev/null +++ b/histogram/src/test/java/io/druid/query/aggregation/histogram/QuantilesTest.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.aggregation.histogram; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.jackson.DefaultObjectMapper; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.LinkedHashMap; + +public class QuantilesTest +{ + @Test + public void testSerialization() throws Exception + { + ObjectMapper mapper = new DefaultObjectMapper(); + + float[] probabilities = new float[]{0.25f, 0.5f, 0.75f}; + float[] quantiles = new float[]{0.25f, 0.5f, 0.75f}; + float min = 0f; + float max = 4f; + + String theString = mapper.writeValueAsString( + new Quantiles(probabilities, quantiles, min, max) + ); + + Object theObject = mapper.readValue(theString, Object.class); + Assert.assertThat(theObject, CoreMatchers.instanceOf(LinkedHashMap.class)); + + LinkedHashMap theMap = (LinkedHashMap) theObject; + + ArrayList theProbabilities = (ArrayList) theMap.get("probabilities"); + + Assert.assertEquals(probabilities.length, theProbabilities.size()); + for (int i = 0; i < theProbabilities.size(); ++i) { + Assert.assertEquals(probabilities[i], ((Number) theProbabilities.get(i)).floatValue(), 0.0001f); + } + + ArrayList theQuantiles = (ArrayList) theMap.get("quantiles"); + + Assert.assertEquals(quantiles.length, theQuantiles.size()); + for (int i = 0; i < theQuantiles.size(); ++i) { + Assert.assertEquals(quantiles[i], ((Number) theQuantiles.get(i)).floatValue(), 0.0001f); + } + + Assert.assertEquals( + "serialized min. matches expected min.", + min, + ((Number) theMap.get("min")).floatValue(), + 0.0001f + ); + Assert.assertEquals( + "serialized max. matches expected max.", + max, + ((Number) theMap.get("max")).floatValue(), + 0.0001f + ); + + + } +} diff --git a/indexing-hadoop/README b/indexing-hadoop/README deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/pom.xml b/pom.xml index 9580f7aac4f..2b7f88276e8 100644 --- a/pom.xml +++ b/pom.xml @@ -59,6 +59,7 @@ kafka-seven kafka-eight rabbitmq + histogram From d0b849473567755aff734a2c2d771b762cc20073 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 1 Jul 2014 16:13:00 -0700 Subject: [PATCH 124/270] fix approximate histogram serde to not change buffer limit --- .../histogram/ApproximateHistogramFoldingSerde.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java index 196a1e50548..d7b988792bc 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java @@ -110,8 +110,9 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde @Override public ApproximateHistogram fromByteBuffer(ByteBuffer buffer, int numBytes) { - buffer.limit(buffer.position() + numBytes); - return ApproximateHistogram.fromBytes(buffer); + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + readOnlyBuffer.limit(readOnlyBuffer.position() + numBytes); + return ApproximateHistogram.fromBytes(readOnlyBuffer); } @Override From dba7e972d8fbec5a5032ef5d9b7578a878e3a837 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 1 Jul 2014 16:39:24 -0700 Subject: [PATCH 125/270] refactor docs for histograms --- docs/content/Aggregations.md | 27 +--------- docs/content/ApproxHisto.md | 90 +++++++++++++++++++++++++++++++ docs/content/Post-aggregations.md | 66 ----------------------- docs/content/toc.textile | 1 + 4 files changed, 92 insertions(+), 92 deletions(-) create mode 100644 docs/content/ApproxHisto.md diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index cdf33ed2b9b..29740a2858c 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -159,29 +159,4 @@ Uses [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf) to ```json { "type" : "hyperUnique", "name" : , "fieldName" : } -``` - -### ApproxHistogram aggregator - -This aggregator is based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) to compute approximate histograms. - -To use this feature, an "approxHistogram" aggregator must be included at indexing time. The ingestion aggregator can only apply to numeric values. To query for results, an "approxHistogramFold" aggregator must be included in the query. - -```json -{ - "type" : "approxHistogram(ingestion), approxHistogramFold(query)", - "name" : , - "fieldName" : , - "resolution" : , - "numBuckets" : , - "lowerLimit" : , - "upperLimit" : -} -``` - -|Property|Description|Default| -|--------|-----------|-------| -|`resolution`|Number of centroids (data points) to store. The higher the resolution, the more accurate results are, but the slower computation will be.|50| -|`numBuckets`|Number of output buckets for the resulting histogram.|7| -|`lowerLimit`/`upperLimit`|Restrict the approximation to the given range. The values outside this range will be aggregated into two centroids. Counts of values outside this range are still maintained. |-INF/+INF| - +``` \ No newline at end of file diff --git a/docs/content/ApproxHisto.md b/docs/content/ApproxHisto.md new file mode 100644 index 00000000000..4c7229b813e --- /dev/null +++ b/docs/content/ApproxHisto.md @@ -0,0 +1,90 @@ +### ApproxHistogram aggregator + +This aggregator is based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) to compute approximate histograms. + +To use this feature, an "approxHistogram" aggregator must be included at indexing time. The ingestion aggregator can only apply to numeric values. To query for results, an "approxHistogramFold" aggregator must be included in the query. + +```json +{ + "type" : "approxHistogram(ingestion), approxHistogramFold(query)", + "name" : , + "fieldName" : , + "resolution" : , + "numBuckets" : , + "lowerLimit" : , + "upperLimit" : +} +``` + +|Property|Description|Default| +|--------|-----------|-------| +|`resolution`|Number of centroids (data points) to store. The higher the resolution, the more accurate results are, but the slower computation will be.|50| +|`numBuckets`|Number of output buckets for the resulting histogram.|7| +|`lowerLimit`/`upperLimit`|Restrict the approximation to the given range. The values outside this range will be aggregated into two centroids. Counts of values outside this range are still maintained. |-INF/+INF| + + +### Approximate Histogram post-aggregators + +Post-aggregators used to transform opaque approximate histogram objects +into actual histogram representations, and to compute various distribution metrics. + +#### equal buckets post-aggregator + +Computes a visual representation of the approximate histogram with a given number of equal-sized bins + +```json +{ "type" : "equalBuckets", "name" : , "fieldName" : , + "numBuckets" : } +``` + +#### buckets post-aggregator + +Computes a visual representation given an initial breakpoint, offset, and a bucket size. + +```json +{ "type" : "buckets", "name" : , "fieldName" : , + "bucketSize" : , "offset" : } +``` + +#### custom buckets post-aggregator + +Computes a visual representation of the approximate histogram with bins laid out according to the given breaks + +```json +{ "type" : "customBuckets", "name" : , "fieldName" : , + "breaks" : [ , , ... ] } +``` + +#### min post-aggregator + +Returns the minimum value of the underlying approximate histogram aggregator + +```json +{ "type" : "min", "name" : , "fieldName" : } +``` + +#### max post-aggregator + +Returns the maximum value of the underlying approximate histogram aggregator + +```json +{ "type" : "max", "name" : , "fieldName" : } +``` + +#### quantile post-aggregator + +Computes a single quantile based on the underlying approximate histogram aggregator + +```json +{ "type" : "quantile", "name" : , "fieldName" : , + "probability" : } +``` + +#### quantiles post-aggregator + +Computes an array of quantiles based on the underlying approximate histogram aggregator + +```json +{ "type" : "quantiles", "name" : , "fieldName" : , + "probabilities" : [ , , ... ] } +``` \ No newline at end of file diff --git a/docs/content/Post-aggregations.md b/docs/content/Post-aggregations.md index e4ce2211f67..5e279088b3a 100644 --- a/docs/content/Post-aggregations.md +++ b/docs/content/Post-aggregations.md @@ -122,69 +122,3 @@ The format of the query JSON is as follows: ... } ``` - -### Approximate Histogram post-aggregators - -Post-aggregators used to transform opaque approximate histogram objects -into actual histogram representations, and to compute various distribution metrics. - -#### equal buckets post-aggregator - -Computes a visual representation of the approximate histogram with a given number of equal-sized bins - -```json -{ "type" : "equalBuckets", "name" : , "fieldName" : , - "numBuckets" : } -``` - -#### buckets post-aggregator - -Computes a visual representation given an initial breakpoint, offset, and a bucket size. - -```json -{ "type" : "buckets", "name" : , "fieldName" : , - "bucketSize" : , "offset" : } -``` - -#### custom buckets post-aggregator - -Computes a visual representation of the approximate histogram with bins laid out according to the given breaks - -```json -{ "type" : "customBuckets", "name" : , "fieldName" : , - "breaks" : [ , , ... ] } -``` - -#### min post-aggregator - -Returns the minimum value of the underlying approximate histogram aggregator - -```json -{ "type" : "min", "name" : , "fieldName" : } -``` - -#### max post-aggregator - -Returns the maximum value of the underlying approximate histogram aggregator - -```json -{ "type" : "max", "name" : , "fieldName" : } -``` - -#### quantile post-aggregator - -Computes a single quantile based on the underlying approximate histogram aggregator - -```json -{ "type" : "quantile", "name" : , "fieldName" : , - "probability" : } -``` - -#### quantiles post-aggregator - -Computes an array of quantiles based on the underlying approximate histogram aggregator - -```json -{ "type" : "quantiles", "name" : , "fieldName" : , - "probabilities" : [ , , ... ] } -``` diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 72594a9fa87..a4769a6b2f5 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -79,6 +79,7 @@ h2. Experimental * "About Experimental Features":./About-Experimental-Features.html * "Geographic Queries":./GeographicQueries.html * "Select Query":./SelectQuery.html +* "Approximate Histograms and Quantiles":./ApproxHisto.html h2. Development * "Versioning":./Versioning.html From d37527ca79119b7b48799584d1691ef17dddbfd7 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 1 Jul 2014 16:46:58 -0700 Subject: [PATCH 126/270] make histos work with groupby maybe --- .../ApproximateHistogramFoldingSerde.java | 28 +++++++++++-------- .../hyperloglog/HyperUniquesSerde.java | 2 +- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java index d7b988792bc..b1f822586ba 100644 --- a/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java +++ b/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java @@ -67,19 +67,25 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde @Override public ApproximateHistogram extractValue(InputRow inputRow, String metricName) { - List dimValues = inputRow.getDimension(metricName); - if (dimValues != null && dimValues.size() > 0) { - Iterator values = dimValues.iterator(); + Object rawValue = inputRow.getRaw(metricName); - ApproximateHistogram h = new ApproximateHistogram(); - - while (values.hasNext()) { - float value = Float.parseFloat(values.next()); - h.offer(value); - } - return h; + if (rawValue instanceof ApproximateHistogram) { + return (ApproximateHistogram) rawValue; } else { - return new ApproximateHistogram(0); + List dimValues = inputRow.getDimension(metricName); + if (dimValues != null && dimValues.size() > 0) { + Iterator values = dimValues.iterator(); + + ApproximateHistogram h = new ApproximateHistogram(); + + while (values.hasNext()) { + float value = Float.parseFloat(values.next()); + h.offer(value); + } + return h; + } else { + return new ApproximateHistogram(0); + } } } }; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java index b83d52f1cb6..486616b5eca 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java @@ -82,7 +82,7 @@ public class HyperUniquesSerde extends ComplexMetricSerde Object rawValue = inputRow.getRaw(metricName); if (rawValue instanceof HyperLogLogCollector) { - return (HyperLogLogCollector) inputRow.getRaw(metricName); + return (HyperLogLogCollector) rawValue; } else { HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector(); From 1de390801fd1207388984b5a5684925138f01dc3 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 2 Jul 2014 20:34:18 +0530 Subject: [PATCH 127/270] Druid Fireshose Add druidFirehose, can be used to reIndex filtered data from a datasource --- .../guice/IndexingServiceFirehoseModule.java | 6 +- .../firehose/DruidFirehoseFactory.java | 352 ++++++++++++++++++ 2 files changed, 356 insertions(+), 2 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java diff --git a/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java index b8609453f3d..55f94147b67 100644 --- a/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java +++ b/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java @@ -24,8 +24,9 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; +import io.druid.indexing.firehose.DruidFirehoseFactory; import io.druid.initialization.DruidModule; +import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import java.util.List; @@ -37,7 +38,8 @@ public class IndexingServiceFirehoseModule implements DruidModule return ImmutableList.of( new SimpleModule("IndexingServiceFirehoseModule") .registerSubtypes( - new NamedType(EventReceiverFirehoseFactory.class, "receiver") + new NamedType(EventReceiverFirehoseFactory.class, "receiver"), + new NamedType(DruidFirehoseFactory.class, "druid") ) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java new file mode 100644 index 00000000000..215ddd937e4 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java @@ -0,0 +1,352 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.indexing.firehose; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.inject.Injector; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Sequences; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import com.metamx.common.parsers.ParseException; +import com.metamx.emitter.EmittingLogger; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.InputRowParser; +import io.druid.granularity.QueryGranularity; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.TaskToolboxFactory; +import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.task.NoopTask; +import io.druid.query.filter.DimFilter; +import io.druid.query.select.EventHolder; +import io.druid.segment.Cursor; +import io.druid.segment.DimensionSelector; +import io.druid.segment.IndexIO; +import io.druid.segment.ObjectColumnSelector; +import io.druid.segment.QueryableIndexStorageAdapter; +import io.druid.segment.StorageAdapter; +import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.filter.Filters; +import io.druid.segment.loading.SegmentLoadingException; +import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class DruidFirehoseFactory implements FirehoseFactory +{ + private static final EmittingLogger log = new EmittingLogger(DruidFirehoseFactory.class); + private final String dataSource; + private final Interval interval; + private final DimFilter dimFilter; + private final List dimensions; + private final List metrics; + private final Injector injector; + + @JsonCreator + public DruidFirehoseFactory( + @JsonProperty("dataSource") final String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("filter") DimFilter dimFilter, + @JsonProperty("dimensions") List dimensions, + @JsonProperty("metrics") List metrics, + @JacksonInject Injector injector + ) + { + Preconditions.checkNotNull(dataSource, "dataSource"); + Preconditions.checkNotNull(interval, "interval"); + this.dataSource = dataSource; + this.interval = interval; + this.dimFilter = dimFilter; + this.dimensions = dimensions; + this.metrics = metrics; + this.injector = injector; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty("filter") + public DimFilter getDimensionsFilter() + { + return dimFilter; + } + + @JsonProperty + public List getDimensions() + { + return dimensions; + } + + @JsonProperty + public List getMetrics() + { + return metrics; + } + + @Override + public Firehose connect(InputRowParser inputRowParser) throws IOException, ParseException + { + log.info("Connecting firehose: DruidFirehose[%s,%s]", dataSource, interval); + // TODO: have a way to pass the toolbox to Firehose, The instance is initialized Lazily on connect method. + final TaskToolbox toolbox = injector.getInstance(TaskToolboxFactory.class).build( + new NoopTask( + "druid-firehose", + 0, + 0, + null, + null + ) + ); + + try { + final List usedSegments = toolbox + .getTaskActionClient() + .submit(new SegmentListUsedAction(dataSource, interval)); + final Map segmentFileMap = toolbox.fetchSegments(usedSegments); + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( + Ordering.natural().nullsFirst() + ); + for (DataSegment segment : usedSegments) { + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + } + List dims; + if (dimensions != null) { + dims = dimensions; + } else { + Set dimSet = new HashSet<>(); + for (DataSegment segment : usedSegments) { + dimSet.addAll(segment.getDimensions()); + } + dims = Lists.newArrayList(dimSet); + } + + List metricsList; + if (metrics != null) { + metricsList = metrics; + } else { + Set metricsSet = new HashSet<>(); + for (DataSegment segment : usedSegments) { + metricsSet.addAll(segment.getMetrics()); + } + metricsList = Lists.newArrayList(metricsSet); + } + + + final List adapters = Lists.transform( + timeline.lookup(new Interval("1000-01-01/3000-01-01")), + new Function, StorageAdapter>() + { + @Override + public StorageAdapter apply(TimelineObjectHolder input) + { + final DataSegment segment = input.getObject().getChunk(0).getObject(); + final File file = Preconditions.checkNotNull( + segmentFileMap.get(segment), + "File for segment %s", segment.getIdentifier() + ); + + try { + return new QueryableIndexStorageAdapter((IndexIO.loadIndex(file))); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + } + ); + + return new DruidFirehose(adapters, dims, metricsList); + + } + catch (IOException e) { + throw Throwables.propagate(e); + } + catch (SegmentLoadingException e) { + throw Throwables.propagate(e); + } + + } + + @Override + public InputRowParser getParser() + { + return null; + } + + public class DruidFirehose implements Firehose + { + private volatile Yielder rowYielder; + + public DruidFirehose(List adapters, final List dims, final List metrics) + { + Sequence rows = Sequences.concat( + Iterables.transform( + adapters, new Function>() + { + @Nullable + @Override + public Sequence apply(@Nullable StorageAdapter input) + { + return Sequences.concat( + Sequences.map( + input.makeCursors( + Filters.convertDimensionFilters(dimFilter), + interval, + QueryGranularity.ALL + ), new Function>() + { + @Nullable + @Override + public Sequence apply(@Nullable Cursor input) + { + TimestampColumnSelector timestampColumnSelector = input.makeTimestampColumnSelector(); + + Map dimSelectors = Maps.newHashMap(); + for (String dim : dims) { + final DimensionSelector dimSelector = input.makeDimensionSelector(dim); + dimSelectors.put(dim, dimSelector); + } + + Map metSelectors = Maps.newHashMap(); + for (String metric : metrics) { + final ObjectColumnSelector metricSelector = input.makeObjectColumnSelector(metric); + metSelectors.put(metric, metricSelector); + } + + List rowList = Lists.newArrayList(); + while (!input.isDone()) { + final Map theEvent = Maps.newLinkedHashMap(); + final long timestamp = timestampColumnSelector.getTimestamp(); + theEvent.put(EventHolder.timestampKey, new DateTime(timestamp)); + + for (Map.Entry dimSelector : dimSelectors.entrySet()) { + final String dim = dimSelector.getKey(); + final DimensionSelector selector = dimSelector.getValue(); + final IndexedInts vals = selector.getRow(); + + if (vals.size() == 1) { + final String dimVal = selector.lookupName(vals.get(0)); + theEvent.put(dim, dimVal); + } else { + List dimVals = Lists.newArrayList(); + for (int i = 0; i < vals.size(); ++i) { + dimVals.add(selector.lookupName(vals.get(i))); + } + theEvent.put(dim, dimVals); + } + } + + for (Map.Entry metSelector : metSelectors.entrySet()) { + final String metric = metSelector.getKey(); + final ObjectColumnSelector selector = metSelector.getValue(); + theEvent.put(metric, selector.get()); + } + rowList.add(new MapBasedInputRow(timestamp, dims, theEvent)); + input.advance(); + } + return Sequences.simple(rowList); + } + } + ) + ); + } + } + ) + ); + rowYielder = rows.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Object accumulate(Object accumulated, Object in) + { + yield(); + return in; + } + } + ); + } + + @Override + public boolean hasMore() + { + return !rowYielder.isDone(); + } + + @Override + public InputRow nextRow() + { + final InputRow inputRow = rowYielder.get(); + rowYielder = rowYielder.next(null); + return inputRow; + } + + @Override + public Runnable commit() + { + return new Runnable() + { + @Override + public void run() + { + // Nothing to do. + } + }; + } + + @Override + public void close() throws IOException + { + rowYielder.close(); + } + } +} From c646648319632ee972a2b4f25421f47a2774faba Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 2 Jul 2014 10:07:04 -0700 Subject: [PATCH 128/270] fix examples start port --- docs/content/Middlemanager.md | 2 +- docs/content/Simple-Cluster-Configuration.md | 1 + examples/config/overlord/runtime.properties | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/content/Middlemanager.md b/docs/content/Middlemanager.md index c143bb91987..7b71db207ba 100644 --- a/docs/content/Middlemanager.md +++ b/docs/content/Middlemanager.md @@ -52,6 +52,6 @@ Middle managers pass their configurations down to their child peons. The middle |`druid.indexer.runner.javaCommand`|Command required to execute java.|java| |`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""| |`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")| -|`druid.indexer.runner.startPort`|The port that peons begin running on.|8080| +|`druid.indexer.runner.startPort`|The port that peons begin running on.|8081| |`druid.indexer.runner.allowedPrefixes`|Whitelist of prefixes for configs that can be passed down to child peons.|"com.metamx", "druid", "io.druid", "user.timezone","file.encoding"| diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 51bc7778fd9..33e0c8d13bd 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -37,6 +37,7 @@ Configuration: -Ddruid.selectors.indexing.serviceName=overlord -Ddruid.indexer.queue.startDelay=PT0M -Ddruid.indexer.runner.javaOpts="-server -Xmx1g" +-Ddruid.indexer.runner.startPort=8088 -Ddruid.indexer.fork.property.druid.processing.numThreads=1 -Ddruid.indexer.fork.property.druid.computation.buffer.size=100000000 ``` diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index 2e4802e0f50..7a267d7114e 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -13,5 +13,6 @@ druid.db.connector.password=diurd druid.selectors.indexing.serviceName=overlord druid.indexer.queue.startDelay=PT0M druid.indexer.runner.javaOpts="-server -Xmx256m" +druid.indexer.runner.startPort=8088 druid.indexer.fork.property.druid.processing.numThreads=1 druid.indexer.fork.property.druid.computation.buffer.size=100000000 \ No newline at end of file From 31f7bd8bcbc6387f2e2103049864645c55241f7f Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Wed, 2 Jul 2014 14:44:02 -0700 Subject: [PATCH 129/270] AWSCredentialsProvider for s3-extentions - fileSessionCredentials property is added - AWSSessionCredentialsAdapter is adapter class between aws sdk AWSCredentialsProvider to jets3t Credentials - FileSessionCredentialsProvider will refresh its session credentials from the file - S3StorageDruidModule should return different AWSCredentialsProvider on the config settings --- s3-extensions/pom.xml | 10 +++ .../storage/s3/AWSCredentialsConfig.java | 5 ++ .../s3/AWSSessionCredentialsAdapter.java | 43 +++++++++++++ .../s3/FileSessionCredentialsProvider.java | 62 ++++++++++++++++++ .../storage/s3/S3StorageDruidModule.java | 35 ++++++++-- .../s3/TestAWSCredentialsProvider.java | 64 +++++++++++++++++++ .../TestFileSessionCredentialsProvider.java | 31 +++++++++ 7 files changed, 243 insertions(+), 7 deletions(-) create mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java create mode 100644 s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java create mode 100644 s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java create mode 100644 s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 8d9e4fae39e..24807b42db2 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -41,6 +41,10 @@ net.java.dev.jets3t jets3t + + com.amazonaws + aws-java-sdk + org.apache.httpcomponents @@ -65,6 +69,12 @@ junit test + + org.mockito + mockito-core + 1.9.5 + test + diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java b/s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java index e19289ec8ee..dc2615ae264 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/AWSCredentialsConfig.java @@ -31,6 +31,9 @@ public class AWSCredentialsConfig @JsonProperty private String secretKey = ""; + @JsonProperty + private String fileSessionCredentials = ""; + public String getAccessKey() { return accessKey; @@ -40,4 +43,6 @@ public class AWSCredentialsConfig { return secretKey; } + + public String getFileSessionCredentials() { return fileSessionCredentials; } } diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java b/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java new file mode 100644 index 00000000000..8cabf29c135 --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java @@ -0,0 +1,43 @@ +package io.druid.storage.s3; + +import com.amazonaws.auth.AWSCredentialsProvider; +import org.jets3t.service.security.AWSSessionCredentials; + +public class AWSSessionCredentialsAdapter extends AWSSessionCredentials { + private final AWSCredentialsProvider provider; + + public AWSSessionCredentialsAdapter(AWSCredentialsProvider provider) { + super(null, null, null); + this.provider = provider; + } + + @Override + protected String getTypeName() { + return "AWSSessionCredentialsAdapter"; + } + + @Override + public String getVersionPrefix() { + return "Netflix AWSSessionCredentialsAdapter, version: "; + } + + @Override + public String getAccessKey() { + return provider.getCredentials().getAWSAccessKeyId(); + } + + @Override + public String getSecretKey() { + return provider.getCredentials().getAWSSecretKey(); + } + + public String getSessionToken() { + if (provider.getCredentials() instanceof com.amazonaws.auth.AWSSessionCredentials) { + com.amazonaws.auth.AWSSessionCredentials sessionCredentials = + (com.amazonaws.auth.AWSSessionCredentials) provider.getCredentials(); + return sessionCredentials.getSessionToken(); + } else { + return ""; + } + } +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java new file mode 100644 index 00000000000..c37edfb51da --- /dev/null +++ b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java @@ -0,0 +1,62 @@ +package io.druid.storage.s3; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSSessionCredentials; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Properties; + +public class FileSessionCredentialsProvider implements AWSCredentialsProvider { + private final String sessionCredentials; + private String sessionToken; + private String accessKey; + private String secretKey; + + public FileSessionCredentialsProvider(String sessionCredentials) { + this.sessionCredentials = sessionCredentials; + refresh(); + } + + @Override + public AWSCredentials getCredentials() { + return new AWSSessionCredentials() { + @Override + public String getSessionToken() { + return sessionToken; + } + + @Override + public String getAWSAccessKeyId() { + return accessKey; + } + + @Override + public String getAWSSecretKey() { + return secretKey; + } + }; + } + + @Override + public void refresh() { + try { + List lines = Files.readAllLines(Paths.get(sessionCredentials), Charset.defaultCharset()); + Properties props = new Properties(); + for (String line : lines) { + String[] tokens = line.split("="); + props.put(tokens[0], tokens[1]); + } + + sessionToken = props.getProperty("sessionToken"); + accessKey = props.getProperty("accessKey"); + secretKey = props.getProperty("secretKey"); + } catch (IOException e) { + throw new RuntimeException("cannot refresh AWS credentials", e); + } + } +} diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 21faf74db91..3a66292e76a 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -19,18 +19,17 @@ package io.druid.storage.s3; +import com.amazonaws.auth.AWSCredentialsProvider; import com.fasterxml.jackson.databind.Module; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Provides; -import com.google.inject.ProvisionException; import io.druid.guice.Binders; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.initialization.DruidModule; -import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.security.AWSCredentials; import java.util.List; @@ -64,15 +63,37 @@ public class S3StorageDruidModule implements DruidModule @Provides @LazySingleton - public AWSCredentials getJets3tAWSCredentials(AWSCredentialsConfig config) + public AWSCredentialsProvider getAWSCredentialsProvider(final AWSCredentialsConfig config) { - return new AWSCredentials(config.getAccessKey(), config.getSecretKey()); + if (!Strings.isNullOrEmpty(config.getAccessKey()) && !Strings.isNullOrEmpty(config.getSecretKey())) { + return new AWSCredentialsProvider() { + @Override + public com.amazonaws.auth.AWSCredentials getCredentials() { + return new com.amazonaws.auth.AWSCredentials() { + @Override + public String getAWSAccessKeyId() { + return config.getAccessKey(); + } + + @Override + public String getAWSSecretKey() { + return config.getSecretKey(); + } + }; + } + + @Override + public void refresh() {} + }; + } else { + return new FileSessionCredentialsProvider(config.getFileSessionCredentials()); + } } @Provides @LazySingleton - public RestS3Service getRestS3Service(AWSCredentials credentials) + public RestS3Service getRestS3Service(AWSCredentialsProvider credentialsProvider) { - return new RestS3Service(credentials); + return new RestS3Service(new AWSSessionCredentialsAdapter(credentialsProvider)); } } diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java b/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java new file mode 100644 index 00000000000..f677d25813c --- /dev/null +++ b/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java @@ -0,0 +1,64 @@ +package io.druid.storage.s3; + +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AWSSessionCredentials; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; + +public class TestAWSCredentialsProvider { + @Test + public void testWithFixedAWSKeys() { + S3StorageDruidModule module = new S3StorageDruidModule(); + + AWSCredentialsConfig config = mock(AWSCredentialsConfig.class); + doReturn("accessKeySample").when(config).getAccessKey(); + doReturn("secretKeySample").when(config).getSecretKey(); + + AWSCredentialsProvider provider = module.getAWSCredentialsProvider(config); + AWSCredentials credentials = provider.getCredentials(); + assertEquals(credentials.getAWSAccessKeyId(), "accessKeySample"); + assertEquals(credentials.getAWSSecretKey(), "secretKeySample"); + + // try to create + module.getRestS3Service(provider); + } + + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + + @Test + public void testWithFileSessionCredentials() throws IOException { + S3StorageDruidModule module = new S3StorageDruidModule(); + + AWSCredentialsConfig config = mock(AWSCredentialsConfig.class); + doReturn("").when(config).getAccessKey(); + doReturn("").when(config).getSecretKey(); + File file = folder.newFile(); + PrintWriter out = new PrintWriter(file.getAbsolutePath()); + out.println("sessionToken=sessionTokenSample\nsecretKey=secretKeySample\naccessKey=accessKeySample"); + out.close(); + doReturn(file.getAbsolutePath()).when(config).getFileSessionCredentials(); + + AWSCredentialsProvider provider = module.getAWSCredentialsProvider(config); + AWSCredentials credentials = provider.getCredentials(); + assertTrue(credentials instanceof AWSSessionCredentials); + AWSSessionCredentials sessionCredentials = (AWSSessionCredentials) credentials; + assertEquals(sessionCredentials.getAWSAccessKeyId(), "accessKeySample"); + assertEquals(sessionCredentials.getAWSSecretKey(), "secretKeySample"); + assertEquals(sessionCredentials.getSessionToken(), "sessionTokenSample"); + + // try to create + module.getRestS3Service(provider); + } +} diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java b/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java new file mode 100644 index 00000000000..c03dae19f77 --- /dev/null +++ b/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java @@ -0,0 +1,31 @@ +package io.druid.storage.s3; + +import com.amazonaws.auth.AWSSessionCredentials; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; + +import static org.junit.Assert.assertEquals; + +public class TestFileSessionCredentialsProvider { + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + + @Test + public void test() throws IOException { + File file = folder.newFile(); + PrintWriter out = new PrintWriter(file.getAbsolutePath()); + out.println("sessionToken=sessionTokenSample\nsecretKey=secretKeySample\naccessKey=accessKeySample"); + out.close(); + + FileSessionCredentialsProvider provider = new FileSessionCredentialsProvider(file.getAbsolutePath()); + AWSSessionCredentials sessionCredentials = (AWSSessionCredentials) provider.getCredentials(); + assertEquals(sessionCredentials.getSessionToken(), "sessionTokenSample"); + assertEquals(sessionCredentials.getAWSAccessKeyId(), "accessKeySample"); + assertEquals(sessionCredentials.getAWSSecretKey(), "secretKeySample"); + } +} From 5e5fa1f357afa73a6174bd2a176611e9d2f28141 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Wed, 2 Jul 2014 15:53:39 -0700 Subject: [PATCH 130/270] A few fixes - EasyMock instead of Mockito - FileSessionCredentialsProvider fields should be volatile - getRestS3Service should create AWSCredentials not a AWSSessionCredentials with fixed credentials --- s3-extensions/pom.xml | 5 ++--- .../s3/AWSSessionCredentialsAdapter.java | 17 ++++++++--------- .../s3/FileSessionCredentialsProvider.java | 6 +++--- .../storage/s3/S3StorageDruidModule.java | 12 ++++++++++-- .../s3/TestAWSCredentialsProvider.java | 19 ++++++++++--------- 5 files changed, 33 insertions(+), 26 deletions(-) diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 24807b42db2..1af0b941bdd 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -70,9 +70,8 @@ test - org.mockito - mockito-core - 1.9.5 + org.easymock + easymock test diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java b/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java index 8cabf29c135..abdc5e5cde8 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java @@ -8,7 +8,10 @@ public class AWSSessionCredentialsAdapter extends AWSSessionCredentials { public AWSSessionCredentialsAdapter(AWSCredentialsProvider provider) { super(null, null, null); - this.provider = provider; + if(provider.getCredentials() instanceof com.amazonaws.auth.AWSSessionCredentials) + this.provider = provider; + else + throw new IllegalArgumentException("provider does not contain session credentials"); } @Override @@ -18,7 +21,7 @@ public class AWSSessionCredentialsAdapter extends AWSSessionCredentials { @Override public String getVersionPrefix() { - return "Netflix AWSSessionCredentialsAdapter, version: "; + return "AWSSessionCredentialsAdapter, version: "; } @Override @@ -32,12 +35,8 @@ public class AWSSessionCredentialsAdapter extends AWSSessionCredentials { } public String getSessionToken() { - if (provider.getCredentials() instanceof com.amazonaws.auth.AWSSessionCredentials) { - com.amazonaws.auth.AWSSessionCredentials sessionCredentials = - (com.amazonaws.auth.AWSSessionCredentials) provider.getCredentials(); - return sessionCredentials.getSessionToken(); - } else { - return ""; - } + com.amazonaws.auth.AWSSessionCredentials sessionCredentials = + (com.amazonaws.auth.AWSSessionCredentials) provider.getCredentials(); + return sessionCredentials.getSessionToken(); } } diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java index c37edfb51da..35a999682ba 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java @@ -13,9 +13,9 @@ import java.util.Properties; public class FileSessionCredentialsProvider implements AWSCredentialsProvider { private final String sessionCredentials; - private String sessionToken; - private String accessKey; - private String secretKey; + private volatile String sessionToken; + private volatile String accessKey; + private volatile String secretKey; public FileSessionCredentialsProvider(String sessionCredentials) { this.sessionCredentials = sessionCredentials; diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java index 3a66292e76a..3d2434365ae 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3StorageDruidModule.java @@ -30,6 +30,7 @@ import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.initialization.DruidModule; import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.security.AWSCredentials; import java.util.List; @@ -92,8 +93,15 @@ public class S3StorageDruidModule implements DruidModule @Provides @LazySingleton - public RestS3Service getRestS3Service(AWSCredentialsProvider credentialsProvider) + public RestS3Service getRestS3Service(AWSCredentialsProvider provider) { - return new RestS3Service(new AWSSessionCredentialsAdapter(credentialsProvider)); + if(provider.getCredentials() instanceof com.amazonaws.auth.AWSSessionCredentials) { + return new RestS3Service(new AWSSessionCredentialsAdapter(provider)); + } else { + return new RestS3Service(new AWSCredentials( + provider.getCredentials().getAWSAccessKeyId(), + provider.getCredentials().getAWSSecretKey() + )); + } } } diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java b/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java index f677d25813c..e98e178550f 100644 --- a/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java +++ b/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java @@ -3,6 +3,7 @@ package io.druid.storage.s3; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSSessionCredentials; +import org.easymock.EasyMock; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -13,17 +14,16 @@ import java.io.PrintWriter; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; public class TestAWSCredentialsProvider { @Test public void testWithFixedAWSKeys() { S3StorageDruidModule module = new S3StorageDruidModule(); - AWSCredentialsConfig config = mock(AWSCredentialsConfig.class); - doReturn("accessKeySample").when(config).getAccessKey(); - doReturn("secretKeySample").when(config).getSecretKey(); + AWSCredentialsConfig config = EasyMock.createMock(AWSCredentialsConfig.class); + EasyMock.expect(config.getAccessKey()).andReturn("accessKeySample").atLeastOnce(); + EasyMock.expect(config.getSecretKey()).andReturn("secretKeySample").atLeastOnce(); + EasyMock.replay(config); AWSCredentialsProvider provider = module.getAWSCredentialsProvider(config); AWSCredentials credentials = provider.getCredentials(); @@ -41,14 +41,15 @@ public class TestAWSCredentialsProvider { public void testWithFileSessionCredentials() throws IOException { S3StorageDruidModule module = new S3StorageDruidModule(); - AWSCredentialsConfig config = mock(AWSCredentialsConfig.class); - doReturn("").when(config).getAccessKey(); - doReturn("").when(config).getSecretKey(); + AWSCredentialsConfig config = EasyMock.createMock(AWSCredentialsConfig.class); + EasyMock.expect(config.getAccessKey()).andReturn(""); + EasyMock.expect(config.getSecretKey()).andReturn(""); File file = folder.newFile(); PrintWriter out = new PrintWriter(file.getAbsolutePath()); out.println("sessionToken=sessionTokenSample\nsecretKey=secretKeySample\naccessKey=accessKeySample"); out.close(); - doReturn(file.getAbsolutePath()).when(config).getFileSessionCredentials(); + EasyMock.expect(config.getFileSessionCredentials()).andReturn(file.getAbsolutePath()).atLeastOnce(); + EasyMock.replay(config); AWSCredentialsProvider provider = module.getAWSCredentialsProvider(config); AWSCredentials credentials = provider.getCredentials(); From 518ab473f3ed1046ea5a907bfc0e712109ab4fdb Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 3 Jul 2014 09:58:12 +0530 Subject: [PATCH 131/270] improve port finding strategy for task runner 1) Recycle free ports 2) Choose only ports that are free & not used by any other application --- .../indexing/overlord/ForkingTaskRunner.java | 7 +- .../druid/indexing/overlord/PortFinder.java | 94 +++++++++++++++++++ .../indexing/overlord/PortFinderTest.java | 34 +++++++ 3 files changed, 132 insertions(+), 3 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java create mode 100644 indexing-service/src/test/java/io/druid/indexing/overlord/PortFinderTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 2be8a6a6a3b..7fe20ef9eab 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -79,6 +79,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer private final DruidNode node; private final ListeningExecutorService exec; private final ObjectMapper jsonMapper; + private final PortFinder portFinder; private final Map tasks = Maps.newHashMap(); @@ -97,6 +98,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer this.taskLogPusher = taskLogPusher; this.jsonMapper = jsonMapper; this.node = node; + this.portFinder = new PortFinder(config.getStartPort()); this.exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(workerConfig.getCapacity())); } @@ -121,7 +123,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer final File attemptDir = new File(taskDir, attemptUUID); final ProcessHolder processHolder; - + final int childPort = portFinder.findUnusedPort(); try { final Closer closer = Closer.create(); try { @@ -154,7 +156,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer } final List command = Lists.newArrayList(); - final int childPort = findUnusedPort(); final String childHost = String.format("%s:%d", node.getHostNoPort(), childPort); command.add(config.getJavaCommand()); @@ -258,7 +259,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer taskWorkItem.processHolder.process.destroy(); } } - + portFinder.markPortUnused(childPort); log.info("Removing temporary directory: %s", attemptDir); FileUtils.deleteDirectory(attemptDir); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java b/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java new file mode 100644 index 00000000000..7fbe7078478 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/PortFinder.java @@ -0,0 +1,94 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.indexing.overlord; + +import com.google.common.collect.Sets; +import com.metamx.common.ISE; + +import java.io.IOException; +import java.net.BindException; +import java.net.ServerSocket; +import java.util.Set; + +public class PortFinder +{ + private final Set usedPorts = Sets.newHashSet(); + private final int startPort; + + public PortFinder(int startPort) + { + this.startPort = startPort; + } + + private static boolean canBind(int portNum) + { + ServerSocket ss = null; + boolean isFree = false; + try { + ss = new ServerSocket(portNum); + isFree = true; + } + catch (BindException be) { + isFree = false; // port in use, + } + catch (IOException e) { + throw new RuntimeException(e); + } + finally { + if (ss != null) { + while (!ss.isClosed()) { + try { + ss.close(); + } + catch (IOException e) { + // ignore + } + } + } + } + return isFree; + } + + public synchronized int findUnusedPort() + { + int port = chooseNext(startPort); + while (!canBind(port)) { + port = chooseNext(port + 1); + } + usedPorts.add(port); + return port; + } + + public synchronized void markPortUnused(int port) + { + usedPorts.remove(port); + } + + private int chooseNext(int start) + { + for (int i = start; i < Integer.MAX_VALUE; i++) { + if (!usedPorts.contains(i)) { + return i; + } + } + throw new ISE("All ports are Used.."); + } +} + diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/PortFinderTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/PortFinderTest.java new file mode 100644 index 00000000000..43fb72f05e4 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/PortFinderTest.java @@ -0,0 +1,34 @@ +package io.druid.indexing.overlord; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.net.ServerSocket; + +public class PortFinderTest +{ + private final PortFinder finder = new PortFinder(1200); + + @Test + public void testUsedPort() throws IOException + { + final int port1 = finder.findUnusedPort(); + // verify that the port is free + ServerSocket socket1 = new ServerSocket(port1); + finder.markPortUnused(port1); + final int port2 = finder.findUnusedPort(); + Assert.assertNotEquals("Used port is not reallocated", port1, port2); + // verify that port2 is free + ServerSocket socket2 = new ServerSocket(port2); + + socket1.close(); + // Now port1 should get recycled + Assert.assertEquals(port1, finder.findUnusedPort()); + + socket2.close(); + finder.markPortUnused(port1); + finder.markPortUnused(port2); + + } +} From c1cd9ff2ceb107fe89e885cdac6d2fdec002e0fa Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 2 Jul 2014 23:41:37 -0600 Subject: [PATCH 132/270] fix broken rename of rejection policies --- .../druid/segment/realtime/plumber/FlushingPlumberSchool.java | 2 +- .../druid/segment/realtime/plumber/RealtimePlumberSchool.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 46c3d91c96a..611d16b476f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -65,7 +65,7 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("segmentGranularity") Granularity segmentGranularity, @JsonProperty("versioningPolicy") VersioningPolicy versioningPolicy, - @JsonProperty("rejectionPolicyFactory") RejectionPolicyFactory rejectionPolicyFactory, + @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, @JsonProperty("maxPendingPersists") int maxPendingPersists ) { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java index cd000651709..43020d7cc53 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -75,7 +75,7 @@ public class RealtimePlumberSchool implements PlumberSchool @JsonProperty("basePersistDirectory") File basePersistDirectory, @JsonProperty("segmentGranularity") Granularity segmentGranularity, @JsonProperty("versioningPolicy") VersioningPolicy versioningPolicy, - @JsonProperty("rejectionPolicyFactory") RejectionPolicyFactory rejectionPolicyFactory, + @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, @JsonProperty("maxPendingPersists") int maxPendingPersists ) { From 4ab9af36b0e8aef820d8387af54c98adc6be5818 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Thu, 3 Jul 2014 16:52:47 -0700 Subject: [PATCH 133/270] License header and utf-8 --- .../s3/AWSSessionCredentialsAdapter.java | 19 +++++++++++++++ .../s3/FileSessionCredentialsProvider.java | 23 +++++++++++++++++-- .../s3/TestAWSCredentialsProvider.java | 19 +++++++++++++++ .../TestFileSessionCredentialsProvider.java | 19 +++++++++++++++ 4 files changed, 78 insertions(+), 2 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java b/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java index abdc5e5cde8..1fb81fdbbca 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/AWSSessionCredentialsAdapter.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.s3; import com.amazonaws.auth.AWSCredentialsProvider; diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java index 35a999682ba..9a0030f704b 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java @@ -1,11 +1,30 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.s3; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSSessionCredentials; +import com.google.common.base.Charsets; import java.io.IOException; -import java.nio.charset.Charset; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; @@ -45,7 +64,7 @@ public class FileSessionCredentialsProvider implements AWSCredentialsProvider { @Override public void refresh() { try { - List lines = Files.readAllLines(Paths.get(sessionCredentials), Charset.defaultCharset()); + List lines = Files.readAllLines(Paths.get(sessionCredentials), Charsets.UTF_8); Properties props = new Properties(); for (String line : lines) { String[] tokens = line.split("="); diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java b/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java index e98e178550f..ad3d33995d4 100644 --- a/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java +++ b/s3-extensions/src/test/java/io/druid/storage/s3/TestAWSCredentialsProvider.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.s3; import com.amazonaws.auth.AWSCredentials; diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java b/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java index c03dae19f77..8d9743e80c8 100644 --- a/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java +++ b/s3-extensions/src/test/java/io/druid/storage/s3/TestFileSessionCredentialsProvider.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.storage.s3; import com.amazonaws.auth.AWSSessionCredentials; From 97b58eb193080df1c8ce84be5e7a768a9a88565d Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 4 Jul 2014 12:31:33 +0530 Subject: [PATCH 134/270] review comments 1) Rename firehose to IngestSegment 2) fix segment overlapping, intervals 3) fix overshadow --- .../guice/IndexingServiceFirehoseModule.java | 4 +- ...java => IngestSegmentFirehoseFactory.java} | 45 +++++++++---------- 2 files changed, 23 insertions(+), 26 deletions(-) rename indexing-service/src/main/java/io/druid/indexing/firehose/{DruidFirehoseFactory.java => IngestSegmentFirehoseFactory.java} (90%) diff --git a/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java b/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java index 55f94147b67..988ee329d00 100644 --- a/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java +++ b/indexing-service/src/main/java/io/druid/guice/IndexingServiceFirehoseModule.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; -import io.druid.indexing.firehose.DruidFirehoseFactory; +import io.druid.indexing.firehose.IngestSegmentFirehoseFactory; import io.druid.initialization.DruidModule; import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; @@ -39,7 +39,7 @@ public class IndexingServiceFirehoseModule implements DruidModule new SimpleModule("IndexingServiceFirehoseModule") .registerSubtypes( new NamedType(EventReceiverFirehoseFactory.class, "receiver"), - new NamedType(DruidFirehoseFactory.class, "druid") + new NamedType(IngestSegmentFirehoseFactory.class, "ingestSegment") ) ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java similarity index 90% rename from indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java rename to indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 215ddd937e4..f8f603b6c23 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/DruidFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -61,6 +61,7 @@ import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineObjectHolder; import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.utils.Runnables; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -72,9 +73,9 @@ import java.util.List; import java.util.Map; import java.util.Set; -public class DruidFirehoseFactory implements FirehoseFactory +public class IngestSegmentFirehoseFactory implements FirehoseFactory { - private static final EmittingLogger log = new EmittingLogger(DruidFirehoseFactory.class); + private static final EmittingLogger log = new EmittingLogger(IngestSegmentFirehoseFactory.class); private final String dataSource; private final Interval interval; private final DimFilter dimFilter; @@ -83,7 +84,7 @@ public class DruidFirehoseFactory implements FirehoseFactory private final Injector injector; @JsonCreator - public DruidFirehoseFactory( + public IngestSegmentFirehoseFactory( @JsonProperty("dataSource") final String dataSource, @JsonProperty("interval") Interval interval, @JsonProperty("filter") DimFilter dimFilter, @@ -155,6 +156,9 @@ public class DruidFirehoseFactory implements FirehoseFactory VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( Ordering.natural().nullsFirst() ); + final List> timeLineSegments = timeline.lookup( + interval + ); for (DataSegment segment : usedSegments) { timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); } @@ -163,8 +167,8 @@ public class DruidFirehoseFactory implements FirehoseFactory dims = dimensions; } else { Set dimSet = new HashSet<>(); - for (DataSegment segment : usedSegments) { - dimSet.addAll(segment.getDimensions()); + for (TimelineObjectHolder timelineObjectHolder : timeLineSegments) { + dimSet.addAll(timelineObjectHolder.getObject().getChunk(0).getObject().getDimensions()); } dims = Lists.newArrayList(dimSet); } @@ -174,15 +178,15 @@ public class DruidFirehoseFactory implements FirehoseFactory metricsList = metrics; } else { Set metricsSet = new HashSet<>(); - for (DataSegment segment : usedSegments) { - metricsSet.addAll(segment.getMetrics()); + for (TimelineObjectHolder timelineObjectHolder : timeLineSegments) { + metricsSet.addAll(timelineObjectHolder.getObject().getChunk(0).getObject().getDimensions()); } metricsList = Lists.newArrayList(metricsSet); } final List adapters = Lists.transform( - timeline.lookup(new Interval("1000-01-01/3000-01-01")), + timeLineSegments, new Function, StorageAdapter>() { @Override @@ -234,11 +238,11 @@ public class DruidFirehoseFactory implements FirehoseFactory { @Nullable @Override - public Sequence apply(@Nullable StorageAdapter input) + public Sequence apply(@Nullable StorageAdapter adapter) { return Sequences.concat( Sequences.map( - input.makeCursors( + adapter.makeCursors( Filters.convertDimensionFilters(dimFilter), interval, QueryGranularity.ALL @@ -246,24 +250,24 @@ public class DruidFirehoseFactory implements FirehoseFactory { @Nullable @Override - public Sequence apply(@Nullable Cursor input) + public Sequence apply(@Nullable Cursor cursor) { - TimestampColumnSelector timestampColumnSelector = input.makeTimestampColumnSelector(); + TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { - final DimensionSelector dimSelector = input.makeDimensionSelector(dim); + final DimensionSelector dimSelector = cursor.makeDimensionSelector(dim); dimSelectors.put(dim, dimSelector); } Map metSelectors = Maps.newHashMap(); for (String metric : metrics) { - final ObjectColumnSelector metricSelector = input.makeObjectColumnSelector(metric); + final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric); metSelectors.put(metric, metricSelector); } List rowList = Lists.newArrayList(); - while (!input.isDone()) { + while (!cursor.isDone()) { final Map theEvent = Maps.newLinkedHashMap(); final long timestamp = timestampColumnSelector.getTimestamp(); theEvent.put(EventHolder.timestampKey, new DateTime(timestamp)); @@ -291,7 +295,7 @@ public class DruidFirehoseFactory implements FirehoseFactory theEvent.put(metric, selector.get()); } rowList.add(new MapBasedInputRow(timestamp, dims, theEvent)); - input.advance(); + cursor.advance(); } return Sequences.simple(rowList); } @@ -333,14 +337,7 @@ public class DruidFirehoseFactory implements FirehoseFactory @Override public Runnable commit() { - return new Runnable() - { - @Override - public void run() - { - // Nothing to do. - } - }; + return Runnables.getNoopRunnable(); } @Override From ce478be8992c26220d9a34992198968210640976 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 7 Jul 2014 18:08:37 -0600 Subject: [PATCH 135/270] Remove redundant forking task runner config --- docs/content/Middlemanager.md | 1 - docs/content/Production-Cluster-Configuration.md | 3 +-- .../io/druid/indexing/overlord/ForkingTaskRunner.java | 6 +++++- .../indexing/overlord/ForkingTaskRunnerFactory.java | 6 +++++- .../overlord/config/ForkingTaskRunnerConfig.java | 9 --------- 5 files changed, 11 insertions(+), 14 deletions(-) diff --git a/docs/content/Middlemanager.md b/docs/content/Middlemanager.md index 7b71db207ba..5736268a693 100644 --- a/docs/content/Middlemanager.md +++ b/docs/content/Middlemanager.md @@ -48,7 +48,6 @@ Middle managers pass their configurations down to their child peons. The middle |`druid.worker.capacity`|Maximum number of tasks the middle manager can accept.|Number of available processors - 1| |`druid.indexer.runner.compressZnodes`|Indicates whether or not the middle managers should compress Znodes.|false| |`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288| -|`druid.indexer.runner.taskDir`|Temporary intermediate directory used during task execution.|/tmp/persistent| |`druid.indexer.runner.javaCommand`|Command required to execute java.|java| |`druid.indexer.runner.javaOpts`|-X Java options to run the peon in its own JVM.|""| |`druid.indexer.runner.classpath`|Java classpath for the peon.|System.getProperty("java.class.path")| diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index cb98fdf9d43..6eeb3a0c97b 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -154,8 +154,7 @@ druid.indexer.logs.s3Prefix=prod/logs/v1 # Dedicate more resources to peons druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -druid.indexer.runner.taskDir=/mnt/persistent/task/ -druid.indexer.task.taskDir=/mnt/persistent/task/ +druid.indexer.task.baseTaskDir=/mnt/persistent/task/ druid.indexer.task.chathandler.type=announce druid.indexer.fork.property.druid.indexer.hadoopWorkingPath=/tmp/druid-indexing diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index 7fe20ef9eab..0c740fb9df4 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -42,6 +42,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.emitter.EmittingLogger; import io.druid.guice.annotations.Self; import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.indexing.worker.config.WorkerConfig; @@ -74,6 +75,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer private static final Splitter whiteSpaceSplitter = Splitter.on(CharMatcher.WHITESPACE).omitEmptyStrings(); private final ForkingTaskRunnerConfig config; + private final TaskConfig taskConfig; private final Properties props; private final TaskLogPusher taskLogPusher; private final DruidNode node; @@ -86,6 +88,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer @Inject public ForkingTaskRunner( ForkingTaskRunnerConfig config, + TaskConfig taskConfig, WorkerConfig workerConfig, Properties props, TaskLogPusher taskLogPusher, @@ -94,6 +97,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer ) { this.config = config; + this.taskConfig = taskConfig; this.props = props; this.taskLogPusher = taskLogPusher; this.jsonMapper = jsonMapper; @@ -119,7 +123,7 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer public TaskStatus call() { final String attemptUUID = UUID.randomUUID().toString(); - final File taskDir = new File(config.getTaskDir(), task.getId()); + final File taskDir = new File(taskConfig.getBaseTaskDir(), task.getId()); final File attemptDir = new File(taskDir, attemptUUID); final ProcessHolder processHolder; diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java index 842159b1e3a..94f77089006 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunnerFactory.java @@ -22,6 +22,7 @@ package io.druid.indexing.overlord; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import io.druid.guice.annotations.Self; +import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.overlord.config.ForkingTaskRunnerConfig; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.server.DruidNode; @@ -34,6 +35,7 @@ import java.util.Properties; public class ForkingTaskRunnerFactory implements TaskRunnerFactory { private final ForkingTaskRunnerConfig config; + private final TaskConfig taskConfig; private final WorkerConfig workerConfig; private final Properties props; private final ObjectMapper jsonMapper; @@ -43,6 +45,7 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory @Inject public ForkingTaskRunnerFactory( final ForkingTaskRunnerConfig config, + final TaskConfig taskConfig, final WorkerConfig workerConfig, final Properties props, final ObjectMapper jsonMapper, @@ -50,6 +53,7 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory @Self DruidNode node ) { this.config = config; + this.taskConfig = taskConfig; this.workerConfig = workerConfig; this.props = props; this.jsonMapper = jsonMapper; @@ -60,6 +64,6 @@ public class ForkingTaskRunnerFactory implements TaskRunnerFactory @Override public TaskRunner build() { - return new ForkingTaskRunner(config, workerConfig, props, persistentTaskLogs, jsonMapper, node); + return new ForkingTaskRunner(config, taskConfig, workerConfig, props, persistentTaskLogs, jsonMapper, node); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java index b1562941aba..8d89d834785 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/config/ForkingTaskRunnerConfig.java @@ -29,10 +29,6 @@ import java.util.List; public class ForkingTaskRunnerConfig { - @JsonProperty - @NotNull - private String taskDir = "/tmp/persistent"; - @JsonProperty @NotNull private String javaCommand = "java"; @@ -66,11 +62,6 @@ public class ForkingTaskRunnerConfig "java.io.tmpdir" ); - public String getTaskDir() - { - return taskDir; - } - public String getJavaCommand() { return javaCommand; From 3a849b03416f2be9cd81460c3841b3fa40464309 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Mon, 7 Jul 2014 23:11:08 -0700 Subject: [PATCH 136/270] update refresh manually every hour --- .../s3/FileSessionCredentialsProvider.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java index 9a0030f704b..dcaa404286a 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java @@ -23,12 +23,16 @@ import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSSessionCredentials; import com.google.common.base.Charsets; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; import java.util.Properties; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; public class FileSessionCredentialsProvider implements AWSCredentialsProvider { private final String sessionCredentials; @@ -36,9 +40,20 @@ public class FileSessionCredentialsProvider implements AWSCredentialsProvider { private volatile String accessKey; private volatile String secretKey; + private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("FileSessionCredentialsProviderRefresh-%d").build() + ); + public FileSessionCredentialsProvider(String sessionCredentials) { this.sessionCredentials = sessionCredentials; refresh(); + + scheduler.scheduleAtFixedRate(new Runnable() { + @Override + public void run() { + refresh(); + } + }, 1, 1, TimeUnit.HOURS); // refresh every hour } @Override From 36fc85736c3f4233dee9f28894b95e82467c34c6 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 8 Jul 2014 18:01:31 +0530 Subject: [PATCH 137/270] Add ShardSpec Lookup Optimize choosing shardSpec for Hash Partitions --- .../indexer/HadoopDruidIndexerConfig.java | 56 +++++++++++++------ .../partition/HashBasedNumberedShardSpec.java | 20 +++++++ .../timeline/partition/LinearShardSpec.java | 16 ++++++ .../timeline/partition/NumberedShardSpec.java | 16 ++++++ .../partition/SingleDimensionShardSpec.java | 19 +++++++ 5 files changed, 110 insertions(+), 17 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 8f0f2d43c34..11acfd8df6e 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Maps; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -30,6 +31,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Splitter; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; @@ -51,6 +53,7 @@ import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.server.DruidNode; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.ShardSpec; +import io.druid.timeline.partition.ShardSpecLookup; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -60,6 +63,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.charset.Charset; @@ -169,6 +173,8 @@ public class HadoopDruidIndexerConfig private volatile HadoopIngestionSpec schema; private volatile PathSpec pathSpec; private volatile ColumnConfig columnConfig; + private volatile Map shardSpecLookups = Maps.newHashMap(); + private volatile Map hadoopShardSpecLookup = Maps.newHashMap(); @JsonCreator public HadoopDruidIndexerConfig( @@ -178,6 +184,30 @@ public class HadoopDruidIndexerConfig this.columnConfig = columnConfig; this.schema = schema; this.pathSpec = jsonMapper.convertValue(schema.getIOConfig().getPathSpec(), PathSpec.class); + for (Map.Entry> entry : schema.getTuningConfig().getShardSpecs().entrySet()) { + if (entry.getValue() == null || entry.getValue().isEmpty()) { + continue; + } + final ShardSpec actualSpec = entry.getValue().get(0).getActualSpec(); + shardSpecLookups.put( + entry.getKey(), actualSpec.getLookup( + Lists.transform( + entry.getValue(), new Function() + { + @Nullable + @Override + public ShardSpec apply(@Nullable HadoopyShardSpec input) + { + return input.getActualSpec(); + } + } + ) + ) + ); + for (HadoopyShardSpec hadoopyShardSpec : entry.getValue()) { + hadoopShardSpecLookup.put(hadoopyShardSpec.getActualSpec(), hadoopyShardSpec); + } + } } @JsonProperty @@ -306,25 +336,17 @@ public class HadoopDruidIndexerConfig return Optional.absent(); } - final List shards = schema.getTuningConfig().getShardSpecs().get(timeBucket.get().getStart()); - if (shards == null || shards.isEmpty()) { - return Optional.absent(); - } + final ShardSpec actualSpec = shardSpecLookups.get(timeBucket.get().getStart()).getShardSpec(inputRow); + final HadoopyShardSpec hadoopyShardSpec = hadoopShardSpecLookup.get(actualSpec); - for (final HadoopyShardSpec hadoopyShardSpec : shards) { - final ShardSpec actualSpec = hadoopyShardSpec.getActualSpec(); - if (actualSpec.isInChunk(inputRow)) { - return Optional.of( - new Bucket( - hadoopyShardSpec.getShardNum(), - timeBucket.get().getStart(), - actualSpec.getPartitionNum() - ) - ); - } - } + return Optional.of( + new Bucket( + hadoopyShardSpec.getShardNum(), + timeBucket.get().getStart(), + actualSpec.getPartitionNum() + ) + ); - throw new ISE("row[%s] doesn't fit in any shard[%s]", inputRow, shards); } public Optional> getSegmentGranularIntervals() diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index 5110f886601..ac8b570578b 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -25,12 +25,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import io.druid.data.input.InputRow; import io.druid.data.input.Rows; import java.util.List; +import java.util.Map; public class HashBasedNumberedShardSpec extends NumberedShardSpec { @@ -74,4 +76,22 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec '}'; } + @Override + public ShardSpecLookup getLookup(final List shardSpecs) + { + final ImmutableMap.Builder shardSpecsMapBuilder = ImmutableMap.builder(); + for (ShardSpec spec : shardSpecs) { + shardSpecsMapBuilder.put(spec.getPartitionNum(), spec); + } + final Map shardSpecMap = shardSpecsMapBuilder.build(); + + return new ShardSpecLookup() + { + @Override + public ShardSpec getShardSpec(InputRow row) + { + return shardSpecMap.get((long) hash(row) % getPartitions()); + } + }; + } } \ No newline at end of file diff --git a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java index ea7d3256229..6f9dd6258e0 100644 --- a/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/LinearShardSpec.java @@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import io.druid.data.input.InputRow; +import java.util.List; +import java.util.Set; + public class LinearShardSpec implements ShardSpec { private int partitionNum; @@ -42,6 +45,19 @@ public class LinearShardSpec implements ShardSpec return partitionNum; } + @Override + public ShardSpecLookup getLookup(final List shardSpecs) + { + return new ShardSpecLookup() + { + @Override + public ShardSpec getShardSpec(InputRow row) + { + return shardSpecs.get(0); + } + }; + } + @Override public PartitionChunk createChunk(T obj) { return new LinearPartitionChunk(partitionNum, obj); diff --git a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java index 73a3437a80a..683836d95fb 100644 --- a/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/NumberedShardSpec.java @@ -25,6 +25,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import io.druid.data.input.InputRow; +import java.util.List; +import java.util.Set; + public class NumberedShardSpec implements ShardSpec { @JsonIgnore @@ -52,6 +55,19 @@ public class NumberedShardSpec implements ShardSpec return partitionNum; } + @Override + public ShardSpecLookup getLookup(final List shardSpecs) + { + return new ShardSpecLookup() + { + @Override + public ShardSpec getShardSpec(InputRow row) + { + return shardSpecs.get(0); + } + }; + } + @JsonProperty("partitions") public int getPartitions() { diff --git a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java index 3cb3e5a72ba..197d5e6129f 100644 --- a/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/SingleDimensionShardSpec.java @@ -20,6 +20,7 @@ package io.druid.timeline.partition; import com.fasterxml.jackson.annotation.JsonProperty; +import com.metamx.common.ISE; import io.druid.data.input.InputRow; import java.util.List; @@ -94,6 +95,24 @@ public class SingleDimensionShardSpec implements ShardSpec return partitionNum; } + @Override + public ShardSpecLookup getLookup(final List shardSpecs) + { + return new ShardSpecLookup() + { + @Override + public ShardSpec getShardSpec(InputRow row) + { + for (ShardSpec spec : shardSpecs) { + if (spec.isInChunk(row)) { + return spec; + } + } + throw new ISE("row[%s] doesn't fit in any shard[%s]", row, shardSpecs); + } + }; + } + public void setPartitionNum(int partitionNum) { this.partitionNum = partitionNum; From a1a5e4254fb54f20a0f1432fcc879b650ebcbec2 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 8 Jul 2014 21:14:48 +0530 Subject: [PATCH 138/270] CombiningFirehoseFactory can be used to ingest data from multiple sources. --- .../java/io/druid/guice/FirehoseModule.java | 4 +- .../firehose/CombiningFirehoseFactory.java | 125 +++++++++++++++ .../CombiningFirehoseFactoryTest.java | 147 ++++++++++++++++++ 3 files changed, 275 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java create mode 100644 server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java diff --git a/server/src/main/java/io/druid/guice/FirehoseModule.java b/server/src/main/java/io/druid/guice/FirehoseModule.java index ac10c297dd4..472fa2ff8fa 100644 --- a/server/src/main/java/io/druid/guice/FirehoseModule.java +++ b/server/src/main/java/io/druid/guice/FirehoseModule.java @@ -26,6 +26,7 @@ import com.google.inject.Binder; import io.druid.data.input.ProtoBufInputRowParser; import io.druid.initialization.DruidModule; import io.druid.segment.realtime.firehose.ClippedFirehoseFactory; +import io.druid.segment.realtime.firehose.CombiningFirehoseFactory; import io.druid.segment.realtime.firehose.EventReceiverFirehoseFactory; import io.druid.segment.realtime.firehose.IrcFirehoseFactory; import io.druid.segment.realtime.firehose.LocalFirehoseFactory; @@ -53,7 +54,8 @@ public class FirehoseModule implements DruidModule new NamedType(TimedShutoffFirehoseFactory.class, "timed"), new NamedType(IrcFirehoseFactory.class, "irc"), new NamedType(LocalFirehoseFactory.class, "local"), - new NamedType(EventReceiverFirehoseFactory.class, "receiver") + new NamedType(EventReceiverFirehoseFactory.class, "receiver"), + new NamedType(CombiningFirehoseFactory.class, "combining") ) ); } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java new file mode 100644 index 00000000000..f4a5c8bba6d --- /dev/null +++ b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java @@ -0,0 +1,125 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.segment.realtime.firehose; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.repackaged.com.google.common.base.Preconditions; +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; + +/** + * Creates firehose that combines data from different Firehoses. Useful for ingesting data from multiple sources. + */ +public class CombiningFirehoseFactory implements FirehoseFactory +{ + private final List delegateFactoryList; + + @JsonCreator + public CombiningFirehoseFactory( + @JsonProperty("delegates") List delegateFactoryList + ) + { + Preconditions.checkArgument(!delegateFactoryList.isEmpty()); + this.delegateFactoryList = delegateFactoryList; + } + + @Override + public Firehose connect(InputRowParser parser) throws IOException + { + return new CombiningFirehose(parser); + } + + @Override + public InputRowParser getParser() + { + return delegateFactoryList.get(0).getParser(); + } + + @JsonProperty("delegates") + public List getDelegateFactoryList() + { + return delegateFactoryList; + } + + public class CombiningFirehose implements Firehose + { + private final InputRowParser parser; + private final Iterator firehoseFactoryIterator; + private volatile Firehose currentFirehose; + + public CombiningFirehose(InputRowParser parser) throws IOException + { + this.firehoseFactoryIterator = delegateFactoryList.iterator(); + this.parser = parser; + nextFirehose(); + } + + private void nextFirehose() + { + if (firehoseFactoryIterator.hasNext()) { + try { + if (currentFirehose != null) { + currentFirehose.close(); + } + currentFirehose = firehoseFactoryIterator.next().connect(parser); + } + catch (IOException e) { + Throwables.propagate(e); + } + } + } + + @Override + public boolean hasMore() + { + return currentFirehose.hasMore(); + } + + @Override + public InputRow nextRow() + { + InputRow rv = currentFirehose.nextRow(); + if (!currentFirehose.hasMore()) { + nextFirehose(); + } + return rv; + } + + @Override + public Runnable commit() + { + return currentFirehose.commit(); + } + + @Override + public void close() throws IOException + { + currentFirehose.close(); + } + } +} diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java new file mode 100644 index 00000000000..a69327b86be --- /dev/null +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -0,0 +1,147 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.realtime.firehose; + +import com.google.common.collect.Lists; +import com.metamx.common.parsers.ParseException; +import io.druid.data.input.Firehose; +import io.druid.data.input.FirehoseFactory; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; +import io.druid.segment.realtime.firehose.CombiningFirehoseFactory; +import io.druid.utils.Runnables; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class CombiningFirehoseFactoryTest +{ + + @Test + public void testCombiningfirehose() throws IOException + { + List list1 = Arrays.asList(makeRow(1, 1), makeRow(2, 2)); + List list2 = Arrays.asList(makeRow(3, 3), makeRow(4, 4)); + FirehoseFactory combiningFactory = new CombiningFirehoseFactory( + Arrays.asList( + new ListFirehoseFactory( + list1 + ), new ListFirehoseFactory(list2) + ) + ); + final Firehose firehose = combiningFactory.connect(null); + for (int i = 1; i < 5; i++) { + Assert.assertTrue(firehose.hasMore()); + final InputRow inputRow = firehose.nextRow(); + Assert.assertEquals(i, inputRow.getTimestampFromEpoch()); + Assert.assertEquals(i, inputRow.getFloatMetric("test"), 0); + } + Assert.assertFalse(firehose.hasMore()); + } + + private InputRow makeRow(final long timestamp, final float metricValue) + { + return new InputRow() + { + @Override + public List getDimensions() + { + return Arrays.asList("testDim"); + } + + @Override + public long getTimestampFromEpoch() + { + return timestamp; + } + + @Override + public List getDimension(String dimension) + { + return Lists.newArrayList(); + } + + @Override + public float getFloatMetric(String metric) + { + return metricValue; + } + + @Override + public Object getRaw(String dimension) + { + return null; + } + + }; + } + + public static class ListFirehoseFactory implements FirehoseFactory + { + private final List rows; + + ListFirehoseFactory(List rows) + { + this.rows = rows; + } + + @Override + public Firehose connect(InputRowParser inputRowParser) throws IOException, ParseException + { + final Iterator iterator = rows.iterator(); + return new Firehose() + { + @Override + public boolean hasMore() + { + return iterator.hasNext(); + } + + @Override + public InputRow nextRow() + { + return iterator.next(); + } + + @Override + public Runnable commit() + { + return Runnables.getNoopRunnable(); + } + + @Override + public void close() throws IOException + { + // + } + }; + } + + @Override + public InputRowParser getParser() + { + return null; + } + } +} From 7168adcca7be87a8d91882fbebb96b296bd7381b Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Tue, 8 Jul 2014 21:20:17 +0530 Subject: [PATCH 139/270] Use Ingest task instead of Noop Task --- .../IngestSegmentFirehoseFactory.java | 41 +++++++++++++++---- 1 file changed, 32 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index f8f603b6c23..2d270ccdeb1 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -42,10 +42,12 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; +import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.actions.SegmentListUsedAction; -import io.druid.indexing.common.task.NoopTask; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.AbstractTask; import io.druid.query.filter.DimFilter; import io.druid.query.select.EventHolder; import io.druid.segment.Cursor; @@ -137,15 +139,9 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory rowYielder; From 193c7259f6ce3155f8a38343be3976ee22763f43 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Wed, 9 Jul 2014 15:52:17 -0700 Subject: [PATCH 140/270] refresh thread should be daemon --- .../io/druid/storage/s3/FileSessionCredentialsProvider.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java index dcaa404286a..dbb1a288407 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java @@ -41,7 +41,8 @@ public class FileSessionCredentialsProvider implements AWSCredentialsProvider { private volatile String secretKey; private final ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor( - new ThreadFactoryBuilder().setNameFormat("FileSessionCredentialsProviderRefresh-%d").build() + new ThreadFactoryBuilder().setNameFormat("FileSessionCredentialsProviderRefresh-%d") + .setDaemon(true).build() ); public FileSessionCredentialsProvider(String sessionCredentials) { From fa43049240e69b9a4dfb344b97f762d57fabd3fe Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 10 Jul 2014 11:48:46 +0530 Subject: [PATCH 141/270] review comments & pom changes --- .../indexer/HadoopDruidIndexerConfig.java | 7 ++----- pom.xml | 2 +- .../partition/HashBasedNumberedShardSpec.java | 21 ++++++++++++------- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index 11acfd8df6e..1ff6e5a4482 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -36,17 +36,16 @@ import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; -import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.impl.StringInputRowParser; +import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; import io.druid.indexer.partitions.PartitionsSpec; import io.druid.indexer.path.PathSpec; -import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.GranularitySpec; @@ -63,7 +62,6 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.charset.Charset; @@ -194,9 +192,8 @@ public class HadoopDruidIndexerConfig Lists.transform( entry.getValue(), new Function() { - @Nullable @Override - public ShardSpec apply(@Nullable HadoopyShardSpec input) + public ShardSpec apply(HadoopyShardSpec input) { return input.getActualSpec(); } diff --git a/pom.xml b/pom.xml index 2b7f88276e8..7d270176674 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.5 2.5.0 - 0.2.4 + 0.2.5 diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index ac8b570578b..afcb2feb922 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -25,14 +25,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.api.client.repackaged.com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import io.druid.data.input.InputRow; import io.druid.data.input.Rows; +import java.util.Collections; +import java.util.Comparator; import java.util.List; -import java.util.Map; public class HashBasedNumberedShardSpec extends NumberedShardSpec { @@ -79,18 +79,25 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec @Override public ShardSpecLookup getLookup(final List shardSpecs) { - final ImmutableMap.Builder shardSpecsMapBuilder = ImmutableMap.builder(); - for (ShardSpec spec : shardSpecs) { - shardSpecsMapBuilder.put(spec.getPartitionNum(), spec); + // Sort on basis of partitionNumber + Collections.sort( + shardSpecs, new Comparator() + { + @Override + public int compare(ShardSpec o1, ShardSpec o2) + { + return Integer.compare(o1.getPartitionNum(), o2.getPartitionNum()); + } } - final Map shardSpecMap = shardSpecsMapBuilder.build(); + ); return new ShardSpecLookup() { @Override public ShardSpec getShardSpec(InputRow row) { - return shardSpecMap.get((long) hash(row) % getPartitions()); + int index = (int) ((long) hash(row)) % getPartitions(); + return shardSpecs.get(index); } }; } From a12688bc8bab700899af0a525a76c22d79aceab8 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 10 Jul 2014 12:21:27 +0530 Subject: [PATCH 142/270] fix partitionNum calculation & remove extra sorting --- pom.xml | 2 +- .../partition/HashBasedNumberedShardSpec.java | 16 +--------------- .../shard/HashBasedNumberedShardSpecTest.java | 12 ++++++++++++ 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index 7d270176674..dfa203ec905 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.5 2.5.0 - 0.2.5 + 0.2.5-SNAPSHOT diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index afcb2feb922..8f347ee6cfd 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -30,8 +30,6 @@ import com.google.common.hash.Hashing; import io.druid.data.input.InputRow; import io.druid.data.input.Rows; -import java.util.Collections; -import java.util.Comparator; import java.util.List; public class HashBasedNumberedShardSpec extends NumberedShardSpec @@ -79,24 +77,12 @@ public class HashBasedNumberedShardSpec extends NumberedShardSpec @Override public ShardSpecLookup getLookup(final List shardSpecs) { - // Sort on basis of partitionNumber - Collections.sort( - shardSpecs, new Comparator() - { - @Override - public int compare(ShardSpec o1, ShardSpec o2) - { - return Integer.compare(o1.getPartitionNum(), o2.getPartitionNum()); - } - } - ); - return new ShardSpecLookup() { @Override public ShardSpec getShardSpec(InputRow row) { - int index = (int) ((long) hash(row)) % getPartitions(); + int index = Math.abs(hash(row) % getPartitions()); return shardSpecs.get(index); } }; diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index bfd573dae89..5f176015c2a 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -194,4 +194,16 @@ public class HashBasedNumberedShardSpecTest return 0; } } + + @Test + public void testValidity(){ + for(int i=Integer.MIN_VALUE;i<=Integer.MAX_VALUE;i++){ + { + int partitionNum = Math.abs((int) ((long) i % 2)); + if(partitionNum != 0 && partitionNum != 1){ + throw new ISE("for i "+ i+ "partitionNum "+ partitionNum); + } + } + } + } } From a31376ee8371a62a092662c04ddde3a360d0d009 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 10 Jul 2014 12:23:20 +0530 Subject: [PATCH 143/270] point to correct version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index dfa203ec905..7d270176674 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.5 2.5.0 - 0.2.5-SNAPSHOT + 0.2.5 From 972c5dac31b68fc321cb2a408a516daadb6b9095 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Mon, 14 Jul 2014 21:17:53 +0530 Subject: [PATCH 144/270] improve memory usage and rename firehose --- .../IngestSegmentFirehoseFactory.java | 97 ++++++++++++------- 1 file changed, 61 insertions(+), 36 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 2d270ccdeb1..85f1c0b4c59 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -71,6 +71,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -138,7 +139,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory rowYielder; - public DruidFirehose(List adapters, final List dims, final List metrics) + public IngestSegmentFirehose(List adapters, final List dims, final List metrics) { Sequence rows = Sequences.concat( Iterables.transform( @@ -273,54 +274,78 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory apply(@Nullable Cursor cursor) + public Sequence apply(@Nullable final Cursor cursor) { - TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); + final TimestampColumnSelector timestampColumnSelector = cursor.makeTimestampColumnSelector(); - Map dimSelectors = Maps.newHashMap(); + final Map dimSelectors = Maps.newHashMap(); for (String dim : dims) { final DimensionSelector dimSelector = cursor.makeDimensionSelector(dim); dimSelectors.put(dim, dimSelector); } - Map metSelectors = Maps.newHashMap(); + final Map metSelectors = Maps.newHashMap(); for (String metric : metrics) { final ObjectColumnSelector metricSelector = cursor.makeObjectColumnSelector(metric); metSelectors.put(metric, metricSelector); } - List rowList = Lists.newArrayList(); - while (!cursor.isDone()) { - final Map theEvent = Maps.newLinkedHashMap(); - final long timestamp = timestampColumnSelector.getTimestamp(); - theEvent.put(EventHolder.timestampKey, new DateTime(timestamp)); + return Sequences.simple( + new Iterable() + { + @Override + public Iterator iterator() + { + return new Iterator() + { + @Override + public boolean hasNext() + { + return !cursor.isDone(); + } - for (Map.Entry dimSelector : dimSelectors.entrySet()) { - final String dim = dimSelector.getKey(); - final DimensionSelector selector = dimSelector.getValue(); - final IndexedInts vals = selector.getRow(); + @Override + public InputRow next() + { + final Map theEvent = Maps.newLinkedHashMap(); + final long timestamp = timestampColumnSelector.getTimestamp(); + theEvent.put(EventHolder.timestampKey, new DateTime(timestamp)); - if (vals.size() == 1) { - final String dimVal = selector.lookupName(vals.get(0)); - theEvent.put(dim, dimVal); - } else { - List dimVals = Lists.newArrayList(); - for (int i = 0; i < vals.size(); ++i) { - dimVals.add(selector.lookupName(vals.get(i))); + for (Map.Entry dimSelector : dimSelectors.entrySet()) { + final String dim = dimSelector.getKey(); + final DimensionSelector selector = dimSelector.getValue(); + final IndexedInts vals = selector.getRow(); + + if (vals.size() == 1) { + final String dimVal = selector.lookupName(vals.get(0)); + theEvent.put(dim, dimVal); + } else { + List dimVals = Lists.newArrayList(); + for (int i = 0; i < vals.size(); ++i) { + dimVals.add(selector.lookupName(vals.get(i))); + } + theEvent.put(dim, dimVals); + } + } + + for (Map.Entry metSelector : metSelectors.entrySet()) { + final String metric = metSelector.getKey(); + final ObjectColumnSelector selector = metSelector.getValue(); + theEvent.put(metric, selector.get()); + } + cursor.advance(); + return new MapBasedInputRow(timestamp, dims, theEvent); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException("Remove Not Supported"); + } + }; } - theEvent.put(dim, dimVals); } - } - - for (Map.Entry metSelector : metSelectors.entrySet()) { - final String metric = metSelector.getKey(); - final ObjectColumnSelector selector = metSelector.getValue(); - theEvent.put(metric, selector.get()); - } - rowList.add(new MapBasedInputRow(timestamp, dims, theEvent)); - cursor.advance(); - } - return Sequences.simple(rowList); + ); } } ) From f56d60b4511d15c3caaeda1dee56b8deb3f14c4b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 14 Jul 2014 16:24:01 -0700 Subject: [PATCH 145/270] fix storage class npe --- .../druid/storage/s3/S3DataSegmentMover.java | 38 +++++++++++-------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index ea585a0be44..379dd8374fa 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -24,6 +24,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.inject.Inject; +import com.metamx.common.ISE; import com.metamx.common.MapUtils; import com.metamx.common.logger.Logger; import io.druid.segment.loading.DataSegmentMover; @@ -120,23 +121,30 @@ public class S3DataSegmentMover implements DataSegmentMover if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { if (s3Bucket.equals(targetS3Bucket) && s3Path.equals(targetS3Path)) { log.info("No need to move file[s3://%s/%s] onto itself", s3Bucket, s3Path); - } else if (s3Client.getObjectDetails(s3Bucket, s3Path) - .getStorageClass() - .equals(S3Object.STORAGE_CLASS_GLACIER)) { - log.warn("Cannot move file[s3://%s/%s] of storage class glacier."); } else { - log.info( - "Moving file[s3://%s/%s] to [s3://%s/%s]", - s3Bucket, - s3Path, - targetS3Bucket, - targetS3Path - ); - final S3Object target = new S3Object(targetS3Path); - if(!config.getDisableAcl()) { - target.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + final S3Object[] list = s3Client.listObjects(s3Bucket, s3Path, ""); + if (list.length == 0) { + // should never happen + throw new ISE("Unable to list object [s3://%s/%s]", s3Bucket, s3Path); + } + final S3Object s3Object = list[0]; + if (s3Object.getStorageClass() != null && + s3Object.getStorageClass().equals(S3Object.STORAGE_CLASS_GLACIER)) { + log.warn("Cannot move file[s3://%s/%s] of storage class glacier, skipping."); + } else { + log.info( + "Moving file[s3://%s/%s] to [s3://%s/%s]", + s3Bucket, + s3Path, + targetS3Bucket, + targetS3Path + ); + final S3Object target = new S3Object(targetS3Path); + if (!config.getDisableAcl()) { + target.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + } + s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, target, false); } - s3Client.moveObject(s3Bucket, s3Path, targetS3Bucket, target, false); } } else { // ensure object exists in target location From 12bc3ac27bed120cb17bca69a719a044cbcdee3c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 15 Jul 2014 10:58:46 -0700 Subject: [PATCH 146/270] fix broken S3 moving test --- .../storage/s3/S3DataSegmentMoverTest.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java index 9497d9a05fc..b5bad120d38 100644 --- a/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java +++ b/s3-extensions/src/test/java/io/druid/storage/s3/S3DataSegmentMoverTest.java @@ -133,15 +133,18 @@ public class S3DataSegmentMoverTest } @Override - public StorageObject getObjectDetails(String bucketName, String objectKey) throws ServiceException + public S3Object[] listObjects(String bucketName, String objectKey, String separator) { - if (isObjectInBucket(bucketName, objectKey)) { - final S3Object object = new S3Object(objectKey); - object.setStorageClass(S3Object.STORAGE_CLASS_STANDARD); - return object; - } else { - return null; + try { + if (isObjectInBucket(bucketName, objectKey)) { + final S3Object object = new S3Object(objectKey); + object.setStorageClass(S3Object.STORAGE_CLASS_STANDARD); + return new S3Object[]{object}; + } + } catch (ServiceException e) { + // return empty list } + return new S3Object[]{}; } @Override From 2f90bfcf9a75902906e14efde05c752788b963fc Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 15 Jul 2014 12:12:33 -0700 Subject: [PATCH 147/270] rejiggering the cache for groupBy and select --- pom.xml | 2 +- .../java/io/druid/query/CacheStrategy.java | 2 + .../src/main/java/io/druid/query/Druids.java | 156 +++++++ .../query/GroupByParallelQueryRunner.java | 63 +-- .../main/java/io/druid/query/QueryConfig.java | 8 + .../java/io/druid/query/QueryToolChest.java | 18 +- .../io/druid/query/groupby/GroupByQuery.java | 5 +- .../query/groupby/GroupByQueryConfig.java | 9 + .../query/groupby/GroupByQueryEngine.java | 64 +-- .../query/groupby/GroupByQueryHelper.java | 32 +- .../groupby/GroupByQueryQueryToolChest.java | 180 ++++++-- .../groupby/GroupByQueryRunnerFactory.java | 70 ++-- .../groupby/having/AlwaysHavingSpec.java | 9 + .../query/groupby/having/AndHavingSpec.java | 22 + .../groupby/having/EqualToHavingSpec.java | 18 + .../groupby/having/GreaterThanHavingSpec.java | 19 + .../query/groupby/having/HavingSpec.java | 24 +- .../groupby/having/LessThanHavingSpec.java | 28 +- .../query/groupby/having/NeverHavingSpec.java | 42 ++ .../query/groupby/having/NotHavingSpec.java | 14 + .../query/groupby/having/OrHavingSpec.java | 45 +- .../groupby/orderby/DefaultLimitSpec.java | 31 +- .../query/groupby/orderby/LimitSpec.java | 3 + .../query/groupby/orderby/NoopLimitSpec.java | 9 + .../groupby/orderby/OrderByColumnSpec.java | 27 +- .../SegmentMetadataQueryQueryToolChest.java | 16 + .../SegmentMetadataQueryRunnerFactory.java | 5 +- .../search/SearchQueryQueryToolChest.java | 8 + .../druid/query/select/SelectQueryConfig.java | 37 ++ .../select/SelectQueryQueryToolChest.java | 11 +- .../druid/query/select/SelectResultValue.java | 8 +- .../TimeBoundaryQueryQueryToolChest.java | 6 + .../TimeseriesQueryQueryToolChest.java | 9 +- .../TimeseriesQueryRunnerFactory.java | 2 - .../query/topn/TopNQueryQueryToolChest.java | 7 + .../segment/incremental/IncrementalIndex.java | 2 +- .../SpatialDimensionRowFormatter.java | 14 + .../query/groupby/GroupByQueryRunnerTest.java | 174 ++++++-- .../GroupByTimeseriesQueryRunnerTest.java | 17 +- .../query/groupby/having/HavingSpecTest.java | 7 + .../query/metadata/SegmentAnalyzerTest.java | 10 +- .../metadata/SegmentMetadataQueryTest.java | 5 +- .../query/select/SelectQueryRunnerTest.java | 2 +- .../java/io/druid/segment/TestHelper.java | 45 +- .../druid/client/CachingClusteredClient.java | 16 +- .../io/druid/client/CachingQueryRunner.java | 14 +- .../io/druid/client/cache/CacheConfig.java | 15 + .../io/druid/guice/QueryToolChestModule.java | 2 + .../firehose/WikipediaIrcDecoder.java | 13 + .../client/CachingClusteredClientTest.java | 387 +++++++++++++++++- .../segment/realtime/RealtimeManagerTest.java | 13 + .../segment/realtime/plumber/SinkTest.java | 25 ++ .../shard/HashBasedNumberedShardSpecTest.java | 14 + 53 files changed, 1521 insertions(+), 263 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java create mode 100644 processing/src/main/java/io/druid/query/select/SelectQueryConfig.java diff --git a/pom.xml b/pom.xml index 2b7f88276e8..dfa203ec905 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.5 2.5.0 - 0.2.4 + 0.2.5-SNAPSHOT diff --git a/processing/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java index f77affcb5bf..737ed30ad5b 100644 --- a/processing/src/main/java/io/druid/query/CacheStrategy.java +++ b/processing/src/main/java/io/druid/query/CacheStrategy.java @@ -35,5 +35,7 @@ public interface CacheStrategy> public Function pullFromCache(); + public int getCacheLimit(); + public Sequence mergeSequences(Sequence> seqOfSequences); } diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index 932af432ad1..ff03e8db7f7 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -35,6 +35,8 @@ import io.druid.query.search.SearchResultValue; import io.druid.query.search.search.InsensitiveContainsSearchQuerySpec; import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQuerySpec; +import io.druid.query.select.PagingSpec; +import io.druid.query.select.SelectQuery; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.timeboundary.TimeBoundaryQuery; @@ -943,4 +945,158 @@ public class Druids { return new SegmentMetadataQueryBuilder(); } + + /** + * A Builder for SelectQuery. + *

+ * Required: dataSource(), intervals() must be called before build() + *

+ * Usage example: + *


+   *   SelectQuery query = new SelectQueryBuilder()
+   *                                  .dataSource("Example")
+   *                                  .interval("2010/2013")
+   *                                  .build();
+   * 
+ * + * @see io.druid.query.select.SelectQuery + */ + public static class SelectQueryBuilder + { + private DataSource dataSource; + private QuerySegmentSpec querySegmentSpec; + private Map context; + private DimFilter dimFilter; + private QueryGranularity granularity; + private List dimensions; + private List metrics; + private PagingSpec pagingSpec; + + public SelectQueryBuilder() + { + dataSource = null; + querySegmentSpec = null; + context = null; + dimFilter = null; + granularity = QueryGranularity.ALL; + dimensions = Lists.newArrayList(); + metrics = Lists.newArrayList(); + pagingSpec = null; + } + + public SelectQuery build() + { + return new SelectQuery( + dataSource, + querySegmentSpec, + dimFilter, + granularity, + dimensions, + metrics, + pagingSpec, + context + ); + } + + public SelectQueryBuilder copy(SelectQueryBuilder builder) + { + return new SelectQueryBuilder() + .dataSource(builder.dataSource) + .intervals(builder.querySegmentSpec) + .context(builder.context); + } + + public SelectQueryBuilder dataSource(String ds) + { + dataSource = new TableDataSource(ds); + return this; + } + + public SelectQueryBuilder dataSource(DataSource ds) + { + dataSource = ds; + return this; + } + + public SelectQueryBuilder intervals(QuerySegmentSpec q) + { + querySegmentSpec = q; + return this; + } + + public SelectQueryBuilder intervals(String s) + { + querySegmentSpec = new LegacySegmentSpec(s); + return this; + } + + public SelectQueryBuilder intervals(List l) + { + querySegmentSpec = new LegacySegmentSpec(l); + return this; + } + + public SelectQueryBuilder context(Map c) + { + context = c; + return this; + } + + public SelectQueryBuilder filters(String dimensionName, String value) + { + dimFilter = new SelectorDimFilter(dimensionName, value); + return this; + } + + public SelectQueryBuilder filters(String dimensionName, String value, String... values) + { + List fields = Lists.newArrayList(new SelectorDimFilter(dimensionName, value)); + for (String val : values) { + fields.add(new SelectorDimFilter(dimensionName, val)); + } + dimFilter = new OrDimFilter(fields); + return this; + } + + public SelectQueryBuilder filters(DimFilter f) + { + dimFilter = f; + return this; + } + + public SelectQueryBuilder granularity(String g) + { + granularity = QueryGranularity.fromString(g); + return this; + } + + public SelectQueryBuilder granularity(QueryGranularity g) + { + granularity = g; + return this; + } + + public SelectQueryBuilder dimensions(List d) + { + dimensions = d; + return this; + } + + public SelectQueryBuilder metrics(List m) + { + metrics = m; + return this; + } + + public SelectQueryBuilder pagingSpec(PagingSpec p) + { + pagingSpec = p; + return this; + } + } + + public static SelectQueryBuilder newSelectQueryBuilder() + { + return new SelectQueryBuilder(); + } } diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index cb3402eb2ce..a460cd68485 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -25,7 +25,6 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; @@ -41,7 +40,6 @@ import io.druid.query.groupby.GroupByQueryConfig; import io.druid.query.groupby.GroupByQueryHelper; import io.druid.segment.incremental.IncrementalIndex; -import java.util.Arrays; import java.util.List; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -50,49 +48,37 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -public class GroupByParallelQueryRunner implements QueryRunner +public class GroupByParallelQueryRunner implements QueryRunner { private static final Logger log = new Logger(GroupByParallelQueryRunner.class); - private final Iterable> queryables; + private final Iterable> queryables; private final ListeningExecutorService exec; - private final Ordering ordering; private final Supplier configSupplier; private final QueryWatcher queryWatcher; - public GroupByParallelQueryRunner( ExecutorService exec, - Ordering ordering, Supplier configSupplier, QueryWatcher queryWatcher, - QueryRunner... queryables - ) - { - this(exec, ordering, configSupplier, queryWatcher, Arrays.asList(queryables)); - } - - public GroupByParallelQueryRunner( - ExecutorService exec, - Ordering ordering, Supplier configSupplier, - QueryWatcher queryWatcher, - Iterable> queryables + Iterable> queryables ) { this.exec = MoreExecutors.listeningDecorator(exec); - this.ordering = ordering; this.queryWatcher = queryWatcher; this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull())); this.configSupplier = configSupplier; } @Override - public Sequence run(final Query queryParam) + public Sequence run(final Query queryParam) { final GroupByQuery query = (GroupByQuery) queryParam; - final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( + final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, configSupplier.get() ); + final Pair> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair(); + final boolean bySegment = query.getContextBySegment(false); final int priority = query.getContextPriority(0); if (Iterables.isEmpty(queryables)) { @@ -102,10 +88,10 @@ public class GroupByParallelQueryRunner implements QueryRunner Lists.newArrayList( Iterables.transform( queryables, - new Function, ListenableFuture>() + new Function, ListenableFuture>() { @Override - public ListenableFuture apply(final QueryRunner input) + public ListenableFuture apply(final QueryRunner input) { return exec.submit( new AbstractPrioritizedCallable(priority) @@ -114,6 +100,12 @@ public class GroupByParallelQueryRunner implements QueryRunner public Boolean call() throws Exception { try { + if (bySegment) { + input.run(queryParam) + .accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs); + return true; + } + input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); return true; } @@ -137,7 +129,7 @@ public class GroupByParallelQueryRunner implements QueryRunner try { queryWatcher.registerQuery(query, futures); final Number timeout = query.getContextValue("timeout", (Number) null); - if(timeout == null) { + if (timeout == null) { futures.get(); } else { futures.get(timeout.longValue(), TimeUnit.MILLISECONDS); @@ -148,10 +140,10 @@ public class GroupByParallelQueryRunner implements QueryRunner futures.cancel(true); throw new QueryInterruptedException("Query interrupted"); } - catch(CancellationException e) { + catch (CancellationException e) { throw new QueryInterruptedException("Query cancelled"); } - catch(TimeoutException e) { + catch (TimeoutException e) { log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); futures.cancel(true); throw new QueryInterruptedException("Query timeout"); @@ -160,7 +152,22 @@ public class GroupByParallelQueryRunner implements QueryRunner throw Throwables.propagate(e.getCause()); } - return Sequences.simple(indexAccumulatorPair.lhs.iterableWithPostAggregations(null)); - } + if (bySegment) { + return Sequences.simple(bySegmentAccumulatorPair.lhs); + } + return Sequences.simple( + Iterables.transform( + indexAccumulatorPair.lhs.iterableWithPostAggregations(null), + new Function() + { + @Override + public T apply(Row input) + { + return (T) input; + } + } + ) + ); + } } diff --git a/processing/src/main/java/io/druid/query/QueryConfig.java b/processing/src/main/java/io/druid/query/QueryConfig.java index f5da67a39a1..53d757f6c75 100644 --- a/processing/src/main/java/io/druid/query/QueryConfig.java +++ b/processing/src/main/java/io/druid/query/QueryConfig.java @@ -29,8 +29,16 @@ public class QueryConfig @JsonProperty private Period chunkPeriod = new Period(); + @JsonProperty + private int maxResultsToCache = Integer.MAX_VALUE; + public Period getChunkPeriod() { return chunkPeriod; } + + public int getMaxResultsToCache() + { + return maxResultsToCache; + } } diff --git a/processing/src/main/java/io/druid/query/QueryToolChest.java b/processing/src/main/java/io/druid/query/QueryToolChest.java index 8299ecaad0a..d2722c622be 100644 --- a/processing/src/main/java/io/druid/query/QueryToolChest.java +++ b/processing/src/main/java/io/druid/query/QueryToolChest.java @@ -41,13 +41,17 @@ public abstract class QueryToolChest mergeSequences(Sequence> seqOfSequences); public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public abstract Function makePreComputeManipulatorFn(QueryType query, MetricManipulationFn fn); + public abstract Function makePreComputeManipulatorFn( + QueryType query, + MetricManipulationFn fn + ); public Function makePostComputeManipulatorFn(QueryType query, MetricManipulationFn fn) { @@ -56,19 +60,23 @@ public abstract class QueryToolChest getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query) { + public CacheStrategy getCacheStrategy(QueryType query) + { return null; } - public QueryRunner preMergeQueryDecoration(QueryRunner runner) { + public QueryRunner preMergeQueryDecoration(QueryRunner runner) + { return runner; } - public QueryRunner postMergeQueryDecoration(QueryRunner runner) { + public QueryRunner postMergeQueryDecoration(QueryRunner runner) + { return runner; } - public List filterSegments(QueryType query, List segments) { + public List filterSegments(QueryType query, List segments) + { return segments; } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 98ac83e32cd..a0fab78da07 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -51,7 +51,6 @@ import io.druid.query.groupby.orderby.OrderByColumnSpec; import io.druid.query.spec.LegacySegmentSpec; import io.druid.query.spec.QuerySegmentSpec; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -111,14 +110,14 @@ public class GroupByQuery extends BaseQuery new Function, Sequence>() { @Override - public Sequence apply(@Nullable Sequence input) + public Sequence apply(Sequence input) { return Sequences.filter( input, new Predicate() { @Override - public boolean apply(@Nullable Row input) + public boolean apply(Row input) { return GroupByQuery.this.havingSpec.eval(input); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java index 0139e16742e..5e723a2ff23 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -35,6 +35,9 @@ public class GroupByQueryConfig extends QueryConfig @JsonProperty private int maxResults = 500000; + @JsonProperty + private int maxResultsToCache = 10000; + public boolean isSingleThreaded() { return singleThreaded; @@ -54,4 +57,10 @@ public class GroupByQueryConfig extends QueryConfig { return maxResults; } + + @Override + public int getMaxResultsToCache() + { + return maxResultsToCache; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index 8599c6bdd16..a3cc06c0b9a 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -81,7 +81,7 @@ public class GroupByQueryEngine this.intermediateResultsBufferPool = intermediateResultsBufferPool; } - public Sequence process(final GroupByQuery query, StorageAdapter storageAdapter) + public Sequence process(final GroupByQuery query, final StorageAdapter storageAdapter) { if (storageAdapter == null) { throw new ISE( @@ -104,41 +104,41 @@ public class GroupByQueryEngine return Sequences.concat( Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(@Nullable final Cursor cursor) - { - return new BaseSequence( - new BaseSequence.IteratorMaker() + Sequences.map( + cursors, + new Function>() + { + @Override + public Sequence apply(final Cursor cursor) { - @Override - public RowIterator make() - { - return new RowIterator(query, cursor, bufferHolder.get(), config.get()); - } + return new BaseSequence<>( + new BaseSequence.IteratorMaker() + { + @Override + public RowIterator make() + { + return new RowIterator(query, cursor, bufferHolder.get(), config.get()); + } - @Override - public void cleanup(RowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } + @Override + public void cleanup(RowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + } + ); } - ); + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + CloseQuietly.close(bufferHolder); + } } - } - ), - new Closeable() - { - @Override - public void close() throws IOException - { - CloseQuietly.close(bufferHolder); - } - } - ) + ) ); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index b7183b12354..093b2ee3c7a 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -24,21 +24,18 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; -import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.data.input.Rows; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; import io.druid.segment.incremental.IncrementalIndex; -import io.druid.segment.incremental.IncrementalIndexSchema; -import javax.annotation.Nullable; import java.util.List; public class GroupByQueryHelper { - public static Pair> createIndexAccumulatorPair( + public static Pair> createIndexAccumulatorPair( final GroupByQuery query, final GroupByQueryConfig config ) @@ -80,13 +77,18 @@ public class GroupByQueryHelper aggs.toArray(new AggregatorFactory[aggs.size()]) ); - Accumulator accumulator = new Accumulator() + Accumulator accumulator = new Accumulator() { @Override - public IncrementalIndex accumulate(IncrementalIndex accumulated, Row in) + public IncrementalIndex accumulate(IncrementalIndex accumulated, T in) { - if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions), false) > config.getMaxResults()) { - throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults()); + if (in instanceof Row) { + if (accumulated.add(Rows.toCaseInsensitiveInputRow((Row) in, dimensions), false) + > config.getMaxResults()) { + throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults()); + } + } else { + throw new ISE("Unable to accumulate something of type [%s]", in.getClass()); } return accumulated; @@ -95,4 +97,18 @@ public class GroupByQueryHelper return new Pair<>(index, accumulator); } + public static Pair> createBySegmentAccumulatorPair() + { + List init = Lists.newArrayList(); + Accumulator accumulator = new Accumulator() + { + @Override + public List accumulate(List accumulated, T in) + { + accumulated.add(in); + return accumulated; + } + }; + return new Pair<>(init, accumulator); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 025af21a2d7..d0ba9b603b1 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -20,25 +20,32 @@ package io.druid.query.groupby; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; import com.google.inject.Inject; +import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; -import com.metamx.common.guava.ConcatSequence; +import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.service.ServiceMetricEvent; +import io.druid.collections.OrderedMergeSequence; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; +import io.druid.granularity.QueryGranularity; +import io.druid.query.CacheStrategy; import io.druid.query.DataSource; import io.druid.query.DataSourceUtil; import io.druid.query.IntervalChunkingQueryRunner; import io.druid.query.Query; +import io.druid.query.QueryCacheHelper; import io.druid.query.QueryDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; @@ -46,11 +53,16 @@ import io.druid.query.SubqueryQueryRunner; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.dimension.DimensionSpec; +import io.druid.query.filter.DimFilter; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Minutes; +import java.nio.ByteBuffer; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -58,6 +70,11 @@ import java.util.Map; */ public class GroupByQueryQueryToolChest extends QueryToolChest { + private static final byte GROUPBY_QUERY = 0x14; + private static final TypeReference OBJECT_TYPE_REFERENCE = + new TypeReference() + { + }; private static final TypeReference TYPE_REFERENCE = new TypeReference() { }; @@ -66,16 +83,20 @@ public class GroupByQueryQueryToolChest extends QueryToolChest configSupplier; + private final ObjectMapper jsonMapper; private GroupByQueryEngine engine; // For running the outer query around a subquery @Inject public GroupByQueryQueryToolChest( Supplier configSupplier, + ObjectMapper jsonMapper, GroupByQueryEngine engine ) { this.configSupplier = configSupplier; + this.jsonMapper = jsonMapper; this.engine = engine; } @@ -87,11 +108,14 @@ public class GroupByQueryQueryToolChest extends QueryToolChest run(Query input) { - if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { - return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); - } else { + if (input.getContextBySegment(false)) { return runner.run(input); } + + if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { + return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); + } + return runner.run(input); } }; } @@ -136,22 +160,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest postAggregate(final GroupByQuery query, IncrementalIndex index) { - return Sequences.map( - Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())), - new Function() - { - @Override - public Row apply(Row input) - { - final MapBasedRow row = (MapBasedRow) input; - return new MapBasedRow( - query.getGranularity() - .toDateTime(row.getTimestampFromEpoch()), - row.getEvent() - ); - } - } - ); + return Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())); } private IncrementalIndex makeIncrementalIndex(GroupByQuery query, Sequence rows) @@ -169,7 +178,8 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { - return new ConcatSequence<>(seqOfSequences); + return new OrderedMergeSequence<>(Ordering.natural().nullsFirst(), seqOfSequences); + //return new ConcatSequence<>(seqOfSequences); } @Override @@ -191,7 +201,10 @@ public class GroupByQueryQueryToolChest extends QueryToolChest makePreComputeManipulatorFn(final GroupByQuery query, final MetricManipulationFn fn) + public Function makePreComputeManipulatorFn( + final GroupByQuery query, + final MetricManipulationFn fn + ) { return new Function() { @@ -200,7 +213,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest values = Maps.newHashMap(((MapBasedRow) input).getEvent()); + final Map values = Maps.newHashMap(inputRow.getEvent()); for (AggregatorFactory agg : query.getAggregatorSpecs()) { values.put(agg.getName(), fn.manipulate(agg, inputRow.getEvent().get(agg.getName()))); } @@ -220,8 +233,125 @@ public class GroupByQueryQueryToolChest extends QueryToolChest preMergeQueryDecoration(QueryRunner runner) { - return new SubqueryQueryRunner( - new IntervalChunkingQueryRunner(runner, configSupplier.get().getChunkPeriod()) + return new SubqueryQueryRunner<>( + new IntervalChunkingQueryRunner<>(runner, configSupplier.get().getChunkPeriod()) ); } + + @Override + public CacheStrategy getCacheStrategy(final GroupByQuery query) + { + return new CacheStrategy() + { + @Override + public byte[] computeCacheKey(GroupByQuery query) + { + final DimFilter dimFilter = query.getDimFilter(); + final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); + final byte[] aggregatorBytes = QueryCacheHelper.computeAggregatorBytes(query.getAggregatorSpecs()); + final byte[] granularityBytes = query.getGranularity().cacheKey(); + final byte[][] dimensionsBytes = new byte[query.getDimensions().size()][]; + int dimensionsBytesSize = 0; + int index = 0; + for (DimensionSpec dimension : query.getDimensions()) { + dimensionsBytes[index] = dimension.getCacheKey(); + dimensionsBytesSize += dimensionsBytes[index].length; + ++index; + } + final byte[] havingBytes = query.getHavingSpec() == null ? new byte[]{} : query.getHavingSpec().getCacheKey(); + final byte[] limitBytes = query.getLimitSpec().getCacheKey(); + + ByteBuffer buffer = ByteBuffer + .allocate( + 1 + + granularityBytes.length + + filterBytes.length + + aggregatorBytes.length + + dimensionsBytesSize + + havingBytes.length + + limitBytes.length + ) + .put(GROUPBY_QUERY) + .put(granularityBytes) + .put(filterBytes) + .put(aggregatorBytes); + + for (byte[] dimensionsByte : dimensionsBytes) { + buffer.put(dimensionsByte); + } + + return buffer + .put(havingBytes) + .put(limitBytes) + .array(); + } + + @Override + public TypeReference getCacheObjectClazz() + { + return OBJECT_TYPE_REFERENCE; + } + + @Override + public Function prepareForCache() + { + return new Function() + { + @Override + public Object apply(Row input) + { + if (input instanceof MapBasedRow) { + final MapBasedRow row = (MapBasedRow) input; + final List retVal = Lists.newArrayListWithCapacity(2); + retVal.add(row.getTimestamp().getMillis()); + retVal.add(row.getEvent()); + + return retVal; + } + + throw new ISE("Don't know how to cache input rows of type[%s]", input.getClass()); + } + }; + } + + @Override + public Function pullFromCache() + { + return new Function() + { + private final QueryGranularity granularity = query.getGranularity(); + + @Override + public Row apply(Object input) + { + Iterator results = ((List) input).iterator(); + + DateTime timestamp = granularity.toDateTime(((Number) results.next()).longValue()); + + return new MapBasedRow( + timestamp, + (Map) jsonMapper.convertValue( + results.next(), + new TypeReference>() + { + } + ) + ); + } + }; + } + + @Override + public int getCacheLimit() + { + return configSupplier.get().getMaxResultsToCache(); + } + + @Override + public Sequence mergeSequences(Sequence> seqOfSequences) + { + return new MergeSequence<>(Ordering.natural().nullsFirst(), seqOfSequences); + } + }; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java index e8634089c2f..24e85b9f1c2 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -22,18 +22,20 @@ package io.druid.query.groupby; import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; -import com.google.common.collect.Ordering; -import com.google.common.primitives.Longs; +import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.metamx.common.ISE; +import com.metamx.common.Pair; +import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.ExecutorExecutingSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.logger.Logger; import io.druid.data.input.Row; +import io.druid.query.AbstractPrioritizedCallable; import io.druid.query.ConcatQueryRunner; import io.druid.query.GroupByParallelQueryRunner; import io.druid.query.Query; @@ -44,7 +46,9 @@ import io.druid.query.QueryToolChest; import io.druid.query.QueryWatcher; import io.druid.segment.Segment; import io.druid.segment.StorageAdapter; +import io.druid.segment.incremental.IncrementalIndex; +import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -88,8 +92,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory( + return new ConcatQueryRunner<>( Sequences.map( Sequences.simple(queryRunners), new Function, QueryRunner>() @@ -102,34 +107,54 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory run(final Query query) { + final GroupByQuery queryParam = (GroupByQuery) query; + final Pair> indexAccumulatorPair = GroupByQueryHelper + .createIndexAccumulatorPair( + queryParam, + config.get() + ); + final Pair> bySegmentAccumulatorPair = GroupByQueryHelper.createBySegmentAccumulatorPair(); + final int priority = query.getContextPriority(0); + final boolean bySegment = query.getContextBySegment(false); - ListenableFuture> future = queryExecutor.submit( - new Callable>() + final ListenableFuture future = queryExecutor.submit( + new AbstractPrioritizedCallable(priority) { @Override - public Sequence call() throws Exception + public Boolean call() throws Exception { - return new ExecutorExecutingSequence( - input.run(query), - queryExecutor - ); + if (bySegment) { + input.run(queryParam) + .accumulate( + bySegmentAccumulatorPair.lhs, + bySegmentAccumulatorPair.rhs + ); + return true; + } + + input.run(query).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); + return true; } } ); try { queryWatcher.registerQuery(query, future); - final Number timeout = query.getContextValue("timeout", (Number)null); - return timeout == null ? future.get() : future.get(timeout.longValue(), TimeUnit.MILLISECONDS); + final Number timeout = query.getContextValue("timeout", (Number) null); + if (timeout == null) { + future.get(); + } else { + future.get(timeout.longValue(), TimeUnit.MILLISECONDS); + } } catch (InterruptedException e) { log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); future.cancel(true); throw new QueryInterruptedException("Query interrupted"); } - catch(CancellationException e) { + catch (CancellationException e) { throw new QueryInterruptedException("Query cancelled"); } - catch(TimeoutException e) { + catch (TimeoutException e) { log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); future.cancel(true); throw new QueryInterruptedException("Query timeout"); @@ -137,6 +162,12 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory - { - @Override - public int compare(Row left, Row right) - { - return Longs.compare(left.getTimestampFromEpoch(), right.getTimestampFromEpoch()); - } - } } diff --git a/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java index 51b938a9891..3868378d7a2 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/AlwaysHavingSpec.java @@ -22,12 +22,21 @@ package io.druid.query.groupby.having; import io.druid.data.input.Row; /** + * A "having" spec that always evaluates to true */ public class AlwaysHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x0; + @Override public boolean eval(Row row) { return true; } + + @Override + public byte[] getCacheKey() + { + return new byte[]{CACHE_KEY}; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java index 8e0fdb6a91f..4b0a0ce30fd 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/AndHavingSpec.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import io.druid.data.input.Row; +import java.nio.ByteBuffer; import java.util.List; /** @@ -31,6 +32,8 @@ import java.util.List; */ public class AndHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x2; + private List havingSpecs; @JsonCreator @@ -57,6 +60,25 @@ public class AndHavingSpec implements HavingSpec return true; } + @Override + public byte[] getCacheKey() + { + final byte[][] havingBytes = new byte[havingSpecs.size()][]; + int havingBytesSize = 0; + int index = 0; + for (HavingSpec havingSpec : havingSpecs) { + havingBytes[index] = havingSpec.getCacheKey(); + havingBytesSize += havingBytes[index].length; + ++index; + } + + ByteBuffer buffer = ByteBuffer.allocate(1 + havingBytesSize).put(CACHE_KEY); + for (byte[] havingByte : havingBytes) { + buffer.put(havingByte); + } + return buffer.array(); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java index 34bd7b887f2..bc26524e60b 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/EqualToHavingSpec.java @@ -21,14 +21,20 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Bytes; import io.druid.data.input.Row; +import java.nio.ByteBuffer; +import java.util.Arrays; + /** * The "=" operator in a "having" clause. This is similar to SQL's "having aggregation = value", * except that in SQL an aggregation is an expression instead of an aggregation name as in Druid. */ public class EqualToHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x3; + private String aggregationName; private Number value; @@ -62,6 +68,18 @@ public class EqualToHavingSpec implements HavingSpec return Float.compare(value.floatValue(), metricValue) == 0; } + @Override + public byte[] getCacheKey() + { + final byte[] aggBytes = aggregationName.getBytes(); + final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); + return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) + .put(CACHE_KEY) + .put(aggBytes) + .put(valBytes) + .array(); + } + /** * This method treats internal value as double mainly for ease of test. */ diff --git a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java index 0c2c9e7810b..6d4c2c78071 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/GreaterThanHavingSpec.java @@ -21,7 +21,12 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Bytes; import io.druid.data.input.Row; +import io.druid.query.Result; + +import java.nio.ByteBuffer; +import java.util.Arrays; /** * The ">" operator in a "having" clause. This is similar to SQL's "having aggregation > value", @@ -29,6 +34,8 @@ import io.druid.data.input.Row; */ public class GreaterThanHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x4; + private String aggregationName; private Number value; @@ -62,6 +69,18 @@ public class GreaterThanHavingSpec implements HavingSpec return Float.compare(metricValue, value.floatValue()) > 0; } + @Override + public byte[] getCacheKey() + { + final byte[] aggBytes = aggregationName.getBytes(); + final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); + return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) + .put(CACHE_KEY) + .put(aggBytes) + .put(valBytes) + .array(); + } + /** * This method treats internal value as double mainly for ease of test. */ diff --git a/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java index 00a8389cff0..37ad9e1b8df 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/HavingSpec.java @@ -38,6 +38,11 @@ import io.druid.data.input.Row; }) public interface HavingSpec { + // Atoms for easy combination, but for now they are mostly useful + // for testing. + public static final HavingSpec NEVER = new NeverHavingSpec(); + public static final HavingSpec ALWAYS = new AlwaysHavingSpec(); + /** * Evaluates if a given row satisfies the having spec. * @@ -49,22 +54,5 @@ public interface HavingSpec */ public boolean eval(Row row); - // Atoms for easy combination, but for now they are mostly useful - // for testing. - /** - * A "having" spec that always evaluates to false - */ - public static final HavingSpec NEVER = new HavingSpec() - { - @Override - public boolean eval(Row row) - { - return false; - } - }; - - /** - * A "having" spec that always evaluates to true - */ - public static final HavingSpec ALWAYS = new AlwaysHavingSpec(); + public byte[] getCacheKey(); } diff --git a/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java index ce8dd20b661..1f1a27bd9a7 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/LessThanHavingSpec.java @@ -20,7 +20,12 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.primitives.Bytes; import io.druid.data.input.Row; +import io.druid.query.Result; + +import java.nio.ByteBuffer; +import java.util.Arrays; /** * The "<" operator in a "having" clause. This is similar to SQL's "having aggregation < value", @@ -28,14 +33,15 @@ import io.druid.data.input.Row; */ public class LessThanHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x5; + private String aggregationName; private Number value; - public LessThanHavingSpec - ( - @JsonProperty("aggregation") String aggName, - @JsonProperty("value") Number value - ) + public LessThanHavingSpec( + @JsonProperty("aggregation") String aggName, + @JsonProperty("value") Number value + ) { this.aggregationName = aggName; this.value = value; @@ -61,6 +67,18 @@ public class LessThanHavingSpec implements HavingSpec return Float.compare(metricValue, value.floatValue()) < 0; } + @Override + public byte[] getCacheKey() + { + final byte[] aggBytes = aggregationName.getBytes(); + final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); + return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) + .put(CACHE_KEY) + .put(aggBytes) + .put(valBytes) + .array(); + } + /** * This method treats internal value as double mainly for ease of test. */ diff --git a/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java new file mode 100644 index 00000000000..bad1cd03f37 --- /dev/null +++ b/processing/src/main/java/io/druid/query/groupby/having/NeverHavingSpec.java @@ -0,0 +1,42 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.groupby.having; + +import io.druid.data.input.Row; + +/** + * A "having" spec that always evaluates to false + */ +public class NeverHavingSpec implements HavingSpec +{ + private static final byte CACHE_KEY = 0x1; + + @Override + public boolean eval(Row row) + { + return false; + } + + @Override + public byte[] getCacheKey() + { + return new byte[]{CACHE_KEY}; + } +} diff --git a/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java index e963e0154e5..590e47d3d0b 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/NotHavingSpec.java @@ -22,12 +22,17 @@ package io.druid.query.groupby.having; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.data.input.Row; +import io.druid.query.Result; + +import java.nio.ByteBuffer; /** * The logical "not" operator for the "having" clause. */ public class NotHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x6; + private HavingSpec havingSpec; @JsonCreator @@ -48,6 +53,15 @@ public class NotHavingSpec implements HavingSpec return !havingSpec.eval(row); } + @Override + public byte[] getCacheKey() + { + return ByteBuffer.allocate(1 + havingSpec.getCacheKey().length) + .put(CACHE_KEY) + .put(havingSpec.getCacheKey()) + .array(); + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java b/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java index 73f46dfaa24..c413a9cf2f0 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/having/OrHavingSpec.java @@ -23,7 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import io.druid.data.input.Row; +import io.druid.query.Result; +import java.nio.ByteBuffer; import java.util.List; /** @@ -31,23 +33,27 @@ import java.util.List; */ public class OrHavingSpec implements HavingSpec { + private static final byte CACHE_KEY = 0x7; + private List havingSpecs; @JsonCreator - public OrHavingSpec(@JsonProperty("havingSpecs") List havingSpecs) { + public OrHavingSpec(@JsonProperty("havingSpecs") List havingSpecs) + { this.havingSpecs = havingSpecs == null ? ImmutableList.of() : havingSpecs; } @JsonProperty("havingSpecs") - public List getHavingSpecs(){ + public List getHavingSpecs() + { return havingSpecs; } @Override public boolean eval(Row row) { - for(HavingSpec havingSpec: havingSpecs) { - if(havingSpec.eval(row)){ + for (HavingSpec havingSpec : havingSpecs) { + if (havingSpec.eval(row)) { return true; } } @@ -55,15 +61,40 @@ public class OrHavingSpec implements HavingSpec return false; } + @Override + public byte[] getCacheKey() + { + final byte[][] havingBytes = new byte[havingSpecs.size()][]; + int havingBytesSize = 0; + int index = 0; + for (HavingSpec havingSpec : havingSpecs) { + havingBytes[index] = havingSpec.getCacheKey(); + havingBytesSize += havingBytes[index].length; + ++index; + } + + ByteBuffer buffer = ByteBuffer.allocate(1 + havingBytesSize).put(CACHE_KEY); + for (byte[] havingByte : havingBytes) { + buffer.put(havingByte); + } + return buffer.array(); + } + @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } OrHavingSpec that = (OrHavingSpec) o; - if (havingSpecs != null ? !havingSpecs.equals(that.havingSpecs) : that.havingSpecs != null) return false; + if (havingSpecs != null ? !havingSpecs.equals(that.havingSpecs) : that.havingSpecs != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index 3d78e112cb5..bf6fc138ad4 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; +import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; @@ -37,6 +38,7 @@ import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import javax.annotation.Nullable; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; import java.util.List; @@ -46,6 +48,8 @@ import java.util.Map; */ public class DefaultLimitSpec implements LimitSpec { + private static final byte CACHE_KEY = 0x1; + private final List columns; private final int limit; @@ -196,7 +200,7 @@ public class DefaultLimitSpec implements LimitSpec @Override public Sequence apply( - @Nullable Sequence input + Sequence input ) { return Sequences.limit(input, limit); @@ -275,12 +279,12 @@ public class DefaultLimitSpec implements LimitSpec { this.limit = limit; - this.sorter = new TopNSorter(ordering); + this.sorter = new TopNSorter<>(ordering); } @Override public Sequence apply( - @Nullable Sequence input + Sequence input ) { final ArrayList materializedList = Sequences.toList(input, Lists.newArrayList()); @@ -347,4 +351,25 @@ public class DefaultLimitSpec implements LimitSpec result = 31 * result + limit; return result; } + + @Override + public byte[] getCacheKey() + { + final byte[][] columnBytes = new byte[columns.size()][]; + int columnsBytesSize = 0; + int index = 0; + for (OrderByColumnSpec column : columns) { + columnBytes[index] = column.getCacheKey(); + columnsBytesSize += columnBytes[index].length; + ++index; + } + + ByteBuffer buffer = ByteBuffer.allocate(1 + columnsBytesSize + 4) + .put(CACHE_KEY); + for (byte[] columnByte : columnBytes) { + buffer.put(columnByte); + } + buffer.put(Ints.toByteArray(limit)); + return buffer.array(); + } } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java index fa50d62016c..3e8153355e0 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Function; import com.metamx.common.guava.Sequence; import io.druid.data.input.Row; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -45,4 +46,6 @@ public interface LimitSpec ); public LimitSpec merge(LimitSpec other); + + public byte[] getCacheKey(); } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java index e71038d4918..bc05f1cb352 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.metamx.common.guava.Sequence; import io.druid.data.input.Row; +import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; @@ -33,6 +34,8 @@ import java.util.List; */ public class NoopLimitSpec implements LimitSpec { + private static final byte CACHE_KEY = 0x0; + @Override public Function, Sequence> build( List dimensions, List aggs, List postAggs @@ -63,4 +66,10 @@ public class NoopLimitSpec implements LimitSpec public int hashCode() { return 0; } + + @Override + public byte[] getCacheKey() + { + return new byte[]{CACHE_KEY}; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java index 147f1911816..bf0432ad5c9 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/OrderByColumnSpec.java @@ -29,6 +29,7 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import javax.annotation.Nullable; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -37,14 +38,19 @@ import java.util.Map; */ public class OrderByColumnSpec { + public static enum Direction + { + ASCENDING, + DESCENDING + } + /** * Maintain a map of the enum values so that we can just do a lookup and get a null if it doesn't exist instead * of an exception thrown. */ private static final Map stupidEnumMap; - static - { + static { final ImmutableMap.Builder bob = ImmutableMap.builder(); for (Direction direction : Direction.values()) { bob.put(direction.toString(), direction); @@ -62,16 +68,14 @@ public class OrderByColumnSpec if (obj instanceof String) { return new OrderByColumnSpec(obj.toString(), null); - } - else if (obj instanceof Map) { + } else if (obj instanceof Map) { final Map map = (Map) obj; final String dimension = map.get("dimension").toString(); final Direction direction = determineDirection(map.get("direction")); return new OrderByColumnSpec(dimension, direction); - } - else { + } else { throw new ISE("Cannot build an OrderByColumnSpec from a %s", obj.getClass()); } } @@ -176,9 +180,14 @@ public class OrderByColumnSpec '}'; } - public static enum Direction + public byte[] getCacheKey() { - ASCENDING, - DESCENDING + final byte[] dimensionBytes = dimension.getBytes(); + final byte[] directionBytes = direction.name().getBytes(); + + return ByteBuffer.allocate(dimensionBytes.length + dimensionBytes.length) + .put(dimensionBytes) + .put(directionBytes) + .array(); } } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index a1f649fed06..a03b91d9c6b 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -27,6 +27,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; +import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; @@ -37,6 +38,7 @@ import io.druid.common.utils.JodaUtils; import io.druid.query.CacheStrategy; import io.druid.query.DataSourceUtil; import io.druid.query.Query; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.ResultMergeQueryRunner; @@ -60,6 +62,14 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest mergeResults(final QueryRunner runner) { @@ -216,6 +226,12 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index c8e7208638c..00909ea6cbd 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -55,16 +55,19 @@ import java.util.concurrent.TimeoutException; public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory { private static final SegmentAnalyzer analyzer = new SegmentAnalyzer(); - private static final SegmentMetadataQueryQueryToolChest toolChest = new SegmentMetadataQueryQueryToolChest(); private static final Logger log = new Logger(SegmentMetadataQueryRunnerFactory.class); + + private final SegmentMetadataQueryQueryToolChest toolChest; private final QueryWatcher queryWatcher; @Inject public SegmentMetadataQueryRunnerFactory( + SegmentMetadataQueryQueryToolChest toolChest, QueryWatcher queryWatcher ) { + this.toolChest = toolChest; this.queryWatcher = queryWatcher; } diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index a0a2487cca4..e3e6ad77152 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -73,6 +73,7 @@ public class SearchQueryQueryToolChest extends QueryToolChest OBJECT_TYPE_REFERENCE = new TypeReference() { }; + private final SearchQueryConfig config; @Inject @@ -247,6 +248,13 @@ public class SearchQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryConfig.java b/processing/src/main/java/io/druid/query/select/SelectQueryConfig.java new file mode 100644 index 00000000000..e5975df41c5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/select/SelectQueryConfig.java @@ -0,0 +1,37 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.query.select; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.query.QueryConfig; + +/** + */ +public class SelectQueryConfig extends QueryConfig +{ + @JsonProperty + private int maxResultsToCache = 10000; + + @Override + public int getMaxResultsToCache() + { + return maxResultsToCache; + } +} diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index f12c5c18544..1a48e66269c 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -70,11 +70,12 @@ public class SelectQueryQueryToolChest extends QueryToolChest>() { }; - private final QueryConfig config; + + private final SelectQueryConfig config; private final ObjectMapper jsonMapper; @Inject - public SelectQueryQueryToolChest(QueryConfig config, ObjectMapper jsonMapper) + public SelectQueryQueryToolChest(SelectQueryConfig config, ObjectMapper jsonMapper) { this.config = config; this.jsonMapper = jsonMapper; @@ -267,6 +268,12 @@ public class SelectQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/select/SelectResultValue.java b/processing/src/main/java/io/druid/query/select/SelectResultValue.java index d3ce5d6ef68..5493723702f 100644 --- a/processing/src/main/java/io/druid/query/select/SelectResultValue.java +++ b/processing/src/main/java/io/druid/query/select/SelectResultValue.java @@ -21,12 +21,7 @@ package io.druid.query.select; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonValue; -import com.google.common.base.Function; -import com.google.common.collect.Lists; -import com.metamx.common.ISE; -import javax.annotation.Nullable; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -41,7 +36,8 @@ public class SelectResultValue implements Iterable @JsonCreator public SelectResultValue( @JsonProperty("pagingIdentifiers") Map pagingIdentifiers, - @JsonProperty("events") List events) + @JsonProperty("events") List events + ) { this.pagingIdentifiers = pagingIdentifiers; this.events = events; diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index fdde44b8581..b77b11105e3 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -189,6 +189,12 @@ public class TimeBoundaryQueryQueryToolChest }; } + @Override + public int getCacheLimit() + { + return Integer.MAX_VALUE; + } + @Override public Sequence> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 7d0a5781ea4..2f1361ddfd4 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -71,6 +71,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest>() { }; + private final QueryConfig config; @Inject @@ -173,7 +174,7 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest, Object>() { @Override - public Object apply(@Nullable final Result input) + public Object apply(final Result input) { TimeseriesResultValue results = input.getValue(); final List retVal = Lists.newArrayListWithCapacity(1 + aggs.size()); @@ -219,6 +220,12 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java index 724d4818226..c46363ac389 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryRunnerFactory.java @@ -19,13 +19,11 @@ package io.druid.query.timeseries; -import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.metamx.common.ISE; import com.metamx.common.guava.Sequence; import io.druid.query.ChainedExecutionQueryRunner; import io.druid.query.Query; -import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryToolChest; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index f275651383f..2408e090dc0 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -72,6 +72,7 @@ public class TopNQueryQueryToolChest extends QueryToolChest OBJECT_TYPE_REFERENCE = new TypeReference() { }; + private final TopNQueryConfig config; @Inject @@ -367,6 +368,12 @@ public class TopNQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 4f6f6a0f67f..8d68738c22c 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -534,7 +534,7 @@ public class IncrementalIndex implements Iterable } } - return new MapBasedRow(timeAndDims.getTimestamp(), theVals); + return new MapBasedRow(gran.toDateTime(timeAndDims.getTimestamp()), theVals); } } ); diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java index 82311045d27..a9bec20d29c 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowFormatter.java @@ -32,7 +32,9 @@ import com.google.common.primitives.Floats; import com.metamx.common.ISE; import com.metamx.common.parsers.ParseException; import io.druid.data.input.InputRow; +import io.druid.data.input.Row; import io.druid.data.input.impl.SpatialDimensionSchema; +import org.joda.time.DateTime; import java.util.Arrays; import java.util.List; @@ -128,6 +130,12 @@ public class SpatialDimensionRowFormatter return row.getTimestampFromEpoch(); } + @Override + public DateTime getTimestamp() + { + return row.getTimestamp(); + } + @Override public List getDimension(String dimension) { @@ -157,6 +165,12 @@ public class SpatialDimensionRowFormatter { return row.toString(); } + + @Override + public int compareTo(Row o) + { + return getTimestamp().compareTo(o.getTimestamp()); + } }; if (!spatialPartialDimNames.isEmpty()) { diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index d718c0775d9..afe13f31184 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -19,6 +19,7 @@ package io.druid.query.groupby; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; @@ -35,6 +36,7 @@ import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -99,41 +101,71 @@ public class GroupByQueryRunnerTest @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { - GroupByQueryConfig config = new GroupByQueryConfig(); + final ObjectMapper mapper = new DefaultObjectMapper(); + final StupidPool pool = new StupidPool( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ); + + final GroupByQueryConfig config = new GroupByQueryConfig(); config.setMaxIntermediateRows(10000); final Supplier configSupplier = Suppliers.ofInstance(config); - final GroupByQueryEngine engine = new GroupByQueryEngine( - configSupplier, - new StupidPool( - new Supplier() - { - @Override - public ByteBuffer get() - { - return ByteBuffer.allocate(1024 * 1024); - } - } - ) - ); + final GroupByQueryEngine engine = new GroupByQueryEngine(configSupplier, pool); final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory( engine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, engine) + new GroupByQueryQueryToolChest(configSupplier, mapper, engine) ); + GroupByQueryConfig singleThreadedConfig = new GroupByQueryConfig() + { + @Override + public boolean isSingleThreaded() + { + return true; + } + }; + singleThreadedConfig.setMaxIntermediateRows(10000); + + final Supplier singleThreadedConfigSupplier = Suppliers.ofInstance(singleThreadedConfig); + final GroupByQueryEngine singleThreadEngine = new GroupByQueryEngine(singleThreadedConfigSupplier, pool); + + final GroupByQueryRunnerFactory singleThreadFactory = new GroupByQueryRunnerFactory( + singleThreadEngine, + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + singleThreadedConfigSupplier, + new GroupByQueryQueryToolChest(singleThreadedConfigSupplier, mapper, singleThreadEngine) + ); + + + Function function = new Function() + { + @Override + public Object apply(@Nullable Object input) + { + return new Object[]{factory, ((Object[]) input)[0]}; + } + }; + return Lists.newArrayList( - Iterables.transform( - QueryRunnerTestHelper.makeQueryRunners(factory), new Function() - { - @Override - public Object apply(@Nullable Object input) - { - return new Object[]{factory, ((Object[]) input)[0]}; - } - } + Iterables.concat( + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(factory), + function + ), + Iterables.transform( + QueryRunnerTestHelper.makeQueryRunners(singleThreadFactory), + function + ) ) ); } @@ -795,7 +827,11 @@ public class GroupByQueryRunnerTest ) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest( + configSupplier, + new DefaultObjectMapper(), + engine + ).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -844,7 +880,11 @@ public class GroupByQueryRunnerTest ) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest( + configSupplier, + new DefaultObjectMapper(), + engine + ).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -893,7 +933,11 @@ public class GroupByQueryRunnerTest ) ); - QueryRunner mergeRunner = new GroupByQueryQueryToolChest(configSupplier, engine).mergeResults(runner); + QueryRunner mergeRunner = new GroupByQueryQueryToolChest( + configSupplier, + new DefaultObjectMapper(), + engine + ).mergeResults(runner); TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); } @@ -1182,6 +1226,12 @@ public class GroupByQueryRunnerTest { return (row.getFloatMetric("idx_subpostagg") < 3800); } + + @Override + public byte[] getCacheKey() + { + return new byte[0]; + } } ) .addOrderByColumn("alias") @@ -1281,6 +1331,12 @@ public class GroupByQueryRunnerTest { return (row.getFloatMetric("idx_subpostagg") < 3800); } + + @Override + public byte[] getCacheKey() + { + return new byte[0]; + } } ) .addOrderByColumn("alias") @@ -1325,11 +1381,61 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0, "js_outer_agg", 123.92274475097656), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0, "js_outer_agg", 82.62254333496094), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0, "js_outer_agg", 125.58358001708984), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0, "js_outer_agg", 124.13470458984375), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0, "js_outer_agg", 162.74722290039062) + createExpectedRow( + "2011-04-01", + "alias", + "travel", + "rows", + 1L, + "idx", + 11119.0, + "js_outer_agg", + 123.92274475097656 + ), + createExpectedRow( + "2011-04-01", + "alias", + "technology", + "rows", + 1L, + "idx", + 11078.0, + "js_outer_agg", + 82.62254333496094 + ), + createExpectedRow( + "2011-04-01", + "alias", + "news", + "rows", + 1L, + "idx", + 11121.0, + "js_outer_agg", + 125.58358001708984 + ), + createExpectedRow( + "2011-04-01", + "alias", + "health", + "rows", + 1L, + "idx", + 11120.0, + "js_outer_agg", + 124.13470458984375 + ), + createExpectedRow( + "2011-04-01", + "alias", + "entertainment", + "rows", + 1L, + "idx", + 11158.0, + "js_outer_agg", + 162.74722290039062 + ) ); // Subqueries are handled by the ToolChest @@ -1350,7 +1456,6 @@ public class GroupByQueryRunnerTest return Sequences.toList(queryResult, Lists.newArrayList()); } - private Row createExpectedRow(final String timestamp, Object... vals) { return createExpectedRow(new DateTime(timestamp), vals); @@ -1365,6 +1470,7 @@ public class GroupByQueryRunnerTest theVals.put(vals[i].toString(), vals[i + 1]); } - return new MapBasedRow(new DateTime(timestamp), theVals); + DateTime ts = new DateTime(timestamp); + return new MapBasedRow(ts, theVals); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index a9fb506ca0b..dc2eb6657e3 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -28,6 +28,7 @@ import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; +import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Query; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -74,7 +75,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest engine, QueryRunnerTestHelper.NOOP_QUERYWATCHER, configSupplier, - new GroupByQueryQueryToolChest(configSupplier, engine) + new GroupByQueryQueryToolChest(configSupplier, new DefaultObjectMapper(), engine) ); final Collection objects = QueryRunnerTestHelper.makeQueryRunners(factory); @@ -98,13 +99,13 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest return Sequences.map( groupByRunner.run( GroupByQuery.builder() - .setDataSource(tsQuery.getDataSource()) - .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) - .setGranularity(tsQuery.getGranularity()) - .setDimFilter(tsQuery.getDimensionsFilter()) - .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) - .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) - .build() + .setDataSource(tsQuery.getDataSource()) + .setQuerySegmentSpec(tsQuery.getQuerySegmentSpec()) + .setGranularity(tsQuery.getGranularity()) + .setDimFilter(tsQuery.getDimensionsFilter()) + .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) + .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) + .build() ), new Function>() { diff --git a/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java index 460ace869b7..b476ffcd608 100644 --- a/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java +++ b/processing/src/test/java/io/druid/query/groupby/having/HavingSpecTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.Result; import org.junit.Test; import java.util.ArrayList; @@ -148,6 +149,12 @@ public class HavingSpecTest counter.incrementAndGet(); return value; } + + @Override + public byte[] getCacheKey() + { + return new byte[0]; + } } @Test diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java index 70c65f8da88..b9834c106cc 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java @@ -22,10 +22,10 @@ package io.druid.query.metadata; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.query.LegacyDataSource; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; -import io.druid.query.QueryWatcher; import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.SegmentAnalysis; import io.druid.query.metadata.metadata.SegmentMetadataQuery; @@ -67,7 +67,7 @@ public class SegmentAnalyzerTest final SegmentAnalysis analysis = results.get(0); Assert.assertEquals("test_1", analysis.getId()); - final Map columns = analysis.getColumns(); + final Map columns = analysis.getColumns(); Assert.assertEquals(TestIndex.COLUMNS.length, columns.size()); // All columns including time for (String dimension : TestIndex.DIMENSIONS) { @@ -91,12 +91,16 @@ public class SegmentAnalyzerTest * *Awesome* method name auto-generated by IntelliJ! I love IntelliJ! * * @param index + * * @return */ private List getSegmentAnalysises(Segment index) { final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner( - (QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER), index + (QueryRunnerFactory) new SegmentMetadataQueryRunnerFactory( + new SegmentMetadataQueryQueryToolChest(new QueryConfig()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ), index ); final SegmentMetadataQuery query = new SegmentMetadataQuery( diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index ed1740460f8..0c58576c1d4 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -26,6 +26,7 @@ import com.metamx.common.guava.Sequences; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.Query; +import io.druid.query.QueryConfig; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; @@ -45,7 +46,9 @@ public class SegmentMetadataQueryTest { @SuppressWarnings("unchecked") private final QueryRunner runner = makeQueryRunner( - new SegmentMetadataQueryRunnerFactory(QueryRunnerTestHelper.NOOP_QUERYWATCHER) + new SegmentMetadataQueryRunnerFactory( + new SegmentMetadataQueryQueryToolChest(new QueryConfig()), + QueryRunnerTestHelper.NOOP_QUERYWATCHER) ); private ObjectMapper mapper = new DefaultObjectMapper(); diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index 07f99165873..e5b6fd4760f 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -59,7 +59,7 @@ public class SelectQueryRunnerTest { return QueryRunnerTestHelper.makeQueryRunners( new SelectQueryRunnerFactory( - new SelectQueryQueryToolChest(new QueryConfig(), new DefaultObjectMapper()), + new SelectQueryQueryToolChest(new SelectQueryConfig(), new DefaultObjectMapper()), new SelectQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index a1fab7efecb..68f4a908049 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -22,6 +22,7 @@ package io.druid.segment; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import io.druid.data.input.Row; import io.druid.query.Result; import org.junit.Assert; @@ -41,7 +42,11 @@ public class TestHelper assertResults(expectedResults, results, ""); } - public static void assertExpectedResults(Iterable> expectedResults, Iterable> results, String failMsg) + public static void assertExpectedResults( + Iterable> expectedResults, + Iterable> results, + String failMsg + ) { assertResults(expectedResults, results, failMsg); } @@ -56,23 +61,33 @@ public class TestHelper assertObjects(expectedResults, Sequences.toList(results, Lists.newArrayList()), failMsg); } - private static void assertResults(Iterable> expectedResults, Iterable> actualResults, String failMsg) + private static void assertResults( + Iterable> expectedResults, + Iterable> actualResults, + String failMsg + ) { Iterator resultsIter = actualResults.iterator(); Iterator resultsIter2 = actualResults.iterator(); Iterator expectedResultsIter = expectedResults.iterator(); while (resultsIter.hasNext() && resultsIter2.hasNext() && expectedResultsIter.hasNext()) { - Result expectedNext = expectedResultsIter.next(); - final Result next = resultsIter.next(); - final Result next2 = resultsIter2.next(); + Object expectedNext = expectedResultsIter.next(); + final Object next = resultsIter.next(); + final Object next2 = resultsIter2.next(); - assertResult(failMsg, expectedNext, next); - assertResult( - String.format("%s: Second iterator bad, multiple calls to iterator() should be safe", failMsg), - expectedNext, - next2 - ); + if (expectedNext instanceof Row) { + // HACK! Special casing for groupBy + Assert.assertEquals(failMsg, expectedNext, next); + Assert.assertEquals(failMsg, expectedNext, next2); + } else { + assertResult(failMsg, (Result) expectedNext, (Result) next); + assertResult( + String.format("%s: Second iterator bad, multiple calls to iterator() should be safe", failMsg), + (Result) expectedNext, + (Result) next2 + ); + } } if (resultsIter.hasNext()) { @@ -90,7 +105,9 @@ public class TestHelper if (expectedResultsIter.hasNext()) { Assert.fail( String.format( - "%s: Expected expectedResultsIter to be exhausted, next element was %s", failMsg, expectedResultsIter.next() + "%s: Expected expectedResultsIter to be exhausted, next element was %s", + failMsg, + expectedResultsIter.next() ) ); } @@ -130,7 +147,9 @@ public class TestHelper if (expectedResultsIter.hasNext()) { Assert.fail( String.format( - "%s: Expected expectedResultsIter to be exhausted, next element was %s", failMsg, expectedResultsIter.next() + "%s: Expected expectedResultsIter to be exhausted, next element was %s", + failMsg, + expectedResultsIter.next() ) ); } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index fe38811f332..1e296f0a4d1 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -127,9 +127,12 @@ public class CachingClusteredClient implements QueryRunner final boolean useCache = query.getContextUseCache(true) && strategy != null - && cacheConfig.isUseCache(); + && cacheConfig.isUseCache() + && cacheConfig.isQueryCacheable(query); final boolean populateCache = query.getContextPopulateCache(true) - && strategy != null && cacheConfig.isPopulateCache(); + && strategy != null + && cacheConfig.isPopulateCache() + && cacheConfig.isQueryCacheable(query); final boolean isBySegment = query.getContextBySegment(false); @@ -239,6 +242,15 @@ public class CachingClusteredClient implements QueryRunner } descriptors.add(segment.rhs); + System.out.println( + String.format( + "Server %s has %s_%s_%s", + server.getHost(), + segment.rhs.getInterval(), + segment.rhs.getPartitionNumber(), + segment.rhs.getVersion() + ) + ); } } diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index ad2718e6a45..41d3813ada7 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -78,14 +78,16 @@ public class CachingQueryRunner implements QueryRunner final boolean populateCache = query.getContextPopulateCache(true) && strategy != null - && cacheConfig.isPopulateCache(); + && cacheConfig.isPopulateCache() + && cacheConfig.isQueryCacheable(query); final boolean useCache = query.getContextUseCache(true) - && strategy != null - && cacheConfig.isUseCache(); + && strategy != null + && cacheConfig.isUseCache() + && cacheConfig.isQueryCacheable(query); final Cache.NamedKey key; - if(strategy != null && (useCache || populateCache)) { + if (strategy != null && (useCache || populateCache)) { key = CacheUtil.computeSegmentCacheKey( segmentIdentifier, segmentDescriptor, @@ -95,10 +97,10 @@ public class CachingQueryRunner implements QueryRunner key = null; } - if(useCache) { + if (useCache) { final Function cacheFn = strategy.pullFromCache(); final byte[] cachedResult = cache.get(key); - if(cachedResult != null) { + if (cachedResult != null) { final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); return Sequences.map( diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index 817cdb4070a..f81c01de07e 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -20,16 +20,26 @@ package io.druid.client.cache; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Sets; +import io.druid.query.Query; + +import java.util.Arrays; +import java.util.List; public class CacheConfig { public static String USE_CACHE = "useCache"; public static String POPULATE_CACHE = "populateCache"; + @JsonProperty private boolean useCache = true; + @JsonProperty private boolean populateCache = true; + @JsonProperty + private List unCacheable = Arrays.asList(Query.GROUP_BY, Query.SELECT); + public boolean isPopulateCache() { return populateCache; @@ -39,4 +49,9 @@ public class CacheConfig { return useCache; } + + public boolean isQueryCacheable(Query query) + { + return !Sets.newHashSet(unCacheable).contains(query.getType()); + } } diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index ed5b59591a9..0b06eddc015 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -35,6 +35,7 @@ import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQueryConfig; import io.druid.query.select.SelectQuery; +import io.druid.query.select.SelectQueryConfig; import io.druid.query.select.SelectQueryQueryToolChest; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; @@ -74,6 +75,7 @@ public class QueryToolChestModule implements Module JsonConfigProvider.bind(binder, "druid.query", QueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); + JsonConfigProvider.bind(binder, "druid.query.select", SelectQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class); } } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java index d2c671930e6..4695521089f 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java @@ -29,6 +29,7 @@ import com.maxmind.geoip2.exception.GeoIp2Exception; import com.maxmind.geoip2.model.Omni; import com.metamx.common.logger.Logger; import io.druid.data.input.InputRow; +import io.druid.data.input.Row; import org.apache.commons.io.FileUtils; import org.joda.time.DateTime; @@ -211,6 +212,12 @@ class WikipediaIrcDecoder implements IrcDecoder return timestamp.getMillis(); } + @Override + public DateTime getTimestamp() + { + return timestamp; + } + @Override public List getDimension(String dimension) { @@ -234,6 +241,12 @@ class WikipediaIrcDecoder implements IrcDecoder return metrics.get(metric); } + @Override + public int compareTo(Row o) + { + return timestamp.compareTo(o.getTimestamp()); + } + @Override public String toString() { diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 90ac6f24e03..4acd5128af4 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -30,6 +32,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.metamx.common.ISE; +import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.MergeIterable; import com.metamx.common.guava.Sequence; @@ -42,6 +45,9 @@ import io.druid.client.selector.HighestPriorityTierSelectorStrategy; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.RandomServerSelectorStrategy; import io.druid.client.selector.ServerSelector; +import io.druid.collections.StupidPool; +import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; import io.druid.jackson.DefaultObjectMapper; @@ -56,7 +62,6 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.Result; import io.druid.query.SegmentDescriptor; -import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -64,13 +69,24 @@ import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.aggregation.post.FieldAccessPostAggregator; +import io.druid.query.dimension.DefaultDimensionSpec; +import io.druid.query.dimension.DimensionSpec; import io.druid.query.filter.DimFilter; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryEngine; +import io.druid.query.groupby.GroupByQueryQueryToolChest; import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.SearchResultValue; -import io.druid.query.search.search.InsensitiveContainsSearchQuerySpec; import io.druid.query.search.search.SearchHit; import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQueryConfig; +import io.druid.query.select.EventHolder; +import io.druid.query.select.PagingSpec; +import io.druid.query.select.SelectQuery; +import io.druid.query.select.SelectQueryConfig; +import io.druid.query.select.SelectQueryQueryToolChest; +import io.druid.query.select.SelectResultValue; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; @@ -105,6 +121,7 @@ import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; @@ -657,19 +674,19 @@ public class CachingClusteredClientTest @Test public void testSearchCaching() throws Exception { + final Druids.SearchQueryBuilder builder = Druids.newSearchQueryBuilder() + .dataSource(DATA_SOURCE) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .limit(1000) + .intervals(SEG_SPEC) + .dimensions(Arrays.asList("a_dim")) + .query("how") + .context(CONTEXT); + testQueryCaching( client, - new SearchQuery( - new TableDataSource(DATA_SOURCE), - DIM_FILTER, - GRANULARITY, - 1000, - SEG_SPEC, - Arrays.asList("a_dim"), - new InsensitiveContainsSearchQuerySpec("how"), - null, - CONTEXT - ), + builder.build(), new Interval("2011-01-01/2011-01-02"), makeSearchResults(new DateTime("2011-01-01"), "how", "howdy", "howwwwww", "howwy"), @@ -694,6 +711,188 @@ public class CachingClusteredClientTest new DateTime("2011-01-09T01"), "how6", "howdy6", "howwwwww6", "howww6" ) ); + + QueryRunner runner = new FinalizeResultsQueryRunner(client, new SearchQueryQueryToolChest(new SearchQueryConfig())); + TestHelper.assertExpectedResults( + makeSearchResults( + new DateTime("2011-01-01"), "how", "howdy", "howwwwww", "howwy", + new DateTime("2011-01-02"), "how1", "howdy1", "howwwwww1", "howwy1", + new DateTime("2011-01-05"), "how2", "howdy2", "howwwwww2", "howww2", + new DateTime("2011-01-05T01"), "how2", "howdy2", "howwwwww2", "howww2", + new DateTime("2011-01-06"), "how3", "howdy3", "howwwwww3", "howww3", + new DateTime("2011-01-06T01"), "how3", "howdy3", "howwwwww3", "howww3", + new DateTime("2011-01-07"), "how4", "howdy4", "howwwwww4", "howww4", + new DateTime("2011-01-07T01"), "how4", "howdy4", "howwwwww4", "howww4", + new DateTime("2011-01-08"), "how5", "howdy5", "howwwwww5", "howww5", + new DateTime("2011-01-08T01"), "how5", "howdy5", "howwwwww5", "howww5", + new DateTime("2011-01-09"), "how6", "howdy6", "howwwwww6", "howww6", + new DateTime("2011-01-09T01"), "how6", "howdy6", "howwwwww6", "howww6" + ), + runner.run( + builder.intervals("2011-01-01/2011-01-10") + .build() + ) + ); + } + + @Test + public void testSelectCaching() throws Exception + { + Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .dimensions(Arrays.asList("a")) + .metrics(Arrays.asList("rows")) + .pagingSpec(new PagingSpec(null, 3)) + .context(CONTEXT); + + testQueryCaching( + client, + builder.build(), + new Interval("2011-01-01/2011-01-02"), + makeSelectResults(new DateTime("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1)), + + new Interval("2011-01-02/2011-01-03"), + makeSelectResults(new DateTime("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5)), + + new Interval("2011-01-05/2011-01-10"), + makeSelectResults( + new DateTime("2011-01-05"), ImmutableMap.of("a", "d", "rows", 5), + new DateTime("2011-01-06"), ImmutableMap.of("a", "e", "rows", 6), + new DateTime("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), + new DateTime("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), + new DateTime("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9) + ), + + new Interval("2011-01-05/2011-01-10"), + makeSelectResults( + new DateTime("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), + new DateTime("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), + new DateTime("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), + new DateTime("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), + new DateTime("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) + ) + ); + + QueryRunner runner = new FinalizeResultsQueryRunner( + client, + new SelectQueryQueryToolChest( + new SelectQueryConfig(), + jsonMapper + ) + ); + TestHelper.assertExpectedResults( + makeSelectResults( + new DateTime("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1), + new DateTime("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5), + new DateTime("2011-01-05"), ImmutableMap.of("a", "d", "rows", 5), + new DateTime("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), + new DateTime("2011-01-06"), ImmutableMap.of("a", "e", "rows", 6), + new DateTime("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), + new DateTime("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), + new DateTime("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), + new DateTime("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), + new DateTime("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), + new DateTime("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), + new DateTime("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) + ), + runner.run( + builder.intervals("2011-01-01/2011-01-10") + .build() + ) + ); + } + + @Test + public void testGroupByCaching() throws Exception + { + GroupByQuery.Builder builder = new GroupByQuery.Builder() + .setDataSource(DATA_SOURCE) + .setQuerySegmentSpec(SEG_SPEC) + .setDimFilter(DIM_FILTER) + .setGranularity(GRANULARITY) + .setDimensions(Arrays.asList(new DefaultDimensionSpec("a", "a"))) + .setAggregatorSpecs(AGGS) + .setPostAggregatorSpecs(POST_AGGS) + .setContext(CONTEXT); + + testQueryCaching( + client, + builder.build(), + new Interval("2011-01-01/2011-01-02"), + makeGroupByResults(new DateTime("2011-01-01"), ImmutableMap.of("a", "a", "rows", 1, "imps", 1, "impers", 1)), + + new Interval("2011-01-02/2011-01-03"), + makeGroupByResults(new DateTime("2011-01-02"), ImmutableMap.of("a", "b", "rows", 2, "imps", 2, "impers", 2)), + + new Interval("2011-01-05/2011-01-10"), + makeGroupByResults( + new DateTime("2011-01-05"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3), + new DateTime("2011-01-06"), ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4), + new DateTime("2011-01-07"), ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5), + new DateTime("2011-01-08"), ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6), + new DateTime("2011-01-09"), ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7) + ), + + new Interval("2011-01-05/2011-01-10"), + makeGroupByResults( + new DateTime("2011-01-05T01"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3), + new DateTime("2011-01-06T01"), ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4), + new DateTime("2011-01-07T01"), ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5), + new DateTime("2011-01-08T01"), ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6), + new DateTime("2011-01-09T01"), ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7) + ) + ); + + Supplier configSupplier = new Supplier() + { + @Override + public GroupByQueryConfig get() + { + return new GroupByQueryConfig(); + } + }; + QueryRunner runner = new FinalizeResultsQueryRunner( + client, + new GroupByQueryQueryToolChest( + configSupplier, + jsonMapper, + new GroupByQueryEngine( + configSupplier, + new StupidPool<>( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ) + ) + ); + TestHelper.assertExpectedObjects( + makeGroupByResults( + new DateTime("2011-01-05T"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3), + new DateTime("2011-01-05T01"), ImmutableMap.of("a", "c", "rows", 3, "imps", 3, "impers", 3), + new DateTime("2011-01-06T"), ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4), + new DateTime("2011-01-06T01"), ImmutableMap.of("a", "d", "rows", 4, "imps", 4, "impers", 4), + new DateTime("2011-01-07T"), ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5), + new DateTime("2011-01-07T01"), ImmutableMap.of("a", "e", "rows", 5, "imps", 5, "impers", 5), + new DateTime("2011-01-08T"), ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6), + new DateTime("2011-01-08T01"), ImmutableMap.of("a", "f", "rows", 6, "imps", 6, "impers", 6), + new DateTime("2011-01-09T"), ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7), + new DateTime("2011-01-09T01"), ImmutableMap.of("a", "g", "rows", 7, "imps", 7, "impers", 7) + ), + runner.run( + builder.setInterval("2011-01-05/2011-01-10") + .build() + ), + "" + ); } @Test @@ -751,7 +950,8 @@ public class CachingClusteredClientTest final QueryRunner runner, final int numTimesToQuery, boolean expectBySegment, - final Query query, Object... args // does this assume query intervals must be ordered? + final Query query, + Object... args // does this assume query intervals must be ordered? ) { if (args.length % 2 != 0) { @@ -843,6 +1043,30 @@ public class CachingClusteredClientTest EasyMock.expect(queryable.run(EasyMock.capture(capture))) .andReturn(toQueryableSearchResults(segmentIds, intervals, results)) .once(); + } else if (query instanceof SelectQuery) { + List segmentIds = Lists.newArrayList(); + List intervals = Lists.newArrayList(); + List>> results = Lists.newArrayList(); + for (ServerExpectation expectation : expectations) { + segmentIds.add(expectation.getSegmentId()); + intervals.add(expectation.getInterval()); + results.add(expectation.getResults()); + } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) + .andReturn(toQueryableSelectResults(segmentIds, intervals, results)) + .once(); + } else if (query instanceof GroupByQuery) { + List segmentIds = Lists.newArrayList(); + List intervals = Lists.newArrayList(); + List> results = Lists.newArrayList(); + for (ServerExpectation expectation : expectations) { + segmentIds.add(expectation.getSegmentId()); + intervals.add(expectation.getInterval()); + results.add(expectation.getResults()); + } + EasyMock.expect(queryable.run(EasyMock.capture(capture))) + .andReturn(toQueryableGroupByResults(segmentIds, intervals, results)) + .once(); } else if (query instanceof TimeBoundaryQuery) { List segmentIds = Lists.newArrayList(); List intervals = Lists.newArrayList(); @@ -1097,6 +1321,74 @@ public class CachingClusteredClientTest ); } + private Sequence> toQueryableSelectResults( + Iterable segmentIds, Iterable intervals, Iterable>> results + ) + { + return Sequences.simple( + FunctionalIterable + .create(segmentIds) + .trinaryTransform( + intervals, + results, + new TrinaryFn>, Result>() + { + @Override + @SuppressWarnings("unchecked") + public Result apply( + final String segmentId, + final Interval interval, + final Iterable> results + ) + { + return new Result( + results.iterator().next().getTimestamp(), + new BySegmentResultValueClass( + Lists.newArrayList(results), + segmentId, + interval + ) + ); + } + } + ) + ); + } + + private Sequence toQueryableGroupByResults( + Iterable segmentIds, Iterable intervals, Iterable> results + ) + { + return Sequences.simple( + FunctionalIterable + .create(segmentIds) + .trinaryTransform( + intervals, + results, + new TrinaryFn, Result>() + { + @Override + @SuppressWarnings("unchecked") + public Result apply( + final String segmentId, + final Interval interval, + final Iterable results + ) + { + return new Result( + results.iterator().next().getTimestamp(), + new BySegmentResultValueClass( + Lists.newArrayList(results), + segmentId, + interval + ) + ); + } + } + ) + ); + } + private Sequence> toQueryableTimeBoundaryResults( Iterable segmentIds, Iterable intervals, @@ -1305,6 +1597,40 @@ public class CachingClusteredClientTest return retVal; } + private Iterable> makeSelectResults(Object... objects) + { + List> retVal = Lists.newArrayList(); + int index = 0; + while (index < objects.length) { + DateTime timestamp = (DateTime) objects[index++]; + + List values = Lists.newArrayList(); + while (index < objects.length && !(objects[index] instanceof DateTime)) { + values.add(new EventHolder(null, 0, (Map) objects[index++])); + } + + retVal.add(new Result<>(timestamp, new SelectResultValue(null, values))); + } + return retVal; + } + + private Iterable makeGroupByResults(Object... objects) + { + List retVal = Lists.newArrayList(); + int index = 0; + while (index < objects.length) { + DateTime timestamp = (DateTime) objects[index++]; + + //List values = Lists.newArrayList(); + //while (index < objects.length && !(objects[index] instanceof DateTime)) { + retVal.add(new MapBasedRow(timestamp, (Map) objects[index++])); + //} + + //retVal.add(new Result<>(timestamp, values)); + } + return retVal; + } + private T makeMock(List mocks, Class clazz) { T obj = EasyMock.createMock(clazz); @@ -1324,6 +1650,8 @@ public class CachingClusteredClientTest protected CachingClusteredClient makeClient() { + final Supplier groupByQueryConfigSupplier = Suppliers.ofInstance(new GroupByQueryConfig()); + return new CachingClusteredClient( new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() @@ -1333,6 +1661,30 @@ public class CachingClusteredClientTest ) .put(TopNQuery.class, new TopNQueryQueryToolChest(new TopNQueryConfig())) .put(SearchQuery.class, new SearchQueryQueryToolChest(new SearchQueryConfig())) + .put( + SelectQuery.class, + new SelectQueryQueryToolChest(new SelectQueryConfig(), jsonMapper) + ) + .put( + GroupByQuery.class, + new GroupByQueryQueryToolChest( + groupByQueryConfigSupplier, + jsonMapper, + new GroupByQueryEngine( + groupByQueryConfigSupplier, + new StupidPool<>( + new Supplier() + { + @Override + public ByteBuffer get() + { + return ByteBuffer.allocate(1024 * 1024); + } + } + ) + ) + ) + ) .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) .build() ), @@ -1364,6 +1716,13 @@ public class CachingClusteredClientTest cache, jsonMapper, new CacheConfig() + { + @Override + public boolean isQueryCacheable(Query query) + { + return true; + } + } ); } diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index d1d6e1e6f67..ea231c2611e 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -27,6 +27,7 @@ import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.Row; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; import io.druid.query.Query; @@ -168,6 +169,12 @@ public class RealtimeManagerTest return timestamp; } + @Override + public DateTime getTimestamp() + { + return new DateTime(timestamp); + } + @Override public List getDimension(String dimension) { @@ -185,6 +192,12 @@ public class RealtimeManagerTest { return null; } + + @Override + public int compareTo(Row o) + { + return 0; + } }; } diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index b068a994cb7..cdd182e3e1f 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import io.druid.data.input.InputRow; +import io.druid.data.input.Row; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -81,6 +82,12 @@ public class SinkTest return new DateTime("2013-01-01").getMillis(); } + @Override + public DateTime getTimestamp() + { + return new DateTime("2013-01-01"); + } + @Override public List getDimension(String dimension) { @@ -98,6 +105,12 @@ public class SinkTest { return null; } + + @Override + public int compareTo(Row o) + { + return 0; + } } ); @@ -122,6 +135,12 @@ public class SinkTest return new DateTime("2013-01-01").getMillis(); } + @Override + public DateTime getTimestamp() + { + return new DateTime("2013-01-01"); + } + @Override public List getDimension(String dimension) { @@ -139,6 +158,12 @@ public class SinkTest { return null; } + + @Override + public int compareTo(Row o) + { + return 0; + } } ); diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index bfd573dae89..afab880ff81 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -25,10 +25,12 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import io.druid.TestUtil; import io.druid.data.input.InputRow; +import io.druid.data.input.Row; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.ShardSpec; import junit.framework.Assert; +import org.joda.time.DateTime; import org.junit.Test; import java.util.ArrayList; @@ -176,6 +178,12 @@ public class HashBasedNumberedShardSpecTest return 0; } + @Override + public DateTime getTimestamp() + { + return new DateTime(0); + } + @Override public List getDimension(String s) { @@ -193,5 +201,11 @@ public class HashBasedNumberedShardSpecTest { return 0; } + + @Override + public int compareTo(Row o) + { + return 0; + } } } From bdfeccd092b3e0155715d338d851452f90c2eba9 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 15 Jul 2014 15:57:58 -0700 Subject: [PATCH 148/270] doc and timeout fix --- docs/content/Production-Cluster-Configuration.md | 2 ++ .../main/java/io/druid/segment/loading/SegmentLoaderConfig.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 6eeb3a0c97b..689a34b98a9 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -4,6 +4,8 @@ layout: doc_page Production Cluster Configuration ================================ +__This configuration is an example of what a production cluster could look like. Many other hardware combinations are possible! Cheaper hardware is absolutely possible.__ + This production Druid cluster assumes that MySQL and Zookeeper are already set up. The deep storage that is used for examples is S3 and memcached is used for a distributed cache. The nodes that respond to queries (Historical, Broker, and Middle manager nodes) will use as many cores as are available, depending on usage, so it is best to keep these on dedicated machines. The upper limit of effectively utilized cores is not well characterized yet and would depend on types of queries, query load, and the schema. Historical daemons should have a heap a size of at least 1GB per core for normal usage, but could be squeezed into a smaller heap for testing. Since in-memory caching is essential for good performance, even more RAM is better. Broker nodes will use RAM for caching, so they do more than just route queries. SSDs are highly recommended for Historical nodes not all data is loaded in available memory. diff --git a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java index cefb275e946..2cba40194a0 100644 --- a/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/io/druid/segment/loading/SegmentLoaderConfig.java @@ -38,7 +38,7 @@ public class SegmentLoaderConfig private boolean deleteOnRemove = true; @JsonProperty("dropSegmentDelayMillis") - private int dropSegmentDelayMillis = 5 * 60 * 1000; // 5 mins + private int dropSegmentDelayMillis = 30 * 1000; // 30 seconds @JsonProperty private File infoDir = null; From 27c47507803729774b54977844d54495cfcb5b3b Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 15 Jul 2014 16:00:56 -0700 Subject: [PATCH 149/270] fix infinite loop in test --- .../server/shard/HashBasedNumberedShardSpecTest.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index 5f176015c2a..2ab82655bf0 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -196,14 +196,15 @@ public class HashBasedNumberedShardSpecTest } @Test - public void testValidity(){ - for(int i=Integer.MIN_VALUE;i<=Integer.MAX_VALUE;i++){ + public void testValidity() + { + for (int i = Integer.MIN_VALUE; i < Integer.MAX_VALUE; i++) { { int partitionNum = Math.abs((int) ((long) i % 2)); - if(partitionNum != 0 && partitionNum != 1){ - throw new ISE("for i "+ i+ "partitionNum "+ partitionNum); + if (partitionNum != 0 && partitionNum != 1) { + throw new ISE("for i " + i + "partitionNum " + partitionNum); } } - } + } } } From 935b642b2cf6ae58bbee2d6251b9c9ec659ba673 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 15 Jul 2014 16:13:11 -0700 Subject: [PATCH 150/270] fix overlord binding problems --- services/src/main/java/io/druid/cli/CliOverlord.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 9f95feb43ad..95780b57156 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -44,6 +44,7 @@ import io.druid.guice.PolyBind; import io.druid.indexing.common.actions.LocalTaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionToolbox; +import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.config.TaskStorageConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.indexing.common.tasklogs.SwitchingTaskLogStreamer; @@ -114,6 +115,7 @@ public class CliOverlord extends ServerRunnable public void configure(Binder binder) { JsonConfigProvider.bind(binder, "druid.indexer.queue", TaskQueueConfig.class); + JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); binder.bind(TaskMaster.class).in(ManageLifecycle.class); From b74845e94293e619fc6d26d8a70c4088250834a1 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Wed, 16 Jul 2014 09:39:51 +0530 Subject: [PATCH 151/270] fix test committed it by mistake. --- .../shard/HashBasedNumberedShardSpecTest.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index 2ab82655bf0..bfd573dae89 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -194,17 +194,4 @@ public class HashBasedNumberedShardSpecTest return 0; } } - - @Test - public void testValidity() - { - for (int i = Integer.MIN_VALUE; i < Integer.MAX_VALUE; i++) { - { - int partitionNum = Math.abs((int) ((long) i % 2)); - if (partitionNum != 0 && partitionNum != 1) { - throw new ISE("for i " + i + "partitionNum " + partitionNum); - } - } - } - } } From 534acc9eab8748d0131174ec478bb40ab1f65f41 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 15 Jul 2014 22:50:38 -0700 Subject: [PATCH 152/270] fix mm --- services/src/main/java/io/druid/cli/CliMiddleManager.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index e3356012358..381b382269a 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -35,6 +35,7 @@ import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Self; +import io.druid.indexing.common.config.TaskConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.indexing.overlord.ForkingTaskRunner; import io.druid.indexing.overlord.TaskRunner; @@ -75,6 +76,7 @@ public class CliMiddleManager extends ServerRunnable { IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder); + JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); JsonConfigProvider.bind(binder, "druid.worker", WorkerConfig.class); binder.bind(TaskRunner.class).to(ForkingTaskRunner.class); From b70a6b106139064dd65daa59f84207e4824fffca Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 16 Jul 2014 13:52:06 -0700 Subject: [PATCH 153/270] address code review comments --- .../java/io/druid/query/CacheStrategy.java | 2 - .../query/GroupByParallelQueryRunner.java | 16 ++++---- .../main/java/io/druid/query/QueryConfig.java | 8 ---- .../query/groupby/GroupByQueryConfig.java | 9 ----- .../groupby/GroupByQueryQueryToolChest.java | 24 ++++++++---- .../groupby/GroupByQueryRunnerFactory.java | 12 +++--- .../SegmentMetadataQueryQueryToolChest.java | 6 --- .../search/SearchQueryQueryToolChest.java | 7 ---- .../druid/query/select/SelectQueryConfig.java | 37 ------------------- .../select/SelectQueryQueryToolChest.java | 10 +---- .../TimeBoundaryQueryQueryToolChest.java | 6 --- .../TimeseriesQueryQueryToolChest.java | 6 --- .../query/topn/TopNQueryQueryToolChest.java | 6 --- .../segment/incremental/IncrementalIndex.java | 2 +- .../query/select/SelectQueryRunnerTest.java | 2 +- .../io/druid/guice/QueryToolChestModule.java | 2 - .../client/CachingClusteredClientTest.java | 12 +----- 17 files changed, 36 insertions(+), 131 deletions(-) delete mode 100644 processing/src/main/java/io/druid/query/select/SelectQueryConfig.java diff --git a/processing/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java index 737ed30ad5b..f77affcb5bf 100644 --- a/processing/src/main/java/io/druid/query/CacheStrategy.java +++ b/processing/src/main/java/io/druid/query/CacheStrategy.java @@ -35,7 +35,5 @@ public interface CacheStrategy> public Function pullFromCache(); - public int getCacheLimit(); - public Sequence mergeSequences(Sequence> seqOfSequences); } diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index a460cd68485..1edf07b94d4 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -84,30 +84,30 @@ public class GroupByParallelQueryRunner implements QueryRunner if (Iterables.isEmpty(queryables)) { log.warn("No queryables found."); } - ListenableFuture> futures = Futures.allAsList( + ListenableFuture> futures = Futures.allAsList( Lists.newArrayList( Iterables.transform( queryables, - new Function, ListenableFuture>() + new Function, ListenableFuture>() { @Override - public ListenableFuture apply(final QueryRunner input) + public ListenableFuture apply(final QueryRunner input) { return exec.submit( - new AbstractPrioritizedCallable(priority) + new AbstractPrioritizedCallable(priority) { @Override - public Boolean call() throws Exception + public Void call() throws Exception { try { if (bySegment) { input.run(queryParam) .accumulate(bySegmentAccumulatorPair.lhs, bySegmentAccumulatorPair.rhs); - return true; + } else { + input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); } - input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs); - return true; + return null; } catch (QueryInterruptedException e) { throw Throwables.propagate(e); diff --git a/processing/src/main/java/io/druid/query/QueryConfig.java b/processing/src/main/java/io/druid/query/QueryConfig.java index 53d757f6c75..f5da67a39a1 100644 --- a/processing/src/main/java/io/druid/query/QueryConfig.java +++ b/processing/src/main/java/io/druid/query/QueryConfig.java @@ -29,16 +29,8 @@ public class QueryConfig @JsonProperty private Period chunkPeriod = new Period(); - @JsonProperty - private int maxResultsToCache = Integer.MAX_VALUE; - public Period getChunkPeriod() { return chunkPeriod; } - - public int getMaxResultsToCache() - { - return maxResultsToCache; - } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java index 5e723a2ff23..0139e16742e 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryConfig.java @@ -35,9 +35,6 @@ public class GroupByQueryConfig extends QueryConfig @JsonProperty private int maxResults = 500000; - @JsonProperty - private int maxResultsToCache = 10000; - public boolean isSingleThreaded() { return singleThreaded; @@ -57,10 +54,4 @@ public class GroupByQueryConfig extends QueryConfig { return maxResults; } - - @Override - public int getMaxResultsToCache() - { - return maxResultsToCache; - } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index d0ba9b603b1..88216281c4e 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -160,7 +160,22 @@ public class GroupByQueryQueryToolChest extends QueryToolChest postAggregate(final GroupByQuery query, IncrementalIndex index) { - return Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())); + return Sequences.map( + Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())), + new Function() + { + @Override + public Row apply(Row input) + { + final MapBasedRow row = (MapBasedRow) input; + return new MapBasedRow( + query.getGranularity() + .toDateTime(row.getTimestampFromEpoch()), + row.getEvent() + ); + } + } + ); } private IncrementalIndex makeIncrementalIndex(GroupByQuery query, Sequence rows) @@ -179,7 +194,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { return new OrderedMergeSequence<>(Ordering.natural().nullsFirst(), seqOfSequences); - //return new ConcatSequence<>(seqOfSequences); } @Override @@ -341,12 +355,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java index 24e85b9f1c2..1cc5600c3b6 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryRunnerFactory.java @@ -117,11 +117,11 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory future = queryExecutor.submit( - new AbstractPrioritizedCallable(priority) + final ListenableFuture future = queryExecutor.submit( + new AbstractPrioritizedCallable(priority) { @Override - public Boolean call() throws Exception + public Void call() throws Exception { if (bySegment) { input.run(queryParam) @@ -129,11 +129,11 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory mergeSequences(Sequence> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index e3e6ad77152..e24239f8896 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -248,13 +248,6 @@ public class SearchQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryConfig.java b/processing/src/main/java/io/druid/query/select/SelectQueryConfig.java deleted file mode 100644 index e5975df41c5..00000000000 --- a/processing/src/main/java/io/druid/query/select/SelectQueryConfig.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.query.select; - -import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.query.QueryConfig; - -/** - */ -public class SelectQueryConfig extends QueryConfig -{ - @JsonProperty - private int maxResultsToCache = 10000; - - @Override - public int getMaxResultsToCache() - { - return maxResultsToCache; - } -} diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 1a48e66269c..95480302336 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -71,11 +71,11 @@ public class SelectQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index b77b11105e3..fdde44b8581 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -189,12 +189,6 @@ public class TimeBoundaryQueryQueryToolChest }; } - @Override - public int getCacheLimit() - { - return Integer.MAX_VALUE; - } - @Override public Sequence> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 2f1361ddfd4..e7c7e0849b5 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -220,12 +220,6 @@ public class TimeseriesQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 2408e090dc0..ab1faebf67e 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -368,12 +368,6 @@ public class TopNQueryQueryToolChest extends QueryToolChest> mergeSequences(Sequence>> seqOfSequences) { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 8d68738c22c..4f6f6a0f67f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -534,7 +534,7 @@ public class IncrementalIndex implements Iterable } } - return new MapBasedRow(gran.toDateTime(timeAndDims.getTimestamp()), theVals); + return new MapBasedRow(timeAndDims.getTimestamp(), theVals); } } ); diff --git a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java index e5b6fd4760f..07f99165873 100644 --- a/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectQueryRunnerTest.java @@ -59,7 +59,7 @@ public class SelectQueryRunnerTest { return QueryRunnerTestHelper.makeQueryRunners( new SelectQueryRunnerFactory( - new SelectQueryQueryToolChest(new SelectQueryConfig(), new DefaultObjectMapper()), + new SelectQueryQueryToolChest(new QueryConfig(), new DefaultObjectMapper()), new SelectQueryEngine(), QueryRunnerTestHelper.NOOP_QUERYWATCHER ) diff --git a/server/src/main/java/io/druid/guice/QueryToolChestModule.java b/server/src/main/java/io/druid/guice/QueryToolChestModule.java index 0b06eddc015..ed5b59591a9 100644 --- a/server/src/main/java/io/druid/guice/QueryToolChestModule.java +++ b/server/src/main/java/io/druid/guice/QueryToolChestModule.java @@ -35,7 +35,6 @@ import io.druid.query.search.SearchQueryQueryToolChest; import io.druid.query.search.search.SearchQuery; import io.druid.query.search.search.SearchQueryConfig; import io.druid.query.select.SelectQuery; -import io.druid.query.select.SelectQueryConfig; import io.druid.query.select.SelectQueryQueryToolChest; import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.query.timeboundary.TimeBoundaryQueryQueryToolChest; @@ -75,7 +74,6 @@ public class QueryToolChestModule implements Module JsonConfigProvider.bind(binder, "druid.query", QueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.groupBy", GroupByQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.search", SearchQueryConfig.class); - JsonConfigProvider.bind(binder, "druid.query.select", SelectQueryConfig.class); JsonConfigProvider.bind(binder, "druid.query.topN", TopNQueryConfig.class); } } diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 4acd5128af4..40036ea0e7a 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -32,7 +32,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.metamx.common.ISE; -import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.MergeIterable; import com.metamx.common.guava.Sequence; @@ -84,7 +83,6 @@ import io.druid.query.search.search.SearchQueryConfig; import io.druid.query.select.EventHolder; import io.druid.query.select.PagingSpec; import io.druid.query.select.SelectQuery; -import io.druid.query.select.SelectQueryConfig; import io.druid.query.select.SelectQueryQueryToolChest; import io.druid.query.select.SelectResultValue; import io.druid.query.spec.MultipleIntervalSegmentSpec; @@ -779,7 +777,7 @@ public class CachingClusteredClientTest QueryRunner runner = new FinalizeResultsQueryRunner( client, new SelectQueryQueryToolChest( - new SelectQueryConfig(), + new QueryConfig(), jsonMapper ) ); @@ -1620,13 +1618,7 @@ public class CachingClusteredClientTest int index = 0; while (index < objects.length) { DateTime timestamp = (DateTime) objects[index++]; - - //List values = Lists.newArrayList(); - //while (index < objects.length && !(objects[index] instanceof DateTime)) { retVal.add(new MapBasedRow(timestamp, (Map) objects[index++])); - //} - - //retVal.add(new Result<>(timestamp, values)); } return retVal; } @@ -1663,7 +1655,7 @@ public class CachingClusteredClientTest .put(SearchQuery.class, new SearchQueryQueryToolChest(new SearchQueryConfig())) .put( SelectQuery.class, - new SelectQueryQueryToolChest(new SelectQueryConfig(), jsonMapper) + new SelectQueryQueryToolChest(new QueryConfig(), jsonMapper) ) .put( GroupByQuery.class, From bf71d079f373a5f8761acb186472f89259612b5d Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 16 Jul 2014 15:20:20 -0700 Subject: [PATCH 154/270] fix #636 --- services/src/main/java/io/druid/cli/CliPeon.java | 2 +- services/src/main/java/io/druid/guice/RealtimeModule.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index d4eb0228c9a..d85e974127d 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -111,7 +111,7 @@ public class CliPeon extends GuiceRunnable binder, "druid.indexer.task.chathandler.type", Key.get(ChatHandlerProvider.class), - Key.get(NoopChatHandlerProvider.class) + Key.get(ServiceAnnouncingChatHandlerProvider.class) ); final MapBinder handlerProviderBinder = PolyBind.optionBinder( binder, Key.get(ChatHandlerProvider.class) diff --git a/services/src/main/java/io/druid/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index 7bcce15823a..6770b650069 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -65,7 +65,7 @@ public class RealtimeModule implements Module binder, "druid.realtime.chathandler.type", Key.get(ChatHandlerProvider.class), - Key.get(NoopChatHandlerProvider.class) + Key.get(ServiceAnnouncingChatHandlerProvider.class) ); final MapBinder handlerProviderBinder = PolyBind.optionBinder( binder, Key.get(ChatHandlerProvider.class) From 831d4e95152ac00bc33e515a343eefd2db6dd1dc Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 16 Jul 2014 15:38:30 -0700 Subject: [PATCH 155/270] update README --- README | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/README b/README index c976deda70b..2f3003db430 100644 --- a/README +++ b/README @@ -1,7 +1,7 @@ -The best place for more Druid resources is at: http://www.druid.io +What is Druid? http://www.druid.io Looking for docs? http://druid.io/docs/latest/ -Build with build.sh - Want to get started? http://druid.io/docs/latest/Tutorial:-A-First-Look-at-Druid.html + +Questions about setting up Druid? https://groups.google.com/forum/#!forum/druid-development From e59c9ebdbc96f6debbe6edccb7275cd23b4acb51 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 17 Jul 2014 17:24:57 +0530 Subject: [PATCH 156/270] minor fixes fix IndexOutOfBoundsException fix ingestFirehose --- .../io/druid/indexing/common/task/Task.java | 4 +- .../IngestSegmentFirehoseFactory.java | 35 ++---------- .../druid/indexing/firehose/IngestTask.java | 55 +++++++++++++++++++ .../segment/QueryableIndexStorageAdapter.java | 2 +- 4 files changed, 64 insertions(+), 32 deletions(-) create mode 100644 indexing-service/src/main/java/io/druid/indexing/firehose/IngestTask.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index f9395165f27..f83453fafe9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.firehose.IngestTask; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -53,7 +54,8 @@ import io.druid.query.QueryRunner; @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), @JsonSubTypes.Type(name = "noop", value = NoopTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), - @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class) + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class), + @JsonSubTypes.Type(name = "ingest-task", value = IngestTask.class) }) public interface Task { diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 85f1c0b4c59..1358d3d48c3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -153,12 +153,14 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory timeline = new VersionedIntervalTimeline( Ordering.natural().nullsFirst() ); - final List> timeLineSegments = timeline.lookup( - interval - ); + for (DataSegment segment : usedSegments) { timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); } + final List> timeLineSegments = timeline.lookup( + interval + ); + List dims; if (dimensions != null) { dims = dimensions; @@ -223,33 +225,6 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory rowYielder; diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestTask.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestTask.java new file mode 100644 index 00000000000..a2b37c6af92 --- /dev/null +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestTask.java @@ -0,0 +1,55 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013, 2014 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 io.druid.indexing.firehose; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TaskToolbox; +import io.druid.indexing.common.actions.TaskActionClient; +import io.druid.indexing.common.task.AbstractTask; + +public class IngestTask extends AbstractTask +{ + public IngestTask( + @JsonProperty("id") final String id, + @JsonProperty("dataSource") final String dataSource + ) + { + super(id, dataSource); + } + + @Override + public String getType() + { + return "Ingest-Task"; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return true; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + return TaskStatus.success(getId()); + } +} \ No newline at end of file diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 137024c61f4..7eb83b70f02 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -963,7 +963,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter if (multiValueRow.size() == 0) { return null; } else if (multiValueRow.size() == 1) { - return columnVals.lookupName(multiValueRow.get(1)); + return columnVals.lookupName(multiValueRow.get(0)); } else { final String[] strings = new String[multiValueRow.size()]; for (int i = 0 ; i < multiValueRow.size() ; i++) { From 5197ea527ac096a0380ab6f759f05eab7bc677db Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 12:35:45 -0700 Subject: [PATCH 157/270] disable middlemanagers based on worker version --- .../worker/WorkerCuratorCoordinator.java | 23 ++- .../indexing/worker/http/WorkerResource.java | 70 ++++++++- .../worker/http/WorkerResourceTest.java | 135 ++++++++++++++++++ 3 files changed, 226 insertions(+), 2 deletions(-) create mode 100644 indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index 6669556580b..feda8e81a90 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -129,7 +129,11 @@ public class WorkerCuratorCoordinator try { byte[] rawBytes = jsonMapper.writeValueAsBytes(data); if (rawBytes.length > config.getMaxZnodeBytes()) { - throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxZnodeBytes()); + throw new ISE( + "Length of raw bytes for task too large[%,d > %,d]", + rawBytes.length, + config.getMaxZnodeBytes() + ); } curatorFramework.create() @@ -173,6 +177,11 @@ public class WorkerCuratorCoordinator return getPath(Arrays.asList(baseStatusPath, statusId)); } + public Worker getWorker() + { + return worker; + } + public void unannounceTask(String taskId) { try { @@ -239,4 +248,16 @@ public class WorkerCuratorCoordinator } } } + + public void updateWorkerAnnouncement(Worker newWorker) throws Exception + { + synchronized (lock) { + if (!started) { + log.error("Cannot update worker! Not Started!"); + return; + } + + announcer.update(getAnnouncementsPathForWorker(), jsonMapper.writeValueAsBytes(newWorker)); + } + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java index f38acb23982..b70aa64c111 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java @@ -19,12 +19,18 @@ package io.druid.indexing.worker.http; +import com.google.api.client.util.Lists; +import com.google.common.base.Function; import com.google.common.base.Optional; +import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableMap; import com.google.common.io.InputSupplier; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import io.druid.indexing.overlord.ForkingTaskRunner; +import io.druid.indexing.overlord.TaskRunnerWorkItem; +import io.druid.indexing.worker.Worker; +import io.druid.indexing.worker.WorkerCuratorCoordinator; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; @@ -43,17 +49,78 @@ public class WorkerResource { private static final Logger log = new Logger(WorkerResource.class); + private final WorkerCuratorCoordinator curatorCoordinator; private final ForkingTaskRunner taskRunner; @Inject public WorkerResource( + WorkerCuratorCoordinator curatorCoordinator, ForkingTaskRunner taskRunner ) throws Exception { + this.curatorCoordinator = curatorCoordinator; this.taskRunner = taskRunner; } + @POST + @Path("/disable") + @Produces("application/json") + public Response doDisable() + { + final Worker worker = curatorCoordinator.getWorker(); + final Worker newWorker = new Worker(worker.getHost(), worker.getIp(), worker.getCapacity(), ""); + try { + curatorCoordinator.updateWorkerAnnouncement(newWorker); + return Response.ok(ImmutableMap.of(worker.getHost(), "disabled")).build(); + } + catch (Exception e) { + return Response.serverError().build(); + } + } + + @POST + @Path("/enable") + @Produces("application/json") + public Response doEnable() + { + final Worker worker = curatorCoordinator.getWorker(); + try { + curatorCoordinator.updateWorkerAnnouncement(worker); + return Response.ok(ImmutableMap.of(worker.getHost(), "enabled")).build(); + } + catch (Exception e) { + return Response.serverError().build(); + } + } + + @GET + @Path("/tasks") + @Produces("application/json") + public Response getTasks() + { + try { + return Response.ok( + Lists.newArrayList( + Collections2.transform( + taskRunner.getKnownTasks(), + new Function() + { + @Override + public String apply(TaskRunnerWorkItem input) + { + return input.getTaskId(); + } + } + ) + ) + ).build(); + } + catch (Exception e) { + return Response.serverError().build(); + } + } + @POST @Path("/task/{taskid}/shutdown") @Produces("application/json") @@ -82,7 +149,8 @@ public class WorkerResource if (stream.isPresent()) { try { return Response.ok(stream.get().getInput()).build(); - } catch (Exception e) { + } + catch (Exception e) { log.warn(e, "Failed to read log for task: %s", taskid); return Response.serverError().build(); } diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java new file mode 100644 index 00000000000..5e80a523b56 --- /dev/null +++ b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java @@ -0,0 +1,135 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.indexing.worker.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.curator.PotentiallyGzippedCompressionProvider; +import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig; +import io.druid.indexing.worker.Worker; +import io.druid.indexing.worker.WorkerCuratorCoordinator; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.server.initialization.ZkPathsConfig; +import junit.framework.Assert; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.test.TestingCluster; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import javax.ws.rs.core.Response; + +/** + */ +public class WorkerResourceTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + private static final String basePath = "/test/druid"; + private static final String announcementsPath = String.format("%s/indexer/announcements/host", basePath); + + private TestingCluster testingCluster; + private CuratorFramework cf; + + private Worker worker; + + private WorkerCuratorCoordinator curatorCoordinator; + private WorkerResource workerResource; + + @Before + public void setUp() throws Exception + { + testingCluster = new TestingCluster(1); + testingCluster.start(); + + cf = CuratorFrameworkFactory.builder() + .connectString(testingCluster.getConnectString()) + .retryPolicy(new ExponentialBackoffRetry(1, 10)) + .compressionProvider(new PotentiallyGzippedCompressionProvider(false)) + .build(); + cf.start(); + cf.create().creatingParentsIfNeeded().forPath(basePath); + + worker = new Worker( + "host", + "ip", + 3, + "v1" + ); + + curatorCoordinator = new WorkerCuratorCoordinator( + jsonMapper, + new ZkPathsConfig() + { + @Override + public String getZkBasePath() + { + return basePath; + } + }, + new RemoteTaskRunnerConfig(), + cf, + worker + ); + curatorCoordinator.start(); + + workerResource = new WorkerResource( + curatorCoordinator, + null + ); + } + + @After + public void tearDown() throws Exception + { + curatorCoordinator.stop(); + cf.close(); + testingCluster.close(); + } + + @Test + public void testDoDisable() throws Exception + { + Worker theWorker = jsonMapper.readValue(cf.getData().forPath(announcementsPath), Worker.class); + Assert.assertEquals("v1", theWorker.getVersion()); + + Response res = workerResource.doDisable(); + Assert.assertEquals(Response.Status.OK.getStatusCode(), res.getStatus()); + + theWorker = jsonMapper.readValue(cf.getData().forPath(announcementsPath), Worker.class); + Assert.assertTrue(theWorker.getVersion().isEmpty()); + } + + @Test + public void testDoEnable() throws Exception + { + // Disable the worker + Response res = workerResource.doDisable(); + Assert.assertEquals(Response.Status.OK.getStatusCode(), res.getStatus()); + Worker theWorker = jsonMapper.readValue(cf.getData().forPath(announcementsPath), Worker.class); + Assert.assertTrue(theWorker.getVersion().isEmpty()); + + // Enable the worker + res = workerResource.doEnable(); + Assert.assertEquals(Response.Status.OK.getStatusCode(), res.getStatus()); + theWorker = jsonMapper.readValue(cf.getData().forPath(announcementsPath), Worker.class); + Assert.assertEquals("v1", theWorker.getVersion()); + } +} From ba978d8b795914055ee0737acefe3f3fdb8d39e4 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 13:05:59 -0700 Subject: [PATCH 158/270] some minor cleanups to ingest firehose --- .../java/io/druid/indexing/common/task/Task.java | 4 ++-- .../firehose/IngestSegmentFirehoseFactory.java | 7 ++----- .../{IngestTask.java => ReIngestTask.java} | 6 +++--- .../firehose/CombiningFirehoseFactory.java | 15 ++++++++++++++- .../firehose/CombiningFirehoseFactoryTest.java | 11 +++++------ 5 files changed, 26 insertions(+), 17 deletions(-) rename indexing-service/src/main/java/io/druid/indexing/firehose/{IngestTask.java => ReIngestTask.java} (94%) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index f83453fafe9..9cf0caa2da8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.firehose.IngestTask; +import io.druid.indexing.firehose.ReIngestTask; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -55,7 +55,7 @@ import io.druid.query.QueryRunner; @JsonSubTypes.Type(name = "noop", value = NoopTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class), - @JsonSubTypes.Type(name = "ingest-task", value = IngestTask.class) + @JsonSubTypes.Type(name = "reingest", value = ReIngestTask.class) }) public interface Task { diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 1358d3d48c3..d1b5ccff7ec 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -42,12 +42,9 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; -import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.actions.SegmentListUsedAction; -import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.common.task.AbstractTask; import io.druid.query.filter.DimFilter; import io.druid.query.select.EventHolder; import io.druid.segment.Cursor; @@ -139,10 +136,10 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory { + private static final EmittingLogger log = new EmittingLogger(CombiningFirehoseFactory.class); + private final List delegateFactoryList; @JsonCreator @@ -86,10 +89,20 @@ public class CombiningFirehoseFactory implements FirehoseFactory if (currentFirehose != null) { currentFirehose.close(); } + currentFirehose = firehoseFactoryIterator.next().connect(parser); } catch (IOException e) { - Throwables.propagate(e); + if (currentFirehose != null) { + try { + currentFirehose.close(); + } + catch (IOException e2) { + log.error(e, "Unable to close currentFirehose!"); + throw Throwables.propagate(e2); + } + } + throw Throwables.propagate(e); } } } diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java index a69327b86be..ba46a2182eb 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -42,16 +42,15 @@ public class CombiningFirehoseFactoryTest public void testCombiningfirehose() throws IOException { List list1 = Arrays.asList(makeRow(1, 1), makeRow(2, 2)); - List list2 = Arrays.asList(makeRow(3, 3), makeRow(4, 4)); + List list2 = Arrays.asList(makeRow(3, 3), makeRow(4, 4), makeRow(5, 5)); FirehoseFactory combiningFactory = new CombiningFirehoseFactory( Arrays.asList( - new ListFirehoseFactory( - list1 - ), new ListFirehoseFactory(list2) + new ListFirehoseFactory(list1), + new ListFirehoseFactory(list2) ) ); final Firehose firehose = combiningFactory.connect(null); - for (int i = 1; i < 5; i++) { + for (int i = 1; i < 6; i++) { Assert.assertTrue(firehose.hasMore()); final InputRow inputRow = firehose.nextRow(); Assert.assertEquals(i, inputRow.getTimestampFromEpoch()); @@ -133,7 +132,7 @@ public class CombiningFirehoseFactoryTest @Override public void close() throws IOException { - // + // Do nothing } }; } From c6078ca841d3ebec4ccf2f7a09b4d9a00193ee58 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 13:34:05 -0700 Subject: [PATCH 159/270] address code review --- docs/content/Indexing-Service-Config.md | 3 +- .../indexing/overlord/RemoteTaskRunner.java | 3 +- .../SimpleResourceManagementStrategy.java | 4 +- .../overlord/setup/WorkerSetupData.java | 10 --- .../worker/WorkerCuratorCoordinator.java | 6 +- .../indexing/worker/http/WorkerResource.java | 33 +++++-- .../overlord/RemoteTaskRunnerTest.java | 2 +- .../scaling/EC2AutoScalingStrategyTest.java | 1 - .../SimpleResourceManagementStrategyTest.java | 87 +------------------ .../worker/http/WorkerResourceTest.java | 1 + 10 files changed, 38 insertions(+), 112 deletions(-) diff --git a/docs/content/Indexing-Service-Config.md b/docs/content/Indexing-Service-Config.md index 122a6623504..8916c5f5372 100644 --- a/docs/content/Indexing-Service-Config.md +++ b/docs/content/Indexing-Service-Config.md @@ -22,7 +22,7 @@ The following configs only apply if the overlord is running in remote mode: |Property|Description|Default| |--------|-----------|-------| |`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a middle manager before throwing an error.|PT5M| -|`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |none| +|`druid.indexer.runner.minWorkerVersion`|The minimum middle manager version to send tasks to. |"0"| |`druid.indexer.runner.compressZnodes`|Indicates whether or not the overlord should expect middle managers to compress Znodes.|false| |`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288| @@ -80,7 +80,6 @@ Issuing a GET request at the same URL will return the current worker setup spec |Property|Description|Default| |--------|-----------|-------| -|`minVersion`|The coordinator only assigns tasks to workers with a version greater than the minVersion. If this is not specified, the minVersion will be druid.indexer.runner.minWorkerVersion.|none| |`minNumWorkers`|The minimum number of workers that can be in the cluster at any given time.|0| |`maxNumWorkers`|The maximum number of workers that can be in the cluster at any given time.|0| |`nodeData`|A JSON object that describes how to launch new nodes. Currently, only EC2 is supported.|none; required| diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index c546fbcce06..7b0dab4d512 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -806,8 +806,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer } ); sortedWorkers.addAll(zkWorkers.values()); - final String workerSetupDataMinVer = workerSetupData.get() == null ? null : workerSetupData.get().getMinVersion(); - final String minWorkerVer = workerSetupDataMinVer == null ? config.getMinWorkerVersion() : workerSetupDataMinVer; + final String minWorkerVer = config.getMinWorkerVersion(); for (ZkWorker zkWorker : sortedWorkers) { if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) { diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java index 6d3dd904c5e..6f645f3aec0 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategy.java @@ -274,9 +274,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat @Override public boolean apply(ZkWorker zkWorker) { - final String minVersion = workerSetupData.getMinVersion() != null - ? workerSetupData.getMinVersion() - : config.getWorkerVersion(); + final String minVersion = config.getWorkerVersion(); if (minVersion == null) { throw new ISE("No minVersion found! It should be set in your runtime properties or configuration database."); } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java index e792f347aed..ab778622e3e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/setup/WorkerSetupData.java @@ -28,7 +28,6 @@ public class WorkerSetupData { public static final String CONFIG_KEY = "worker.setup"; - private final String minVersion; private final int minNumWorkers; private final int maxNumWorkers; private final String availabilityZone; @@ -37,7 +36,6 @@ public class WorkerSetupData @JsonCreator public WorkerSetupData( - @JsonProperty("minVersion") String minVersion, @JsonProperty("minNumWorkers") int minNumWorkers, @JsonProperty("maxNumWorkers") int maxNumWorkers, @JsonProperty("availabilityZone") String availabilityZone, @@ -45,7 +43,6 @@ public class WorkerSetupData @JsonProperty("userData") EC2UserData userData ) { - this.minVersion = minVersion; this.minNumWorkers = minNumWorkers; this.maxNumWorkers = maxNumWorkers; this.availabilityZone = availabilityZone; @@ -53,12 +50,6 @@ public class WorkerSetupData this.userData = userData; } - @JsonProperty - public String getMinVersion() - { - return minVersion; - } - @JsonProperty public int getMinNumWorkers() { @@ -93,7 +84,6 @@ public class WorkerSetupData public String toString() { return "WorkerSetupData{" + - "minVersion='" + minVersion + '\'' + ", minNumWorkers=" + minNumWorkers + ", maxNumWorkers=" + maxNumWorkers + ", availabilityZone=" + availabilityZone + diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java index feda8e81a90..32dbf5b85ff 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -51,13 +51,13 @@ public class WorkerCuratorCoordinator private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; private final CuratorFramework curatorFramework; - private final Worker worker; private final Announcer announcer; private final String baseAnnouncementsPath; private final String baseTaskPath; private final String baseStatusPath; + private volatile Worker worker; private volatile boolean started; @Inject @@ -253,10 +253,10 @@ public class WorkerCuratorCoordinator { synchronized (lock) { if (!started) { - log.error("Cannot update worker! Not Started!"); - return; + throw new ISE("Cannot update worker! Not Started!"); } + this.worker = newWorker; announcer.update(getAnnouncementsPathForWorker(), jsonMapper.writeValueAsBytes(newWorker)); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java index b70aa64c111..1084fe28168 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java @@ -48,31 +48,36 @@ import java.io.InputStream; public class WorkerResource { private static final Logger log = new Logger(WorkerResource.class); + private static String DISABLED_VERSION = ""; + private final Worker enabledWorker; + private final Worker disabledWorker; private final WorkerCuratorCoordinator curatorCoordinator; private final ForkingTaskRunner taskRunner; @Inject public WorkerResource( + Worker worker, WorkerCuratorCoordinator curatorCoordinator, ForkingTaskRunner taskRunner ) throws Exception { + this.enabledWorker = worker; + this.disabledWorker = new Worker(worker.getHost(), worker.getIp(), worker.getCapacity(), DISABLED_VERSION); this.curatorCoordinator = curatorCoordinator; this.taskRunner = taskRunner; } + @POST @Path("/disable") @Produces("application/json") public Response doDisable() { - final Worker worker = curatorCoordinator.getWorker(); - final Worker newWorker = new Worker(worker.getHost(), worker.getIp(), worker.getCapacity(), ""); try { - curatorCoordinator.updateWorkerAnnouncement(newWorker); - return Response.ok(ImmutableMap.of(worker.getHost(), "disabled")).build(); + curatorCoordinator.updateWorkerAnnouncement(disabledWorker); + return Response.ok(ImmutableMap.of(disabledWorker.getHost(), "disabled")).build(); } catch (Exception e) { return Response.serverError().build(); @@ -84,10 +89,24 @@ public class WorkerResource @Produces("application/json") public Response doEnable() { - final Worker worker = curatorCoordinator.getWorker(); try { - curatorCoordinator.updateWorkerAnnouncement(worker); - return Response.ok(ImmutableMap.of(worker.getHost(), "enabled")).build(); + curatorCoordinator.updateWorkerAnnouncement(enabledWorker); + return Response.ok(ImmutableMap.of(enabledWorker.getHost(), "enabled")).build(); + } + catch (Exception e) { + return Response.serverError().build(); + } + } + + @GET + @Path("/disabled") + @Produces("application/json") + public Response isEnabled() + { + try { + final Worker theWorker = curatorCoordinator.getWorker(); + final boolean disabled = theWorker.getVersion().equalsIgnoreCase(DISABLED_VERSION); + return Response.ok(ImmutableMap.of(theWorker.getHost(), disabled)).build(); } catch (Exception e) { return Response.serverError().build(); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index fcf9715fe62..26aa7077c56 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -380,7 +380,7 @@ public class RemoteTaskRunnerTest }, cf, new SimplePathChildrenCacheFactory.Builder().build(), - DSuppliers.of(new AtomicReference(new WorkerSetupData("0", 0, 1, null, null, null))), + DSuppliers.of(new AtomicReference(new WorkerSetupData(0, 1, null, null, null))), null ); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java index 1ccacc66df4..7be9a6eaad4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/EC2AutoScalingStrategyTest.java @@ -95,7 +95,6 @@ public class EC2AutoScalingStrategyTest { workerSetupData.set( new WorkerSetupData( - "0", 0, 1, "", diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 21f3277a653..6c13a0704c0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -67,7 +67,7 @@ public class SimpleResourceManagementStrategyTest autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); workerSetupData = new AtomicReference<>( new WorkerSetupData( - "0", 0, 2, null, null, null + 0, 2, null, null, null ) ); @@ -237,7 +237,7 @@ public class SimpleResourceManagementStrategyTest @Test public void testDoSuccessfulTerminate() throws Exception { - workerSetupData.set(new WorkerSetupData("0", 0, 1, null, null, null)); + workerSetupData.set(new WorkerSetupData(0, 1, null, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()); @@ -267,7 +267,7 @@ public class SimpleResourceManagementStrategyTest @Test public void testSomethingTerminating() throws Exception { - workerSetupData.set(new WorkerSetupData("0", 0, 1, null, null, null)); + workerSetupData.set(new WorkerSetupData(0, 1, null, null, null)); EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList("ip")).times(2); @@ -381,7 +381,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.verify(autoScalingStrategy); // Increase minNumWorkers - workerSetupData.set(new WorkerSetupData("0", 3, 5, null, null, null)); + workerSetupData.set(new WorkerSetupData(3, 5, null, null, null)); // Should provision two new workers EasyMock.reset(autoScalingStrategy); @@ -404,85 +404,6 @@ public class SimpleResourceManagementStrategyTest EasyMock.verify(autoScalingStrategy); } - @Test - public void testMinVersionIncrease() throws Exception - { - // Don't terminate anything - EasyMock.reset(autoScalingStrategy); - EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) - .andReturn(Lists.newArrayList("ip")); - EasyMock.replay(autoScalingStrategy); - boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList(), - Arrays.asList( - new TestZkWorker(NoopTask.create(), "h1", "i1", "0"), - new TestZkWorker(NoopTask.create(), "h1", "i2", "0") - ) - ); - Assert.assertFalse(terminatedSomething); - EasyMock.verify(autoScalingStrategy); - - // Don't provision anything - EasyMock.reset(autoScalingStrategy); - EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) - .andReturn(Lists.newArrayList("ip")); - EasyMock.replay(autoScalingStrategy); - boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList(), - Arrays.asList( - new TestZkWorker(NoopTask.create()), - new TestZkWorker(NoopTask.create()) - ) - ); - Assert.assertFalse(provisionedSomething); - EasyMock.verify(autoScalingStrategy); - - // Increase minVersion - workerSetupData.set(new WorkerSetupData("1", 0, 2, null, null, null)); - - // Provision two new workers - EasyMock.reset(autoScalingStrategy); - EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) - .andReturn(Lists.newArrayList("ip")); - EasyMock.expect(autoScalingStrategy.provision()).andReturn( - new AutoScalingData(Lists.newArrayList("h3")) - ); - EasyMock.expect(autoScalingStrategy.provision()).andReturn( - new AutoScalingData(Lists.newArrayList("h4")) - ); - EasyMock.replay(autoScalingStrategy); - provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList(), - Arrays.asList( - new TestZkWorker(NoopTask.create(), "h1", "i1", "0"), - new TestZkWorker(NoopTask.create(), "h2", "i2", "0") - ) - ); - Assert.assertTrue(provisionedSomething); - EasyMock.verify(autoScalingStrategy); - - // Terminate old workers - EasyMock.reset(autoScalingStrategy); - EasyMock.expect(autoScalingStrategy.ipToIdLookup(ImmutableList.of("i1", "i2", "i3", "i4"))).andReturn( - ImmutableList.of("h1", "h2", "h3", "h4") - ); - EasyMock.expect(autoScalingStrategy.terminate(ImmutableList.of("i1", "i2"))).andReturn( - new AutoScalingData(ImmutableList.of("h1", "h2")) - ); - EasyMock.replay(autoScalingStrategy); - terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList(), - Arrays.asList( - new TestZkWorker(null, "h1", "i1", "0"), - new TestZkWorker(null, "h2", "i2", "0"), - new TestZkWorker(NoopTask.create(), "h3", "i3", "1"), - new TestZkWorker(NoopTask.create(), "h4", "i4", "1") - ) - ); - Assert.assertTrue(terminatedSomething); - EasyMock.verify(autoScalingStrategy); - } - @Test public void testNullWorkerSetupData() throws Exception { diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java index 5e80a523b56..2a0a1e4a3c3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/http/WorkerResourceTest.java @@ -91,6 +91,7 @@ public class WorkerResourceTest curatorCoordinator.start(); workerResource = new WorkerResource( + worker, curatorCoordinator, null ); From ef325f15ff70a48ca5faa4436a4ddbfc490a8886 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 13:47:59 -0700 Subject: [PATCH 160/270] update pom and fix broken UT --- pom.xml | 2 +- server/src/main/java/io/druid/client/cache/CacheConfig.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 6f74d7d6d0f..7bde87c2e24 100644 --- a/pom.xml +++ b/pom.xml @@ -41,7 +41,7 @@ UTF-8 0.26.5 2.5.0 - 0.2.6-SNAPSHOT + 0.2.6 diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index f81c01de07e..26c5f383f2d 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -20,7 +20,6 @@ package io.druid.client.cache; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Sets; import io.druid.query.Query; import java.util.Arrays; @@ -52,6 +51,7 @@ public class CacheConfig public boolean isQueryCacheable(Query query) { - return !Sets.newHashSet(unCacheable).contains(query.getType()); + // O(n) impl, but I don't think we'll ever have a million query types here + return !unCacheable.contains(query.getType()); } } From bb5598df541ecdbbf28aa9fa917da137840ecd3e Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 14:07:12 -0700 Subject: [PATCH 161/270] fix doc --- docs/content/Tasks.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Tasks.md b/docs/content/Tasks.md index 845ac707fbc..868e75efe88 100644 --- a/docs/content/Tasks.md +++ b/docs/content/Tasks.md @@ -220,7 +220,7 @@ Kill tasks delete all information about a segment and removes it from deep stora "type": "kill", "id": , "dataSource": , - "segments": + "interval" : } ``` From 465ac77b824dd906200a839f77ea084ef06526e0 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 14:39:52 -0700 Subject: [PATCH 162/270] fix broken UT --- .../firehose/CombiningFirehoseFactoryTest.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java index a69327b86be..210386d2134 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -24,9 +24,11 @@ import com.metamx.common.parsers.ParseException; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.Row; import io.druid.data.input.impl.InputRowParser; import io.druid.segment.realtime.firehose.CombiningFirehoseFactory; import io.druid.utils.Runnables; +import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -76,6 +78,18 @@ public class CombiningFirehoseFactoryTest return timestamp; } + @Override + public DateTime getTimestamp() + { + return new DateTime(timestamp); + } + + @Override + public int compareTo(Row o) + { + return 0; + } + @Override public List getDimension(String dimension) { From d765ff5e75245d4628efc0d85b88fbd50ef6ccfe Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 14:53:26 -0700 Subject: [PATCH 163/270] docs for updating --- docs/content/Rolling-Updates.md | 44 +++++++++++++++++++++++++++++++++ docs/content/toc.textile | 1 + 2 files changed, 45 insertions(+) create mode 100644 docs/content/Rolling-Updates.md diff --git a/docs/content/Rolling-Updates.md b/docs/content/Rolling-Updates.md new file mode 100644 index 00000000000..a6d5cd005bd --- /dev/null +++ b/docs/content/Rolling-Updates.md @@ -0,0 +1,44 @@ +--- +layout: doc_page +--- + + +Rolling Updates +=============== + +For rolling Druid cluster updates with no downtime, we recommend updating Druid nodes in the following order: + +1. Historical Nodes +2. Indexing Service/Real-time Nodes +3. Broker Nodes +4. Coordinator Nodes + +## Historical Nodes + +Historical nodes can be updated one at a time. Each historical node has a startup time to memory map all the segments it was serving before the update. The startup time typically takes a few seconds to a few minutes, depending on the hardware of the node. As long as each historical node is updated with a sufficient delay (greater than the time required to start a single node), you can rolling update the entire historical cluster. + +## Standalone Real-time nodes + +Standalone real-time nodes can be updated one at a time in a rolling fashion. + +## Indexing Service + +### With Autoscaling + +Overlord nodes will try to launch new middle manager nodes and terminate old ones without dropping data. This process is based on the configuration `druid.indexer.runner.minWorkerVersion=#{VERSION}`. Each time you update your overlord node, the `VERSION` value should be increased. + +### Without Autoscaling + +Middle managers can be updated in a rolling fashion based on API. + +To prepare a middle manager for update, send a POST request to `/druid/worker/v1/disable`. The overlord will now no longer send tasks to this middle manager. + +Current tasks will still try to complete. To view all existing tasks, send a GET request to `/druid/worker/v1/tasks`. When this list is empty, the middle manager can be updated. After the middle manager is updated, it is automatically enabled again. You can also manually enable middle managers POSTing to `/druid/worker/v1/enable`. + +## Broker Nodes + +Broker nodes can be updated one at a time in a rolling fashion. There needs to be some delay between updating each node as brokers must load the entire state of the cluster before they return valid results. + +## Coordinator Nodes + +Coordinator nodes can be updated in a rolling fashion. \ No newline at end of file diff --git a/docs/content/toc.textile b/docs/content/toc.textile index a4769a6b2f5..76ba3e31133 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -17,6 +17,7 @@ h2. Getting Started h2. Booting a Druid Cluster * "Simple Cluster Configuration":Simple-Cluster-Configuration.html * "Production Cluster Configuration":Production-Cluster-Configuration.html +* "Rolling Cluster Updates":Rolling-Updates.html h2. Configuration * "Common Configuration":Configuration.html From 83cb7931ac531debec560d4e6e471a70ace4922e Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 14:59:56 -0700 Subject: [PATCH 164/270] add more stuff to docs --- docs/content/Rolling-Updates.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/content/Rolling-Updates.md b/docs/content/Rolling-Updates.md index a6d5cd005bd..6c8680de0cb 100644 --- a/docs/content/Rolling-Updates.md +++ b/docs/content/Rolling-Updates.md @@ -27,6 +27,8 @@ Standalone real-time nodes can be updated one at a time in a rolling fashion. Overlord nodes will try to launch new middle manager nodes and terminate old ones without dropping data. This process is based on the configuration `druid.indexer.runner.minWorkerVersion=#{VERSION}`. Each time you update your overlord node, the `VERSION` value should be increased. +The config `druid.indexer.autoscale.workerVersion=#{VERSION}` also needs to be set. + ### Without Autoscaling Middle managers can be updated in a rolling fashion based on API. From 95a09c04bfac6ec0f0fe419920cd2f9b70d84310 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 17 Jul 2014 16:22:30 -0700 Subject: [PATCH 165/270] log errors in asyncForwardingServlet --- .../main/java/io/druid/server/AsyncQueryForwardingServlet.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 3d82cc013ea..9888f0f8a30 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -373,6 +373,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet Throwable e ) { + log.error(e, "Error processing query response"); // throwing an exception here may cause resource leak try { handleException(objectMapper, asyncContext, e); From beac0be45b6c3893ace90020a7916ccc48caba0d Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 17 Jul 2014 18:04:36 -0700 Subject: [PATCH 166/270] fix enabled endpoint --- .../java/io/druid/indexing/worker/http/WorkerResource.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java index 1084fe28168..ddbbd7f0929 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/http/WorkerResource.java @@ -99,14 +99,14 @@ public class WorkerResource } @GET - @Path("/disabled") + @Path("/enabled") @Produces("application/json") public Response isEnabled() { try { final Worker theWorker = curatorCoordinator.getWorker(); - final boolean disabled = theWorker.getVersion().equalsIgnoreCase(DISABLED_VERSION); - return Response.ok(ImmutableMap.of(theWorker.getHost(), disabled)).build(); + final boolean enabled = !theWorker.getVersion().equalsIgnoreCase(DISABLED_VERSION); + return Response.ok(ImmutableMap.of(theWorker.getHost(), enabled)).build(); } catch (Exception e) { return Response.serverError().build(); From 0e0454a34cb52331e427e19c0754e7b5c7f47c3a Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 18 Jul 2014 06:50:58 +0530 Subject: [PATCH 167/270] switch reingest task to noop & fix compilation switch back to noop task, its confusing to have a reinvest task that does nothing. fix compilation --- .../io/druid/indexing/common/task/Task.java | 4 +- .../IngestSegmentFirehoseFactory.java | 4 +- .../druid/indexing/firehose/ReIngestTask.java | 55 ------------------- .../CombiningFirehoseFactoryTest.java | 13 +++++ 4 files changed, 17 insertions(+), 59 deletions(-) delete mode 100644 indexing-service/src/main/java/io/druid/indexing/firehose/ReIngestTask.java diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java index 9cf0caa2da8..f9395165f27 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/Task.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; -import io.druid.indexing.firehose.ReIngestTask; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -54,8 +53,7 @@ import io.druid.query.QueryRunner; @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), @JsonSubTypes.Type(name = "noop", value = NoopTask.class), @JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class), - @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class), - @JsonSubTypes.Type(name = "reingest", value = ReIngestTask.class) + @JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class) }) public interface Task { diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index d1b5ccff7ec..6987a24e97f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -45,6 +45,7 @@ import io.druid.granularity.QueryGranularity; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.actions.SegmentListUsedAction; +import io.druid.indexing.common.task.NoopTask; import io.druid.query.filter.DimFilter; import io.druid.query.select.EventHolder; import io.druid.segment.Cursor; @@ -138,8 +139,9 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory getDimension(String dimension) { @@ -93,6 +101,11 @@ public class CombiningFirehoseFactoryTest return null; } + @Override + public int compareTo(Row o) + { + return 0; + } }; } From 4ce12470a198bcebaee6db31cae93fed675c9e93 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 18 Jul 2014 18:52:15 +0530 Subject: [PATCH 168/270] Add way to skip determine partitions for index task Add a way to skip determinePartitions for IndexTask by manually specifying numShards. --- .../indexer/DetermineHashedPartitionsJob.java | 11 ++++- .../HadoopDruidDetermineConfigurationJob.java | 2 +- .../druid/indexing/common/task/IndexTask.java | 43 ++++++++++++++++--- .../indexing/common/task/TaskSerdeTest.java | 9 +++- .../indexing/overlord/TaskLifecycleTest.java | 7 ++- .../partition/HashBasedNumberedShardSpec.java | 8 ++-- .../shard/HashBasedNumberedShardSpecTest.java | 11 ++--- 7 files changed, 70 insertions(+), 21 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java index 4bc9cb9f35a..77466b8c0d9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/DetermineHashedPartitionsJob.java @@ -179,7 +179,16 @@ public class DetermineHashedPartitionsJob implements Jobby actualSpecs.add(new HadoopyShardSpec(new NoneShardSpec(), shardCount++)); } else { for (int i = 0; i < numberOfShards; ++i) { - actualSpecs.add(new HadoopyShardSpec(new HashBasedNumberedShardSpec(i, numberOfShards), shardCount++)); + actualSpecs.add( + new HadoopyShardSpec( + new HashBasedNumberedShardSpec( + i, + numberOfShards, + HadoopDruidIndexerConfig.jsonMapper + ), + shardCount++ + ) + ); log.info("DateTime[%s], partition[%d], spec[%s]", bucket, i, actualSpecs.get(i)); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java index 311eec6248e..51ba40abc41 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -67,7 +67,7 @@ public class HadoopDruidDetermineConfigurationJob implements Jobby for (int i = 0; i < shardsPerInterval; i++) { specs.add( new HadoopyShardSpec( - new HashBasedNumberedShardSpec(i, shardsPerInterval), + new HashBasedNumberedShardSpec(i, shardsPerInterval, HadoopDruidIndexerConfig.jsonMapper), shardCount++ ) ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 182a1569903..b7496c70782 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -19,10 +19,13 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -53,12 +56,14 @@ import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.FireDepartmentMetrics; import io.druid.segment.realtime.plumber.Plumber; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; @@ -107,6 +112,8 @@ public class IndexTask extends AbstractFixedIntervalTask @JsonIgnore private final IndexIngestionSpec ingestionSchema; + private final ObjectMapper jsonMapper; + @JsonCreator public IndexTask( @JsonProperty("id") String id, @@ -118,7 +125,8 @@ public class IndexTask extends AbstractFixedIntervalTask @JsonProperty("indexGranularity") final QueryGranularity indexGranularity, @JsonProperty("targetPartitionSize") final int targetPartitionSize, @JsonProperty("firehose") final FirehoseFactory firehoseFactory, - @JsonProperty("rowFlushBoundary") final int rowFlushBoundary + @JsonProperty("rowFlushBoundary") final int rowFlushBoundary, + @JacksonInject ObjectMapper jsonMapper ) { super( @@ -139,9 +147,10 @@ public class IndexTask extends AbstractFixedIntervalTask granularitySpec.withQueryGranularity(indexGranularity == null ? QueryGranularity.NONE : indexGranularity) ), new IndexIOConfig(firehoseFactory), - new IndexTuningConfig(targetPartitionSize, rowFlushBoundary) + new IndexTuningConfig(targetPartitionSize, rowFlushBoundary, null) ); } + this.jsonMapper = jsonMapper; } @Override @@ -175,7 +184,15 @@ public class IndexTask extends AbstractFixedIntervalTask if (targetPartitionSize > 0) { shardSpecs = determinePartitions(bucket, targetPartitionSize); } else { - shardSpecs = ImmutableList.of(new NoneShardSpec()); + int numShards = ingestionSchema.getTuningConfig().getNumShards(); + if (numShards > 0) { + shardSpecs = Lists.newArrayList(); + for (int i = 0; i < numShards; i++) { + shardSpecs.add(new HashBasedNumberedShardSpec(i, numShards, jsonMapper)); + } + } else { + shardSpecs = ImmutableList.of(new NoneShardSpec()); + } } for (final ShardSpec shardSpec : shardSpecs) { final DataSegment segment = generateSegment( @@ -206,6 +223,7 @@ public class IndexTask extends AbstractFixedIntervalTask retVal.add(interval); } } + return retVal; } @@ -477,7 +495,7 @@ public class IndexTask extends AbstractFixedIntervalTask this.dataSchema = dataSchema; this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0) : tuningConfig; + this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null) : tuningConfig; } @Override @@ -530,15 +548,22 @@ public class IndexTask extends AbstractFixedIntervalTask private final int targetPartitionSize; private final int rowFlushBoundary; + private final int numShards; @JsonCreator public IndexTuningConfig( @JsonProperty("targetPartitionSize") int targetPartitionSize, - @JsonProperty("rowFlushBoundary") int rowFlushBoundary - ) + @JsonProperty("rowFlushBoundary") int rowFlushBoundary, + @JsonProperty("numShards") @Nullable Integer numShards + ) { this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; this.rowFlushBoundary = rowFlushBoundary == 0 ? DEFAULT_ROW_FLUSH_BOUNDARY : rowFlushBoundary; + this.numShards = numShards == null ? -1 : numShards; + Preconditions.checkArgument( + this.targetPartitionSize == -1 || this.numShards == -1, + "targetPartitionsSize and shardCount both cannot be set" + ); } @JsonProperty @@ -552,5 +577,11 @@ public class IndexTask extends AbstractFixedIntervalTask { return rowFlushBoundary; } + + @JsonProperty + public int getNumShards() + { + return numShards; + } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 0cd0fde9326..516ea984301 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -19,6 +19,8 @@ package io.druid.indexing.common.task; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -67,16 +69,19 @@ public class TaskSerdeTest QueryGranularity.NONE, 10000, new LocalFirehoseFactory(new File("lol"), "rofl", null), - -1 + -1, + jsonMapper ); for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) { jsonMapper.registerModule(jacksonModule); } + InjectableValues inject = new InjectableValues.Std() + .addValue(ObjectMapper.class, jsonMapper); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change - final IndexTask task2 = (IndexTask) jsonMapper.readValue(json, Task.class); + final IndexTask task2 = jsonMapper.reader(Task.class).with(inject).readValue(json); Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval()); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 55c00a58d9c..33411036e11 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -43,6 +43,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; +import io.druid.indexing.common.TestUtils; import io.druid.segment.column.ColumnConfig; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.indexing.common.SegmentLoaderFactory; @@ -249,7 +250,8 @@ public class TaskLifecycleTest IR("2010-01-02T01", "a", "c", 1) ) ), - -1 + -1, + TestUtils.MAPPER ); final Optional preRunTaskStatus = tsqa.getStatus(indexTask.getId()); @@ -297,7 +299,8 @@ public class TaskLifecycleTest QueryGranularity.NONE, 10000, newMockExceptionalFirehoseFactory(), - -1 + -1, + TestUtils.MAPPER ); final TaskStatus status = runTask(indexTask); diff --git a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java index 8f347ee6cfd..be640a03545 100644 --- a/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/server/src/main/java/io/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -34,18 +34,18 @@ import java.util.List; public class HashBasedNumberedShardSpec extends NumberedShardSpec { - private static final HashFunction hashFunction = Hashing.murmur3_32(); - @JacksonInject - private ObjectMapper jsonMapper; + private final ObjectMapper jsonMapper; @JsonCreator public HashBasedNumberedShardSpec( @JsonProperty("partitionNum") int partitionNum, - @JsonProperty("partitions") int partitions + @JsonProperty("partitions") int partitions, + @JacksonInject ObjectMapper jsonMapper ) { super(partitionNum, partitions); + this.jsonMapper = jsonMapper; } @Override diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index afab880ff81..97b9fefc302 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -26,6 +26,7 @@ import com.metamx.common.ISE; import io.druid.TestUtil; import io.druid.data.input.InputRow; import io.druid.data.input.Row; +import io.druid.jackson.DefaultObjectMapper; import io.druid.timeline.partition.HashBasedNumberedShardSpec; import io.druid.timeline.partition.PartitionChunk; import io.druid.timeline.partition.ShardSpec; @@ -43,7 +44,7 @@ public class HashBasedNumberedShardSpecTest { final ShardSpec spec = TestUtil.MAPPER.readValue( - TestUtil.MAPPER.writeValueAsBytes(new HashBasedNumberedShardSpec(1, 2)), + TestUtil.MAPPER.writeValueAsBytes(new HashBasedNumberedShardSpec(1, 2, TestUtil.MAPPER)), ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); @@ -65,9 +66,9 @@ public class HashBasedNumberedShardSpecTest public void testPartitionChunks() { final List specs = ImmutableList.of( - new HashBasedNumberedShardSpec(0, 3), - new HashBasedNumberedShardSpec(1, 3), - new HashBasedNumberedShardSpec(2, 3) + new HashBasedNumberedShardSpec(0, 3, TestUtil.MAPPER), + new HashBasedNumberedShardSpec(1, 3, TestUtil.MAPPER), + new HashBasedNumberedShardSpec(2, 3, TestUtil.MAPPER) ); final List> chunks = Lists.transform( @@ -141,7 +142,7 @@ public class HashBasedNumberedShardSpecTest int partitions ) { - super(partitionNum, partitions); + super(partitionNum, partitions, TestUtil.MAPPER); } @Override From 9cec7107a9fa09b42ebc992567ce52efde77ea85 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Fri, 18 Jul 2014 09:50:15 -0700 Subject: [PATCH 169/270] Properties.load instead of manual population --- .../storage/s3/FileSessionCredentialsProvider.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java index dbb1a288407..cf6efed0e3d 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/FileSessionCredentialsProvider.java @@ -25,7 +25,10 @@ import com.amazonaws.auth.AWSSessionCredentials; import com.google.common.base.Charsets; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStream; import java.nio.file.Files; import java.nio.file.Paths; import java.util.List; @@ -80,12 +83,10 @@ public class FileSessionCredentialsProvider implements AWSCredentialsProvider { @Override public void refresh() { try { - List lines = Files.readAllLines(Paths.get(sessionCredentials), Charsets.UTF_8); Properties props = new Properties(); - for (String line : lines) { - String[] tokens = line.split("="); - props.put(tokens[0], tokens[1]); - } + InputStream is = new FileInputStream(new File(sessionCredentials)); + props.load(is); + is.close(); sessionToken = props.getProperty("sessionToken"); accessKey = props.getProperty("accessKey"); From 439ff9d5e2ed020c7847dbb0d6f797fdb63c6946 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 18 Jul 2014 23:40:38 +0530 Subject: [PATCH 170/270] upgrade to jetty 9.2.1 Upgrade jetty version - supports Java8 - Async I/O Proxying - other jetty bug fixes --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 7bde87c2e24..ab6838b51b8 100644 --- a/pom.xml +++ b/pom.xml @@ -324,17 +324,17 @@ org.eclipse.jetty jetty-server - 9.1.5.v20140505 + 9.2.1.v20140609 org.eclipse.jetty jetty-servlet - 9.1.5.v20140505 + 9.2.1.v20140609 org.eclipse.jetty jetty-servlets - 9.1.5.v20140505 + 9.2.1.v20140609 joda-time From ea0402277225b73dee41989b0ba0f8e8c242799e Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 18 Jul 2014 11:17:52 -0700 Subject: [PATCH 171/270] [maven-release-plugin] prepare release druid-0.6.129 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 5 ++--- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 17 insertions(+), 18 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index d129375e3b9..b6d39d672b1 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/common/pom.xml b/common/pom.xml index 7dc9cfa1e38..6c47fa04936 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/examples/pom.xml b/examples/pom.xml index 5789834bac3..54d7d8c6ba4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 7d8308c04b7..5e79a408e4c 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/histogram/pom.xml b/histogram/pom.xml index f3d9c31f617..de9decb9f6b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -17,8 +17,7 @@ ~ along with this program; if not, write to the Free Software ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid.extensions druid-histogram @@ -28,7 +27,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 7d2a9c8de06..2f0f41c2f4f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index aa45a4e2b81..de9bdf1bc34 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 05393555a35..a62ef3ca22a 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 014d7af0304..87201af4d5a 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/pom.xml b/pom.xml index ab6838b51b8..4b9d03da3fd 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.129-SNAPSHOT + 0.6.129 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.117-SNAPSHOT + druid-0.6.129 diff --git a/processing/pom.xml b/processing/pom.xml index 755d2f553f7..50b46dde13a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 7ed54d2833d..f6fe35c2552 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1af0b941bdd..5067fdeea4f 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/server/pom.xml b/server/pom.xml index 8fa5e300885..ffd7a5ab06a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 diff --git a/services/pom.xml b/services/pom.xml index f29c5d681cc..66c717049c5 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.129-SNAPSHOT + 0.6.129 From 9e303527f2230f663034aac7a0321c0bda38575c Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 18 Jul 2014 11:17:56 -0700 Subject: [PATCH 172/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index b6d39d672b1..4ea4328b36f 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 6c47fa04936..e481d27fd3b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 54d7d8c6ba4..21a596936b5 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 5e79a408e4c..d4eb205f7ba 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index de9decb9f6b..b2916be6f96 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 2f0f41c2f4f..4f56fa245cc 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index de9bdf1bc34..6f2695fd40a 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index a62ef3ca22a..1d7335f8959 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 87201af4d5a..1fa031143f2 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/pom.xml b/pom.xml index 4b9d03da3fd..ee692516b3d 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.129 + 0.6.130-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.129 + druid-0.6.117-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 50b46dde13a..6f1e179fda2 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index f6fe35c2552..d61f2b48f38 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 5067fdeea4f..161440485a3 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ffd7a5ab06a..21bbfbefb96 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 66c717049c5..790d72a30fe 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.129 + 0.6.130-SNAPSHOT From 09fcfc3b6dd514e70f2e002782e3a6c2d7b49ea6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 18 Jul 2014 11:37:53 -0700 Subject: [PATCH 173/270] Fix race in RemoteTaskRunner that could lead to zombie tasks. --- .../indexing/overlord/RemoteTaskRunner.java | 36 +++++++++---------- .../overlord/RemoteTaskRunnerWorkItem.java | 20 ++++++++++- .../SimpleResourceManagementStrategyTest.java | 28 +++++++-------- 3 files changed, 51 insertions(+), 33 deletions(-) diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index 7b0dab4d512..89cc2ce9d73 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -422,11 +422,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer private RemoteTaskRunnerWorkItem addPendingTask(final Task task) { log.info("Added pending task %s", task.getId()); - final RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem( - task.getId(), - SettableFuture.create(), - null - ); + final RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem(task.getId(), null); pendingTaskPayloads.put(task.getId(), task); pendingTasks.put(task.getId(), taskRunnerWorkItem); runPendingTasks(); @@ -663,17 +659,24 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer if ((tmp = runningTasks.get(taskId)) != null) { taskRunnerWorkItem = tmp; } else { - log.warn( - "Worker[%s] announced a status for a task I didn't know about, adding to runningTasks: %s", - zkWorker.getWorker().getHost(), - taskId - ); - taskRunnerWorkItem = new RemoteTaskRunnerWorkItem( + final RemoteTaskRunnerWorkItem newTaskRunnerWorkItem = new RemoteTaskRunnerWorkItem( taskId, - SettableFuture.create(), zkWorker.getWorker() ); - runningTasks.put(taskId, taskRunnerWorkItem.withWorker(zkWorker.getWorker())); + final RemoteTaskRunnerWorkItem existingItem = runningTasks.putIfAbsent( + taskId, + newTaskRunnerWorkItem + ); + if (existingItem == null) { + log.warn( + "Worker[%s] announced a status for a task I didn't know about, adding to runningTasks: %s", + zkWorker.getWorker().getHost(), + taskId + ); + taskRunnerWorkItem = newTaskRunnerWorkItem; + } else { + taskRunnerWorkItem = existingItem; + } } if (taskStatus.isComplete()) { @@ -835,7 +838,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer // Worker is done with this task zkWorker.setLastCompletedTaskTime(new DateTime()); } else { - log.info("No worker run task[%s] with status[%s]", taskStatus.getId(), taskStatus.getStatusCode()); + log.info("Workerless task[%s] completed with status[%s]", taskStatus.getId(), taskStatus.getStatusCode()); } // Move from running -> complete @@ -843,9 +846,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogStreamer runningTasks.remove(taskStatus.getId()); // Notify interested parties - final ListenableFuture result = taskRunnerWorkItem.getResult(); - if (result != null) { - ((SettableFuture) result).set(taskStatus); - } + taskRunnerWorkItem.setResult(taskStatus); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java index 76d373a049a..86b3e583c26 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java @@ -32,6 +32,24 @@ public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem private final Worker worker; public RemoteTaskRunnerWorkItem( + String taskId, + Worker worker + ) + { + this(taskId, SettableFuture.create(), worker); + } + + public RemoteTaskRunnerWorkItem( + String taskId, + DateTime createdTime, + DateTime queueInsertionTime, + Worker worker + ) + { + this(taskId, SettableFuture.create(), createdTime, queueInsertionTime, worker); + } + + private RemoteTaskRunnerWorkItem( String taskId, SettableFuture result, Worker worker @@ -42,7 +60,7 @@ public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem this.worker = worker; } - public RemoteTaskRunnerWorkItem( + private RemoteTaskRunnerWorkItem( String taskId, SettableFuture result, DateTime createdTime, diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java index 6c13a0704c0..86021525496 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/scaling/SimpleResourceManagementStrategyTest.java @@ -27,8 +27,8 @@ import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.common.guava.DSuppliers; -import io.druid.indexing.common.TestMergeTask; import io.druid.indexing.common.TaskStatus; +import io.druid.indexing.common.TestMergeTask; import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.Task; import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; @@ -113,7 +113,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -141,7 +141,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -157,7 +157,7 @@ public class SimpleResourceManagementStrategyTest provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -188,7 +188,7 @@ public class SimpleResourceManagementStrategyTest EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) .andReturn(Lists.newArrayList()).times(2); EasyMock.expect(autoScalingStrategy.terminateWithIds(EasyMock.>anyObject())) - .andReturn(null); + .andReturn(null); EasyMock.expect(autoScalingStrategy.provision()).andReturn( new AutoScalingData(Lists.newArrayList("fake")) ); @@ -196,7 +196,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -214,7 +214,7 @@ public class SimpleResourceManagementStrategyTest provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -248,7 +248,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -278,7 +278,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -293,7 +293,7 @@ public class SimpleResourceManagementStrategyTest terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -319,7 +319,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(NoopTask.create()), @@ -337,7 +337,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(NoopTask.create()), @@ -412,7 +412,7 @@ public class SimpleResourceManagementStrategyTest boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -421,7 +421,7 @@ public class SimpleResourceManagementStrategyTest boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( Arrays.asList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(new DateTime()) + new RemoteTaskRunnerWorkItem(testTask.getId(), null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) From 39bff5c1c23303284248ec415170ce385072bfd6 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 18 Jul 2014 11:55:46 -0700 Subject: [PATCH 174/270] [maven-release-plugin] prepare release druid-0.6.130 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 4ea4328b36f..8f57ffa7542 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/common/pom.xml b/common/pom.xml index e481d27fd3b..1267ad2628b 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/examples/pom.xml b/examples/pom.xml index 21a596936b5..fe83501c391 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index d4eb205f7ba..a1b156c6e9a 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/histogram/pom.xml b/histogram/pom.xml index b2916be6f96..9e0418ce734 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4f56fa245cc..4ca4c73984e 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 6f2695fd40a..e5a040ee100 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 1d7335f8959..6a89d758014 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 1fa031143f2..81a7d12cb08 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/pom.xml b/pom.xml index ee692516b3d..1871a1441f9 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.130-SNAPSHOT + 0.6.130 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.117-SNAPSHOT + druid-0.6.130 diff --git a/processing/pom.xml b/processing/pom.xml index 6f1e179fda2..6ba9ef61b0e 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index d61f2b48f38..c0c09d9b5cf 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 161440485a3..a6108be269e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/server/pom.xml b/server/pom.xml index 21bbfbefb96..858c982de5c 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 diff --git a/services/pom.xml b/services/pom.xml index 790d72a30fe..609bbcda59f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.130-SNAPSHOT + 0.6.130 From 1de0f897185e863a211c4eb6e9426c6eb237a59e Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 18 Jul 2014 11:55:50 -0700 Subject: [PATCH 175/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 8f57ffa7542..74e85d6107c 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 1267ad2628b..9a198ed9cb3 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index fe83501c391..0f5c345c3e4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index a1b156c6e9a..cb9dd2096a6 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 9e0418ce734..25b7979ae0d 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4ca4c73984e..47310f2d66f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index e5a040ee100..b8325b3b1db 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 6a89d758014..955d942fa41 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 81a7d12cb08..b24b37598ee 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/pom.xml b/pom.xml index 1871a1441f9..4928f152ff7 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.130 + 0.6.131-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.130 + druid-0.6.117-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 6ba9ef61b0e..c90dd961c25 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index c0c09d9b5cf..aea2a950435 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index a6108be269e..6421a24401e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 858c982de5c..15b9b76df68 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 609bbcda59f..af3501d21f4 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.130 + 0.6.131-SNAPSHOT From a1ea56ad121e2cc672fd95ffd805322d72ff7a0c Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 21 Jul 2014 09:58:04 -0700 Subject: [PATCH 176/270] update java util and druid-api to fix guava/hadoop problems --- pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 4928f152ff7..cfd8a332b51 100644 --- a/pom.xml +++ b/pom.xml @@ -30,7 +30,7 @@ scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.117-SNAPSHOT + druid-0.6.131-SNAPSHOT @@ -39,9 +39,9 @@ UTF-8 - 0.26.5 + 0.26.6 2.5.0 - 0.2.6 + 0.2.7 From 8f2fc595ee367efc7561644a49e04e1ac59100ba Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 22 Jul 2014 10:12:02 -0700 Subject: [PATCH 177/270] allow router to override selection based on config --- .../PriorityTieredBrokerSelectorStrategy.java | 46 +++++++++++++++ .../server/router/TieredBrokerConfig.java | 9 +++ .../router/TieredBrokerHostSelector.java | 19 +++--- ...ieredBrokerSelectorStrategiesProvider.java | 59 +++++++++++++++++++ .../router/TieredBrokerSelectorStrategy.java | 19 ++++++ ...eBoundaryTieredBrokerSelectorStrategy.java | 46 +++++++++++++++ .../router/TieredBrokerHostSelectorTest.java | 27 ++++++++- .../src/main/java/io/druid/cli/CliRouter.java | 7 +++ 8 files changed, 223 insertions(+), 9 deletions(-) create mode 100644 server/src/main/java/io/druid/server/router/PriorityTieredBrokerSelectorStrategy.java create mode 100644 server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategiesProvider.java create mode 100644 server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java create mode 100644 server/src/main/java/io/druid/server/router/TimeBoundaryTieredBrokerSelectorStrategy.java diff --git a/server/src/main/java/io/druid/server/router/PriorityTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/PriorityTieredBrokerSelectorStrategy.java new file mode 100644 index 00000000000..4fbcd78a602 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/PriorityTieredBrokerSelectorStrategy.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.router; + +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import io.druid.query.Query; + +/** + */ +public class PriorityTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy +{ + @Override + public Optional getBrokerServiceName(TieredBrokerConfig tierConfig, Query query) + { + final int priority = query.getContextPriority(0); + + if (priority < 0) { + return Optional.of( + Iterables.getFirst( + tierConfig.getTierToBrokerMap().values(), + tierConfig.getDefaultBrokerServiceName() + ) + ); + } + + return Optional.absent(); + } +} diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java b/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java index 67ff109f217..395dd81e2e6 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerConfig.java @@ -54,6 +54,10 @@ public class TieredBrokerConfig @NotNull private Period pollPeriod = new Period("PT1M"); + @JsonProperty + @NotNull + private String strategies = "[{\"type\":\"timeBoundary\"},{\"type\":\"priority\"}]"; + // tier, public LinkedHashMap getTierToBrokerMap() { @@ -88,4 +92,9 @@ public class TieredBrokerConfig { return pollPeriod; } + + public String getStrategies() + { + return strategies; + } } diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java index 8ebe2e55050..a28e6ffc2ff 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerHostSelector.java @@ -19,6 +19,7 @@ package io.druid.server.router; +import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.inject.Inject; @@ -30,7 +31,6 @@ import io.druid.client.selector.HostSelector; import io.druid.curator.discovery.ServerDiscoveryFactory; import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.query.Query; -import io.druid.query.timeboundary.TimeBoundaryQuery; import io.druid.server.coordinator.rules.LoadRule; import io.druid.server.coordinator.rules.Rule; import org.joda.time.DateTime; @@ -50,6 +50,7 @@ public class TieredBrokerHostSelector implements HostSelector private final TieredBrokerConfig tierConfig; private final ServerDiscoveryFactory serverDiscoveryFactory; private final ConcurrentHashMap selectorMap = new ConcurrentHashMap<>(); + private final List strategies; private final Object lock = new Object(); @@ -59,12 +60,14 @@ public class TieredBrokerHostSelector implements HostSelector public TieredBrokerHostSelector( CoordinatorRuleManager ruleManager, TieredBrokerConfig tierConfig, - ServerDiscoveryFactory serverDiscoveryFactory + ServerDiscoveryFactory serverDiscoveryFactory, + List strategies ) { this.ruleManager = ruleManager; this.tierConfig = tierConfig; this.serverDiscoveryFactory = serverDiscoveryFactory; + this.strategies = strategies; } @LifecycleStart @@ -128,12 +131,12 @@ public class TieredBrokerHostSelector implements HostSelector String brokerServiceName = null; - // Somewhat janky way of always selecting highest priority broker for this type of query - if (query instanceof TimeBoundaryQuery) { - brokerServiceName = Iterables.getFirst( - tierConfig.getTierToBrokerMap().values(), - tierConfig.getDefaultBrokerServiceName() - ); + for (TieredBrokerSelectorStrategy strategy : strategies) { + final Optional optionalName = strategy.getBrokerServiceName(tierConfig, query); + if (optionalName.isPresent()) { + brokerServiceName = optionalName.get(); + break; + } } if (brokerServiceName == null) { diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategiesProvider.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategiesProvider.java new file mode 100644 index 00000000000..3300261b7d4 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategiesProvider.java @@ -0,0 +1,59 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.router; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.client.util.Lists; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import com.google.inject.Provider; + +import java.util.List; + +/** + */ +public class TieredBrokerSelectorStrategiesProvider implements Provider> +{ + private final List strategies = Lists.newArrayList(); + + @Inject + public TieredBrokerSelectorStrategiesProvider(ObjectMapper jsonMapper, TieredBrokerConfig config) + { + try { + this.strategies.addAll( + (List) jsonMapper.readValue( + config.getStrategies(), new TypeReference>() + { + } + ) + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Override + public List get() + { + return strategies; + } +} diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java new file mode 100644 index 00000000000..bccf4d0d942 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java @@ -0,0 +1,19 @@ +package io.druid.server.router; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Optional; +import io.druid.query.Query; + +/** + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "timeBoundary", value = TimeBoundaryTieredBrokerSelectorStrategy.class), + @JsonSubTypes.Type(name = "priority", value = PriorityTieredBrokerSelectorStrategy.class) +}) + +public interface TieredBrokerSelectorStrategy +{ + public Optional getBrokerServiceName(TieredBrokerConfig config, Query query); +} diff --git a/server/src/main/java/io/druid/server/router/TimeBoundaryTieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TimeBoundaryTieredBrokerSelectorStrategy.java new file mode 100644 index 00000000000..135e87c1411 --- /dev/null +++ b/server/src/main/java/io/druid/server/router/TimeBoundaryTieredBrokerSelectorStrategy.java @@ -0,0 +1,46 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.router; + +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import io.druid.query.Query; +import io.druid.query.timeboundary.TimeBoundaryQuery; + +/** + */ +public class TimeBoundaryTieredBrokerSelectorStrategy implements TieredBrokerSelectorStrategy +{ + @Override + public Optional getBrokerServiceName(TieredBrokerConfig tierConfig, Query query) + { + // Somewhat janky way of always selecting highest priority broker for this type of query + if (query instanceof TimeBoundaryQuery) { + return Optional.of( + Iterables.getFirst( + tierConfig.getTierToBrokerMap().values(), + tierConfig.getDefaultBrokerServiceName() + ) + ); + } + + return Optional.absent(); + } +} diff --git a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java index 6d058d200fa..3d1d0cd0bb0 100644 --- a/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/io/druid/server/router/TieredBrokerHostSelectorTest.java @@ -84,7 +84,8 @@ public class TieredBrokerHostSelectorTest return "hotBroker"; } }, - factory + factory, + Arrays.asList(new TimeBoundaryTieredBrokerSelectorStrategy(), new PriorityTieredBrokerSelectorStrategy()) ); EasyMock.expect(factory.createSelector(EasyMock.anyObject())).andReturn(selector).atLeastOnce(); EasyMock.replay(factory); @@ -196,6 +197,30 @@ public class TieredBrokerHostSelectorTest Assert.assertEquals("coldBroker", brokerName); } + @Test + public void testPrioritySelect() throws Exception + { + String brokerName = (String) brokerSelector.select( + Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .aggregators(Arrays.asList(new CountAggregatorFactory("count"))) + .intervals( + new MultipleIntervalSegmentSpec( + Arrays.asList( + new Interval("2011-08-31/2011-09-01"), + new Interval("2012-08-31/2012-09-01"), + new Interval("2013-08-31/2013-09-01") + ) + ) + ) + .context(ImmutableMap.of("priority", -1)) + .build() + ).lhs; + + Assert.assertEquals("hotBroker", brokerName); + } + + private static class TestRuleManager extends CoordinatorRuleManager { public TestRuleManager( diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index 681ae78fa27..c9a2f64fb7b 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; +import com.google.inject.TypeLiteral; import com.metamx.common.logger.Logger; import io.airlift.command.Command; import io.druid.client.RoutingDruidClient; @@ -42,6 +43,8 @@ import io.druid.server.router.QueryHostFinder; import io.druid.server.router.Router; import io.druid.server.router.TieredBrokerConfig; import io.druid.server.router.TieredBrokerHostSelector; +import io.druid.server.router.TieredBrokerSelectorStrategiesProvider; +import io.druid.server.router.TieredBrokerSelectorStrategy; import org.eclipse.jetty.server.Server; import java.util.List; @@ -79,6 +82,10 @@ public class CliRouter extends ServerRunnable binder.bind(TieredBrokerHostSelector.class).in(ManageLifecycle.class); binder.bind(QueryHostFinder.class).in(LazySingleton.class); binder.bind(RoutingDruidClient.class).in(LazySingleton.class); + binder.bind(new TypeLiteral>(){}) + .toProvider(TieredBrokerSelectorStrategiesProvider.class) + .in(LazySingleton.class); + binder.bind(JettyServerInitializer.class).to(RouterJettyServerInitializer.class).in(LazySingleton.class); From 09e2f13ffcc82cd1fd550a0e34b48d8513c690c9 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 22 Jul 2014 10:12:47 -0700 Subject: [PATCH 178/270] adding missing header --- .../router/TieredBrokerSelectorStrategy.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java index bccf4d0d942..40a7714d870 100644 --- a/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java +++ b/server/src/main/java/io/druid/server/router/TieredBrokerSelectorStrategy.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.server.router; import com.fasterxml.jackson.annotation.JsonSubTypes; From f847a46ccfda8570ba72e91da9cdd614cc8a5506 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 22 Jul 2014 15:37:48 -0700 Subject: [PATCH 179/270] fix typo --- .../main/java/io/druid/storage/s3/S3DataSegmentMover.java | 2 +- .../main/java/io/druid/cli/RouterJettyServerInitializer.java | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java index 379dd8374fa..08aee176480 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentMover.java @@ -130,7 +130,7 @@ public class S3DataSegmentMover implements DataSegmentMover final S3Object s3Object = list[0]; if (s3Object.getStorageClass() != null && s3Object.getStorageClass().equals(S3Object.STORAGE_CLASS_GLACIER)) { - log.warn("Cannot move file[s3://%s/%s] of storage class glacier, skipping."); + log.warn("Cannot move file[s3://%s/%s] of storage class glacier, skipping.", s3Bucket, s3Path); } else { log.info( "Moving file[s3://%s/%s] to [s3://%s/%s]", diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 54c3a7f69e4..7d39efda9b9 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -38,7 +38,7 @@ import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; -import org.eclipse.jetty.servlets.GzipFilter; +import org.eclipse.jetty.servlets.AsyncGzipFilter; /** */ @@ -85,12 +85,11 @@ public class RouterJettyServerInitializer implements JettyServerInitializer ) ), "/druid/v2/*" ); - queries.addFilter(GzipFilter.class, "/druid/v2/*", null); + queries.addFilter(AsyncGzipFilter.class, "/druid/v2/*", null); queries.addFilter(GuiceFilter.class, "/status/*", null); final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); - root.addFilter(GzipFilter.class, "/*", null); root.addFilter(GuiceFilter.class, "/*", null); final HandlerList handlerList = new HandlerList(); From 00f086665cb723248c246f9c793d5e26356b713b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 22 Jul 2014 16:55:21 -0700 Subject: [PATCH 180/270] default async timeout to server idle timeout --- .../java/io/druid/server/AsyncQueryForwardingServlet.java | 6 ++++++ .../java/io/druid/cli/RouterJettyServerInitializer.java | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 9888f0f8a30..77354b81bf9 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -37,6 +37,7 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.query.DataSourceUtil; import io.druid.query.Query; +import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; import org.jboss.netty.buffer.ChannelBuffer; @@ -66,6 +67,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); private static final Joiner COMMA_JOIN = Joiner.on(","); + private final ServerConfig config; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final QueryHostFinder hostFinder; @@ -74,6 +76,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet private final RequestLogger requestLogger; public AsyncQueryForwardingServlet( + ServerConfig config, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, QueryHostFinder hostFinder, @@ -82,6 +85,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet RequestLogger requestLogger ) { + this.config = config; this.jsonMapper = jsonMapper; this.smileMapper = smileMapper; this.hostFinder = hostFinder; @@ -95,6 +99,8 @@ public class AsyncQueryForwardingServlet extends HttpServlet throws ServletException, IOException { final AsyncContext asyncContext = req.startAsync(req, res); + // default async timeout to be same as maxIdleTime for now + asyncContext.setTimeout(config.getMaxIdleTime().toStandardDuration().getMillis()); asyncContext.start( new Runnable() { diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 54c3a7f69e4..25c8d2c4bfb 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -29,6 +29,7 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.server.AsyncQueryForwardingServlet; import io.druid.server.initialization.JettyServerInitializer; +import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; import org.eclipse.jetty.server.Handler; @@ -38,12 +39,14 @@ import org.eclipse.jetty.server.handler.HandlerList; import org.eclipse.jetty.servlet.DefaultServlet; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.AsyncGzipFilter; import org.eclipse.jetty.servlets.GzipFilter; /** */ public class RouterJettyServerInitializer implements JettyServerInitializer { + private final ServerConfig config; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final QueryHostFinder hostFinder; @@ -53,6 +56,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer @Inject public RouterJettyServerInitializer( + ServerConfig config, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, QueryHostFinder hostFinder, @@ -61,6 +65,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer RequestLogger requestLogger ) { + this.config = config; this.jsonMapper = jsonMapper; this.smileMapper = smileMapper; this.hostFinder = hostFinder; @@ -76,6 +81,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer queries.addServlet( new ServletHolder( new AsyncQueryForwardingServlet( + config, jsonMapper, smileMapper, hostFinder, From e76561e400cfd16b8a3fae375fc8dea1b79445fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 22 Jul 2014 16:55:43 -0700 Subject: [PATCH 181/270] use async gzip filter for async servlet --- .../main/java/io/druid/cli/RouterJettyServerInitializer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 25c8d2c4bfb..ebff67d2a84 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -91,7 +91,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer ) ), "/druid/v2/*" ); - queries.addFilter(GzipFilter.class, "/druid/v2/*", null); + queries.addFilter(AsyncGzipFilter.class, "/druid/v2/*", null); queries.addFilter(GuiceFilter.class, "/status/*", null); final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS); From 03efbb0e023d5ddd5c7a9856a16c0ba372c4d028 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 22 Jul 2014 16:56:34 -0700 Subject: [PATCH 182/270] default query timeout to server idle timeout --- .../main/java/io/druid/server/QueryResource.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 4787b17465a..59ebf5b9441 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -41,6 +41,7 @@ import io.druid.query.DataSourceUtil; import io.druid.query.Query; import io.druid.query.QueryInterruptedException; import io.druid.query.QuerySegmentWalker; +import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import org.joda.time.DateTime; @@ -70,6 +71,7 @@ public class QueryResource public static final String APPLICATION_SMILE = "application/smile"; public static final String APPLICATION_JSON = "application/json"; + private final ServerConfig config; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final QuerySegmentWalker texasRanger; @@ -79,6 +81,7 @@ public class QueryResource @Inject public QueryResource( + ServerConfig config, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, QuerySegmentWalker texasRanger, @@ -87,6 +90,7 @@ public class QueryResource QueryManager queryManager ) { + this.config = config; this.jsonMapper = jsonMapper.copy(); this.jsonMapper.getFactory().configure(JsonGenerator.Feature.AUTO_CLOSE_TARGET, false); @@ -135,6 +139,15 @@ public class QueryResource queryId = UUID.randomUUID().toString(); query = query.withId(queryId); } + if (query.getContextValue("timeout") == null) { + query = query.withOverriddenContext( + ImmutableMap.of( + "timeout", + config.getMaxIdleTime().toStandardDuration().getMillis() + ) + ); + } + if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); From 387350abd8c30349141714fc0e0f742fef233738 Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 23 Jul 2014 13:41:14 -0700 Subject: [PATCH 183/270] fix toc entry --- docs/content/toc.textile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 76ba3e31133..8cacc102716 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -73,7 +73,7 @@ h2. Architecture *** "Peon":./Peons.html * External Dependencies ** "Deep Storage":./Deep-Storage.html -** "MySQL":./MySQL.html +** "Metadata Storage":./MySQL.html ** "ZooKeeper":./ZooKeeper.html h2. Experimental From 27cda43d89946068bf972173893c6a8c4f05e82f Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 24 Jul 2014 13:16:44 +0530 Subject: [PATCH 184/270] add docs doc for inverted topN metric spec --- docs/content/TopNMetricSpec.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/content/TopNMetricSpec.md b/docs/content/TopNMetricSpec.md index 299f8a358ff..10f5f33091d 100644 --- a/docs/content/TopNMetricSpec.md +++ b/docs/content/TopNMetricSpec.md @@ -60,3 +60,19 @@ See [http://www.davekoelle.com/alphanum.html](http://www.davekoelle.com/alphanum |--------|-----------|---------| |type|this indicates an alpha-numeric sort|yes| |previousStop|the starting point of the alpha-numeric sort. For example, if a previousStop value is 'b', all values before 'b' are discarded. This field can be used to paginate through all the dimension values.|no| + +## Inverted TopNMetricSpec + +Sort dimension values in inverted order, i.e inverts the order of the delegate metric spec. It can be used to sort the values in descending order. + +```json +"metric": { + "type": "inverted", + "metric": +} +``` + +|property|description|required?| +|--------|-----------|---------| +|type|this indicates an inverted sort|yes| +|metric|the delegate metric spec. |yes| From baeb93772d7800389562fe393dd79873a7e5b786 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 24 Jul 2014 14:20:43 -0700 Subject: [PATCH 185/270] Rename OrderBy to LimitSpec in docs. --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 2 +- docs/content/Tutorial:-Webstream.md | 2 +- docs/content/Twitter-Tutorial.md | 2 +- docs/content/toc.textile | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index f5fbbc34712..3853474e83c 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -302,7 +302,7 @@ and put the following in there: } ``` -Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [OrderBy](OrderBy.html) thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. +Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [LimitSpec](LimitSpec.html) thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. If you issue the query: diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 3d8d70e8fcd..940665ed75c 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -267,7 +267,7 @@ and put the following in there: } ``` -Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [OrderBy](OrderBy.html) thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. +Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [LimitSpec](LimitSpec.html) thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. If you issue the query: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index a76ed397e65..0221b7b0c17 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -315,7 +315,7 @@ and put the following in there: } ``` -Woah! Our query just got a way more complicated. Now we have these "Filters":Filters.html things and this "OrderBy":OrderBy.html thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. +Woah! Our query just got a way more complicated. Now we have these "Filters":Filters.html things and this "LimitSpec":LimitSpec.html thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. If you issue the query: diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 8cacc102716..5ca3d978fe4 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -49,7 +49,7 @@ h2. Querying ** "DimensionSpecs":./DimensionSpecs.html * Query Types ** "GroupBy":./GroupByQuery.html -*** "OrderBy":./OrderBy.html +*** "LimitSpec":./LimitSpec.html *** "Having":./Having.html ** "Search":./SearchQuery.html *** "SearchQuerySpec":./SearchQuerySpec.html From 9c360146450680cf8133fb25069f145860b5d842 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 24 Jul 2014 14:41:45 -0700 Subject: [PATCH 186/270] remove css from docs --- docs/content/css/docs.css | 60 --------------------------------------- docs/content/css/toc.css | 6 ---- 2 files changed, 66 deletions(-) delete mode 100644 docs/content/css/docs.css delete mode 100644 docs/content/css/toc.css diff --git a/docs/content/css/docs.css b/docs/content/css/docs.css deleted file mode 100644 index c59923a98ca..00000000000 --- a/docs/content/css/docs.css +++ /dev/null @@ -1,60 +0,0 @@ -.doc-content pre, .doc-content pre code { - overflow: auto; - white-space: pre; - word-wrap: normal; -} - -.doc-content p { - margin: 18px 0 18px 0; -} - -/*** HACK: This is a horrible hack, but I have no clue why images don't want to stay in the container **/ -.doc-content img { - max-width: 847.5px; -} - -.doc-content code { - background-color: #e0e0e0; -} - -.doc-content pre code { - background-color: transparent; -} - -.doc-content table, -.doc-content pre { - margin: 35px 0 35px 0; -} - -.doc-content table, -.doc-content table > thead > tr > th, -.doc-content table > tbody > tr > th, -.doc-content table > tfoot > tr > th, -.doc-content table > thead > tr > td, -.doc-content table > tbody > tr > td, -.doc-content table > tfoot > tr > td { - border: 1px solid #dddddd; -} - -.doc-content table > thead > tr > th, -.doc-content table > thead > tr > td { - border-bottom-width: 2px; -} - -.doc-content table > tbody > tr:nth-child(odd) > td, -.doc-content table > tbody > tr:nth-child(odd) > th { - background-color: #f9f9f9; -} - -.doc-content table > tbody > tr:hover > td, -.doc-content table > tbody > tr:hover > th { - background-color: #d5d5d5; -} - -.doc-content table code { - background-color: transparent; -} - -td, th { - padding: 5px; -} diff --git a/docs/content/css/toc.css b/docs/content/css/toc.css deleted file mode 100644 index c074ef7d79a..00000000000 --- a/docs/content/css/toc.css +++ /dev/null @@ -1,6 +0,0 @@ -.toc ul { - list-style: none; - list-style-position: inside; - padding-left: 15px; -} - From 85f739797f43ce708ccfb160af81dd0bac4129e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 24 Jul 2014 16:49:09 -0700 Subject: [PATCH 187/270] remove debug println --- .../java/io/druid/client/CachingClusteredClient.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 1e296f0a4d1..d0135eb7f4a 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -242,15 +242,6 @@ public class CachingClusteredClient implements QueryRunner } descriptors.add(segment.rhs); - System.out.println( - String.format( - "Server %s has %s_%s_%s", - server.getHost(), - segment.rhs.getInterval(), - segment.rhs.getPartitionNumber(), - segment.rhs.getVersion() - ) - ); } } From d3b920769d0642a6d59ca1a4dcff0709a757b6b8 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 25 Jul 2014 09:26:44 -0700 Subject: [PATCH 188/270] a few more cleanups to docs --- docs/content/Tutorial:-A-First-Look-at-Druid.md | 2 +- docs/content/Tutorial:-Webstream.md | 2 +- docs/content/Twitter-Tutorial.md | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 3853474e83c..923f3f45e1b 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -289,7 +289,7 @@ and put the following in there: "dataSource": "wikipedia", "granularity": "all", "dimensions": [ "page" ], - "orderBy": { + "limitSpec": { "type": "default", "columns": [ { "dimension": "edit_count", "direction": "DESCENDING" } ], "limit": 10 diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 940665ed75c..db87404ab88 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -251,7 +251,7 @@ and put the following in there: "dataSource": "webstream", "granularity": "all", "dimensions": [ "geo_region" ], - "orderBy": { + "limitSpec": { "type": "default", "columns": [ { "dimension": "known_users", "direction": "DESCENDING" } diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index 0221b7b0c17..3ed5b839494 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -306,7 +306,7 @@ and put the following in there: "dataSource": "twitterstream", "granularity": "all", "dimensions": ["htags"], - "orderBy": {"type":"default", "columns":[{"dimension": "tweets", "direction":"DESCENDING"}], "limit":5}, + "limitSpec": {"type":"default", "columns":[{"dimension": "tweets", "direction":"DESCENDING"}], "limit":5}, "aggregations":[ { "type": "longSum", "fieldName": "tweets", "name": "tweets"} ], @@ -315,7 +315,7 @@ and put the following in there: } ``` -Woah! Our query just got a way more complicated. Now we have these "Filters":Filters.html things and this "LimitSpec":LimitSpec.html thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. +Woah! Our query just got a way more complicated. Now we have these [Filters](Filters.html) things and this [LimitSpec](LimitSpec.html) thing. Fear not, it turns out the new objects we've introduced to our query can help define the format of our results and provide an answer to our question. If you issue the query: From bb5c5b3ce704f87541d92d8adf92646af17bc902 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 25 Jul 2014 10:50:41 -0700 Subject: [PATCH 189/270] fix config with broker select and add docs --- docs/content/Broker-Config.md | 3 +++ .../java/io/druid/client/selector/TierSelectorStrategy.java | 2 +- services/src/main/java/io/druid/cli/CliBroker.java | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/content/Broker-Config.md b/docs/content/Broker-Config.md index da4c8a9db5d..6da606c70e4 100644 --- a/docs/content/Broker-Config.md +++ b/docs/content/Broker-Config.md @@ -14,6 +14,9 @@ The broker module uses several of the default modules in [Configuration](Configu |--------|---------------|-----------|-------| |`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| |`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to historical nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`| +|`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`| +|`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None| + #### Local Cache diff --git a/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java b/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java index cadbd435478..715962f0e80 100644 --- a/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java +++ b/server/src/main/java/io/druid/client/selector/TierSelectorStrategy.java @@ -29,7 +29,7 @@ import java.util.TreeMap; /** */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = HighestPriorityTierSelectorStrategy.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "tier", defaultImpl = HighestPriorityTierSelectorStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "highestPriority", value = HighestPriorityTierSelectorStrategy.class), @JsonSubTypes.Type(name = "lowestPriority", value = LowestPriorityTierSelectorStrategy.class), diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 2381c13d282..f23102bccdd 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -85,7 +85,7 @@ public class CliBroker extends ServerRunnable binder.bind(Cache.class).toProvider(CacheProvider.class).in(ManageLifecycle.class); JsonConfigProvider.bind(binder, "druid.broker.cache", CacheProvider.class); JsonConfigProvider.bind(binder, "druid.broker.cache", CacheConfig.class); - JsonConfigProvider.bind(binder, "druid.broker.select.tier", TierSelectorStrategy.class); + JsonConfigProvider.bind(binder, "druid.broker.select", TierSelectorStrategy.class); JsonConfigProvider.bind(binder, "druid.broker.select.tier.custom", CustomTierSelectorStrategyConfig.class); JsonConfigProvider.bind(binder, "druid.broker.balancer", ServerSelectorStrategy.class); From 45e443bedbe011fa8df463fb2c77a3cd93fd5e4c Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 25 Jul 2014 16:07:40 -0700 Subject: [PATCH 190/270] a few slight tweaks and bumping up curator version --- docs/content/Broker-Config.md | 2 +- pom.xml | 2 +- .../hyperloglog/HyperLogLogCollectorTest.java | 9 ++++++--- server/src/main/java/io/druid/server/QueryResource.java | 1 - 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/content/Broker-Config.md b/docs/content/Broker-Config.md index 6da606c70e4..80eaf4ce404 100644 --- a/docs/content/Broker-Config.md +++ b/docs/content/Broker-Config.md @@ -12,10 +12,10 @@ The broker module uses several of the default modules in [Configuration](Configu |Property|Possible Values|Description|Default| |--------|---------------|-----------|-------| -|`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| |`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to historical nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`| |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`| |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None| +|`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| #### Local Cache diff --git a/pom.xml b/pom.xml index cfd8a332b51..8ac79919983 100644 --- a/pom.xml +++ b/pom.xml @@ -40,7 +40,7 @@ UTF-8 0.26.6 - 2.5.0 + 2.6.0 0.2.7 diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java index dbc06accf2f..ddf02efecaf 100644 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperLogLogCollectorTest.java @@ -90,7 +90,8 @@ public class HyperLogLogCollectorTest * with 100 values, requiring a floating max as described in * http://druid.io/blog/2014/02/18/hyperloglog-optimizations-for-real-world-systems.html */ - @Ignore @Test + @Ignore + @Test public void testHighCardinalityRollingFold() throws Exception { final HyperLogLogCollector rolling = HyperLogLogCollector.makeLatestCollector(); @@ -130,7 +131,8 @@ public class HyperLogLogCollectorTest Assert.assertEquals(n, rolling.estimateCardinality(), n * 0.05); } - @Ignore @Test + @Ignore + @Test public void testHighCardinalityRollingFold2() throws Exception { final HyperLogLogCollector rolling = HyperLogLogCollector.makeLatestCollector(); @@ -876,7 +878,8 @@ public class HyperLogLogCollectorTest } // Provides a nice printout of error rates as a function of cardinality - @Ignore @Test + @Ignore + @Test public void showErrorRate() throws Exception { HashFunction fn = Hashing.murmur3_128(); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 59ebf5b9441..aee2fb0ddc4 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -148,7 +148,6 @@ public class QueryResource ); } - if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); } From fc95a79abd8dcd58beac9e2b37656f755b2d2032 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 25 Jul 2014 16:10:55 -0700 Subject: [PATCH 191/270] add query priorities to metric emission --- .../main/java/io/druid/server/AsyncQueryForwardingServlet.java | 1 + server/src/main/java/io/druid/server/QueryResource.java | 1 + 2 files changed, 2 insertions(+) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 77354b81bf9..872e7a8343b 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -187,6 +187,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet emitter.emit( new ServiceMetricEvent.Builder() .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) + .setUser3(String.valueOf(query.getContextPriority(0))) .setUser4(query.getType()) .setUser5(COMMA_JOIN.join(query.getIntervals())) .setUser6(String.valueOf(query.hasFilters())) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index aee2fb0ddc4..18ba190393f 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -178,6 +178,7 @@ public class QueryResource emitter.emit( new ServiceMetricEvent.Builder() .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) + .setUser3(String.valueOf(query.getContextPriority(0))) .setUser4(query.getType()) .setUser5(COMMA_JOIN.join(query.getIntervals())) .setUser6(String.valueOf(query.hasFilters())) From 2d4f1889e879515bf55e4325573ad6174dfc0ef2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 25 Jul 2014 16:18:27 -0700 Subject: [PATCH 192/270] completely async request proxying + jetty update --- pom.xml | 6 +- .../io/druid/client/RoutingDruidClient.java | 15 +- .../server/AsyncQueryForwardingServlet.java | 385 ++++++++++-------- .../server/log/EmittingRequestLogger.java | 3 +- .../druid/server/log/FileRequestLogger.java | 2 +- .../druid/server/log/NoopRequestLogger.java | 4 +- .../io/druid/server/log/RequestLogger.java | 4 +- .../cli/RouterJettyServerInitializer.java | 1 - 8 files changed, 225 insertions(+), 195 deletions(-) diff --git a/pom.xml b/pom.xml index 8ac79919983..909fa04dd6d 100644 --- a/pom.xml +++ b/pom.xml @@ -324,17 +324,17 @@ org.eclipse.jetty jetty-server - 9.2.1.v20140609 + 9.2.2.v20140723 org.eclipse.jetty jetty-servlet - 9.2.1.v20140609 + 9.2.2.v20140723 org.eclipse.jetty jetty-servlets - 9.2.1.v20140609 + 9.2.2.v20140723 joda-time diff --git a/server/src/main/java/io/druid/client/RoutingDruidClient.java b/server/src/main/java/io/druid/client/RoutingDruidClient.java index 79ae1c16f6d..3f6c2bf19f8 100644 --- a/server/src/main/java/io/druid/client/RoutingDruidClient.java +++ b/server/src/main/java/io/druid/client/RoutingDruidClient.java @@ -38,6 +38,7 @@ import org.jboss.netty.handler.codec.http.HttpHeaders; import javax.inject.Inject; import java.io.IOException; +import java.net.URI; import java.net.URL; import java.util.concurrent.atomic.AtomicInteger; @@ -72,7 +73,7 @@ public class RoutingDruidClient } public ListenableFuture postQuery( - String url, + URI uri, Query query, HttpResponseHandler responseHandler ) @@ -80,9 +81,9 @@ public class RoutingDruidClient final ListenableFuture future; try { - log.debug("Querying url[%s]", url); + log.debug("Querying url[%s]", uri); future = httpClient - .post(new URL(url)) + .post(uri.toURL()) .setContent(objectMapper.writeValueAsBytes(query)) .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? QueryResource.APPLICATION_SMILE : QueryResource.APPLICATION_JSON) .go(responseHandler); @@ -115,13 +116,13 @@ public class RoutingDruidClient } public ListenableFuture get( - String url, + URI uri, HttpResponseHandler responseHandler ) { try { return httpClient - .get(new URL(url)) + .get(uri.toURL()) .go(responseHandler); } catch (IOException e) { @@ -130,13 +131,13 @@ public class RoutingDruidClient } public ListenableFuture delete( - String url, + URI uri, HttpResponseHandler responseHandler ) { try { return httpClient - .delete(new URL(url)) + .delete(uri.toURL()) .go(responseHandler); } catch (IOException e) { diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 872e7a8343b..ca87d8b14d2 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -27,6 +27,9 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; @@ -37,7 +40,6 @@ import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.query.DataSourceUtil; import io.druid.query.Query; -import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; import org.jboss.netty.buffer.ChannelBuffer; @@ -49,12 +51,13 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; import javax.servlet.AsyncContext; import javax.servlet.ServletException; +import javax.servlet.ServletOutputStream; import javax.servlet.annotation.WebServlet; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.io.OutputStream; +import java.net.URI; import java.util.Map; import java.util.UUID; @@ -67,7 +70,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); private static final Joiner COMMA_JOIN = Joiner.on(","); - private final ServerConfig config; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final QueryHostFinder hostFinder; @@ -76,7 +78,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet private final RequestLogger requestLogger; public AsyncQueryForwardingServlet( - ServerConfig config, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, QueryHostFinder hostFinder, @@ -85,7 +86,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet RequestLogger requestLogger ) { - this.config = config; this.jsonMapper = jsonMapper; this.smileMapper = smileMapper; this.hostFinder = hostFinder; @@ -99,215 +99,247 @@ public class AsyncQueryForwardingServlet extends HttpServlet throws ServletException, IOException { final AsyncContext asyncContext = req.startAsync(req, res); - // default async timeout to be same as maxIdleTime for now - asyncContext.setTimeout(config.getMaxIdleTime().toStandardDuration().getMillis()); - asyncContext.start( - new Runnable() - { - @Override - public void run() - { - try { - final HttpResponseHandler responseHandler = new PassthroughHttpResponseHandler(asyncContext, jsonMapper); + asyncContext.setTimeout(0); - final String host = hostFinder.getDefaultHost(); - routingDruidClient.get(makeUrl(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler); - } - catch (Exception e) { - handleException(jsonMapper, asyncContext, e); - } - } - } + final HttpResponseHandler responseHandler = + new PassthroughHttpResponseHandler(res); + + final URI uri = rewriteURI(hostFinder.getDefaultHost(), req); + asyncComplete( + res, + asyncContext, + jsonMapper, + routingDruidClient.get(uri, responseHandler) ); } @Override protected void doDelete(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException { - final AsyncContext asyncContext = req.startAsync(req, res); - asyncContext.start( - new Runnable() - { - @Override - public void run() - { - try { - final HttpResponseHandler responseHandler = new PassthroughHttpResponseHandler(asyncContext, jsonMapper); + final AsyncContext asyncContext = req.startAsync(); + asyncContext.setTimeout(0); - final String host = hostFinder.getDefaultHost(); - routingDruidClient.delete(makeUrl(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler); - } - catch (Exception e) { - handleException(jsonMapper, asyncContext, e); - } - } - } + final HttpResponseHandler responseHandler = + new PassthroughHttpResponseHandler(res); + + final String host = hostFinder.getDefaultHost(); + + asyncComplete( + res, + asyncContext, + jsonMapper, + routingDruidClient.delete(rewriteURI(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler) ); } @Override - protected void doPost(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException + protected void doPost(final HttpServletRequest req, final HttpServletResponse res) throws ServletException, IOException { + final String id = req.getHeader("X-Client"); + log.info("query id [%s]", id); + final long start = System.currentTimeMillis(); - final AsyncContext asyncContext = req.startAsync(req, res); - asyncContext.start( - new Runnable() - { - @Override - public void run() + final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(req.getContentType()); + final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; + + try { + final Query inputQuery = objectMapper.readValue(req.getInputStream(), Query.class); + if (log.isDebugEnabled()) { + log.debug("Got query [%s]", inputQuery); + } + + final Query query; + if (inputQuery.getId() == null) { + query = inputQuery.withId(UUID.randomUUID().toString()); + } else { + query = inputQuery; + } + + URI rewrittenURI = rewriteURI(hostFinder.getHost(query), req); + + + final AsyncContext asyncContext = req.startAsync(); + // let proxy http client timeout + asyncContext.setTimeout(0); + + ListenableFuture future = routingDruidClient.postQuery( + rewrittenURI, + query, + new PassthroughHttpResponseHandler(res) + ); + + Futures.addCallback( + future, + new FutureCallback() { - final HttpServletRequest request = (HttpServletRequest) asyncContext.getRequest(); - - final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(request.getContentType()); - final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - - Query inputQuery = null; - try { - inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); - if (inputQuery.getId() == null) { - inputQuery = inputQuery.withId(UUID.randomUUID().toString()); - } - final Query query = inputQuery; - - if (log.isDebugEnabled()) { - log.debug("Got query [%s]", inputQuery); - } - - final HttpResponseHandler responseHandler = new PassthroughHttpResponseHandler( - asyncContext, - objectMapper - ) - { - @Override - public ClientResponse done(ClientResponse clientResponse) - { - final long requestTime = System.currentTimeMillis() - start; - log.debug("Request time: %d", requestTime); - - emitter.emit( - new ServiceMetricEvent.Builder() - .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) - .setUser3(String.valueOf(query.getContextPriority(0))) - .setUser4(query.getType()) - .setUser5(COMMA_JOIN.join(query.getIntervals())) - .setUser6(String.valueOf(query.hasFilters())) - .setUser7(request.getRemoteAddr()) - .setUser8(query.getId()) - .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) - .build("request/time", requestTime) - ); - - try { - requestLogger.log( - new RequestLogLine( - new DateTime(), - request.getRemoteAddr(), - query, - new QueryStats( - ImmutableMap.of( - "request/time", - requestTime, - "success", - true - ) - ) - ) - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - - return super.done(clientResponse); - } - }; - - routingDruidClient.postQuery( - makeUrl(hostFinder.getHost(inputQuery), request), - inputQuery, - responseHandler + @Override + public void onSuccess(@Nullable Object o) + { + final long requestTime = System.currentTimeMillis() - start; + emitter.emit( + new ServiceMetricEvent.Builder() + .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) + .setUser3(String.valueOf(query.getContextPriority(0))) + .setUser4(query.getType()) + .setUser5(COMMA_JOIN.join(query.getIntervals())) + .setUser6(String.valueOf(query.hasFilters())) + .setUser7(req.getRemoteAddr()) + .setUser8(query.getId()) + .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) + .build("request/time", requestTime) ); - } - catch (Exception e) { - handleException(objectMapper, asyncContext, e); try { requestLogger.log( new RequestLogLine( new DateTime(), - request.getRemoteAddr(), - inputQuery, - new QueryStats(ImmutableMap.of("success", false, "exception", e.toString())) + req.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "request/time", + requestTime, + "success", + true + ) + ) ) ); } - catch (Exception logError) { - log.error(logError, "Unable to log query [%s]!", inputQuery); + catch (Exception e) { + log.error(e, "Unable to log query [%s]!", query); + } + } + + @Override + public void onFailure(Throwable throwable) + { + try { + final String errorMessage = throwable.getMessage(); + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "success", + false, + "exception", + errorMessage == null ? "no message" : errorMessage) + ) + ) + ); + } + catch (IOException logError) { + log.error(logError, "Unable to log query [%s]!", query); } - log.makeAlert(e, "Exception handling request") - .addData("query", inputQuery) - .addData("peer", request.getRemoteAddr()) + log.makeAlert(throwable, "Exception handling request [%s]", id) + .addData("query", query) + .addData("peer", req.getRemoteAddr()) .emit(); } } + ); + + asyncComplete( + res, + asyncContext, + objectMapper, + future + ); + } catch(IOException e) { + log.warn(e, "Exception parsing query"); + final String errorMessage = e.getMessage() == null ? "no error message" : e.getMessage(); + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + null, + new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) + ) + ); + res.setStatus(HttpServletResponse.SC_BAD_REQUEST); + objectMapper.writeValue( + res.getOutputStream(), + ImmutableMap.of("error", errorMessage) + ); + } catch(Exception e) { + handleException(res, objectMapper, e); + } + } + + private static void asyncComplete( + final HttpServletResponse res, + final AsyncContext asyncContext, + final ObjectMapper objectMapper, + ListenableFuture future + ) + { + Futures.addCallback( + future, + new FutureCallback() + { + @Override + public void onSuccess(@Nullable Object o) + { + asyncContext.complete(); + } + + @Override + public void onFailure(Throwable throwable) + { + log.error(throwable, "Error processing query response"); + try { + handleException(res, objectMapper, throwable); + } catch(Exception err) { + log.error(err, "Unable to handle exception response"); + } + asyncContext.complete(); + } } ); } - private String makeUrl(final String host, final HttpServletRequest req) + private URI rewriteURI(final String host, final HttpServletRequest req) { + final StringBuilder uri = new StringBuilder("http://"); + uri.append(host); + uri.append(req.getRequestURI()); final String queryString = req.getQueryString(); - final String requestURI = req.getRequestURI() == null ? "" : req.getRequestURI(); - - if (queryString == null) { - return String.format("http://%s%s", host, requestURI); + if (queryString != null) { + uri.append("?").append(queryString); } - return String.format("http://%s%s?%s", host, requestURI, queryString); + return URI.create(uri.toString()); } - private static void handleException(ObjectMapper objectMapper, AsyncContext asyncContext, Throwable exception) + private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Throwable exception) throws IOException { - try { - HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); - if (!response.isCommitted()) { - final String errorMessage = exception.getMessage() == null ? "null exception" : exception.getMessage(); + if (!response.isCommitted()) { + final String errorMessage = exception.getMessage() == null ? "null exception" : exception.getMessage(); - response.resetBuffer(); - response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); - objectMapper.writeValue( - response.getOutputStream(), - ImmutableMap.of( - "error", errorMessage - ) - ); - } - response.flushBuffer(); - } - catch (IOException e) { - Throwables.propagate(e); - } - finally { - asyncContext.complete(); + response.resetBuffer(); + response.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); + objectMapper.writeValue( + response.getOutputStream(), + ImmutableMap.of("error", errorMessage) + ); } + response.flushBuffer(); } - private static class PassthroughHttpResponseHandler implements HttpResponseHandler + private static class PassthroughHttpResponseHandler implements HttpResponseHandler { - private final AsyncContext asyncContext; - private final ObjectMapper objectMapper; - private final OutputStream outputStream; + private final HttpServletResponse response; - public PassthroughHttpResponseHandler(AsyncContext asyncContext, ObjectMapper objectMapper) throws IOException + public PassthroughHttpResponseHandler(HttpServletResponse response) throws IOException { - this.asyncContext = asyncContext; - this.objectMapper = objectMapper; - this.outputStream = asyncContext.getResponse().getOutputStream(); + this.response = response; } protected void copyStatusHeaders(HttpResponse clientResponse) { - final HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); response.setStatus(clientResponse.getStatus().getCode()); response.setContentType(clientResponse.headers().get(HttpHeaders.Names.CONTENT_TYPE)); @@ -337,29 +369,29 @@ public class AsyncQueryForwardingServlet extends HttpServlet } @Override - public ClientResponse handleResponse(HttpResponse clientResponse) + public ClientResponse handleResponse(HttpResponse clientResponse) { copyStatusHeaders(clientResponse); try { + final ServletOutputStream outputStream = response.getOutputStream(); ChannelBuffer buf = clientResponse.getContent(); buf.readBytes(outputStream, buf.readableBytes()); + return ClientResponse.unfinished(outputStream); } catch (Exception e) { throw Throwables.propagate(e); } - - return ClientResponse.finished(outputStream); } @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk chunk + public ClientResponse handleChunk( + ClientResponse clientResponse, HttpChunk chunk ) { try { ChannelBuffer buf = chunk.getContent(); - buf.readBytes(outputStream, buf.readableBytes()); + buf.readBytes(clientResponse.getObj(), buf.readableBytes()); } catch (Exception e) { throw Throwables.propagate(e); @@ -368,25 +400,18 @@ public class AsyncQueryForwardingServlet extends HttpServlet } @Override - public ClientResponse done(ClientResponse clientResponse) + public ClientResponse done(ClientResponse clientResponse) { - asyncContext.complete(); return ClientResponse.finished(clientResponse.getObj()); } @Override public void exceptionCaught( - ClientResponse clientResponse, + ClientResponse clientResponse, Throwable e ) { - log.error(e, "Error processing query response"); - // throwing an exception here may cause resource leak - try { - handleException(objectMapper, asyncContext, e); - } catch(Exception err) { - log.error(err, "Unable to handle exception response"); - } + // exceptions are handled on future callback } } } diff --git a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java index 5a71415901d..798978d6d2f 100644 --- a/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/EmittingRequestLogger.java @@ -30,6 +30,7 @@ import io.druid.server.QueryStats; import io.druid.server.RequestLogLine; import org.joda.time.DateTime; +import java.io.IOException; import java.util.Map; public class EmittingRequestLogger implements RequestLogger @@ -44,7 +45,7 @@ public class EmittingRequestLogger implements RequestLogger } @Override - public void log(final RequestLogLine requestLogLine) throws Exception + public void log(final RequestLogLine requestLogLine) throws IOException { emitter.emit(new RequestLogEventBuilder(feed, requestLogLine)); } diff --git a/server/src/main/java/io/druid/server/log/FileRequestLogger.java b/server/src/main/java/io/druid/server/log/FileRequestLogger.java index 51c8ea4dcf6..ba69f58fc8e 100644 --- a/server/src/main/java/io/druid/server/log/FileRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/FileRequestLogger.java @@ -110,7 +110,7 @@ public class FileRequestLogger implements RequestLogger } @Override - public void log(RequestLogLine requestLogLine) throws Exception + public void log(RequestLogLine requestLogLine) throws IOException { synchronized (lock) { fileWriter.write( diff --git a/server/src/main/java/io/druid/server/log/NoopRequestLogger.java b/server/src/main/java/io/druid/server/log/NoopRequestLogger.java index 1088a13032a..39e9bd376e6 100644 --- a/server/src/main/java/io/druid/server/log/NoopRequestLogger.java +++ b/server/src/main/java/io/druid/server/log/NoopRequestLogger.java @@ -21,12 +21,14 @@ package io.druid.server.log; import io.druid.server.RequestLogLine; +import java.io.IOException; + /** */ public class NoopRequestLogger implements RequestLogger { @Override - public void log(RequestLogLine requestLogLine) throws Exception + public void log(RequestLogLine requestLogLine) throws IOException { // This is a no op! } diff --git a/server/src/main/java/io/druid/server/log/RequestLogger.java b/server/src/main/java/io/druid/server/log/RequestLogger.java index 2d4c938252b..9bb859f6d31 100644 --- a/server/src/main/java/io/druid/server/log/RequestLogger.java +++ b/server/src/main/java/io/druid/server/log/RequestLogger.java @@ -21,9 +21,11 @@ package io.druid.server.log; import io.druid.server.RequestLogLine; +import java.io.IOException; + /** */ public interface RequestLogger { - public void log(RequestLogLine requestLogLine) throws Exception; + public void log(RequestLogLine requestLogLine) throws IOException; } diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 0bd008b3969..8b329f035e9 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -80,7 +80,6 @@ public class RouterJettyServerInitializer implements JettyServerInitializer queries.addServlet( new ServletHolder( new AsyncQueryForwardingServlet( - config, jsonMapper, smileMapper, hostFinder, From df59d2acb49fd9f9cb23f2017498fa11a55ef585 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 25 Jul 2014 16:25:47 -0700 Subject: [PATCH 193/270] remove debug code --- .../main/java/io/druid/server/AsyncQueryForwardingServlet.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index ca87d8b14d2..d1abfacc192 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -135,9 +135,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet @Override protected void doPost(final HttpServletRequest req, final HttpServletResponse res) throws ServletException, IOException { - final String id = req.getHeader("X-Client"); - log.info("query id [%s]", id); - final long start = System.currentTimeMillis(); final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(req.getContentType()); final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; From 863929d55025a0a77ea3a2ad7169de9b9f2a35b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 25 Jul 2014 16:26:44 -0700 Subject: [PATCH 194/270] unnecessary arguments --- .../main/java/io/druid/server/AsyncQueryForwardingServlet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index d1abfacc192..90a3595c74d 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -98,7 +98,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet protected void doGet(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException { - final AsyncContext asyncContext = req.startAsync(req, res); + final AsyncContext asyncContext = req.startAsync(); asyncContext.setTimeout(0); final HttpResponseHandler responseHandler = From 82d623ada7f4f9bed7548467ea88bcd76b647191 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 25 Jul 2014 16:35:55 -0700 Subject: [PATCH 195/270] fix compilation error --- .../io/druid/server/AsyncQueryForwardingServlet.java | 3 +-- .../src/main/java/io/druid/server/QueryResource.java | 10 ++++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 90a3595c74d..f2b715a18f4 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -154,7 +154,6 @@ public class AsyncQueryForwardingServlet extends HttpServlet URI rewrittenURI = rewriteURI(hostFinder.getHost(query), req); - final AsyncContext asyncContext = req.startAsync(); // let proxy http client timeout asyncContext.setTimeout(0); @@ -232,7 +231,7 @@ public class AsyncQueryForwardingServlet extends HttpServlet log.error(logError, "Unable to log query [%s]!", query); } - log.makeAlert(throwable, "Exception handling request [%s]", id) + log.makeAlert(throwable, "Exception handling request [%s]", query.getId()) .addData("query", query) .addData("peer", req.getRemoteAddr()) .emit(); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 18ba190393f..0d38a33a33c 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -148,6 +148,16 @@ public class QueryResource ); } + if ((boolean) query.getContextValue("b", false)) { + System.out.println("***NEW QUERY***"); + while (true) { + System.out.println("SLEEPING"); + Thread.sleep(10000); + } + } else if ((boolean) query.getContextValue("a", false)) { + return Response.ok("hi").build(); + } + if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); } From df7a39a682e60326a8db7542da54d8f2308e5df1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Mon, 28 Jul 2014 09:59:21 -0700 Subject: [PATCH 196/270] fix index out of bounds --- .../java/io/druid/segment/QueryableIndexStorageAdapter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index 7eb83b70f02..9dcd136e024 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -508,7 +508,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter if (multiValueRow.size() == 0) { return null; } else if (multiValueRow.size() == 1) { - return columnVals.lookupName(multiValueRow.get(1)); + return columnVals.lookupName(multiValueRow.get(0)); } else { final String[] strings = new String[multiValueRow.size()]; for (int i = 0 ; i < multiValueRow.size() ; i++) { From 61af84a4a151582683efd326989844621810c6d9 Mon Sep 17 00:00:00 2001 From: Jasmine Hegman Date: Mon, 28 Jul 2014 14:44:59 -0700 Subject: [PATCH 197/270] Cleaning up code sample indentations --- .../Tutorial:-Loading-Your-Data-Part-2.md | 66 +++++++++++-------- 1 file changed, 39 insertions(+), 27 deletions(-) diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index c6833454187..71f993ca5b9 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -109,22 +109,27 @@ You should be comfortable starting Druid nodes at this point. If not, it may be { "schema": { "dataSource": "wikipedia", - "aggregators" : [{ - "type" : "count", - "name" : "count" - }, { - "type" : "doubleSum", - "name" : "added", - "fieldName" : "added" - }, { - "type" : "doubleSum", - "name" : "deleted", - "fieldName" : "deleted" - }, { - "type" : "doubleSum", - "name" : "delta", - "fieldName" : "delta" - }], + "aggregators" : [ + { + "type" : "count", + "name" : "count" + }, + { + "type" : "doubleSum", + "name" : "added", + "fieldName" : "added" + }, + { + "type" : "doubleSum", + "name" : "deleted", + "fieldName" : "deleted" + }, + { + "type" : "doubleSum", + "name" : "delta", + "fieldName" : "delta" + } + ], "indexGranularity": "none" }, "config": { @@ -196,13 +201,15 @@ Note: This config uses a "test" [rejection policy](Plumber.html) which will acce Issuing a [TimeBoundaryQuery](TimeBoundaryQuery.html) to the real-time node should yield valid results: ```json -[ { - "timestamp" : "2013-08-31T01:02:33.000Z", - "result" : { - "minTime" : "2013-08-31T01:02:33.000Z", - "maxTime" : "2013-08-31T12:41:27.000Z" +[ + { + "timestamp" : "2013-08-31T01:02:33.000Z", + "result" : { + "minTime" : "2013-08-31T01:02:33.000Z", + "maxTime" : "2013-08-31T12:41:27.000Z" + } } -} ] +] ``` Batch Ingestion @@ -287,22 +294,27 @@ Examining the contents of the file, you should find: }, "targetPartitionSize" : 5000000, "rollupSpec" : { - "aggs": [{ + "aggs": [ + { "type" : "count", "name" : "count" - }, { + }, + { "type" : "doubleSum", "name" : "added", "fieldName" : "added" - }, { + }, + { "type" : "doubleSum", "name" : "deleted", "fieldName" : "deleted" - }, { + }, + { "type" : "doubleSum", "name" : "delta", "fieldName" : "delta" - }], + } + ], "rollupGranularity" : "none" } } From 1844bf4f087f67644fcf2bf00a275b34e63819ba Mon Sep 17 00:00:00 2001 From: Jasmine Hegman Date: Mon, 28 Jul 2014 14:46:39 -0700 Subject: [PATCH 198/270] Adding a section to detail supporting Kafka 8 --- .../Tutorial:-Loading-Your-Data-Part-2.md | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index 71f993ca5b9..d0fc6a8fe45 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -341,3 +341,60 @@ Additional Information ---------------------- Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development). + + +Further Reading +--------------------- + +Ingesting from Kafka 8 +--------------------------------- + + +Continuing from the Kafka 7 examples, to support Kafka 8, a couple changes need to be made: + +- Update realtime node's configs for Kafka 8 extensions + - e.g. + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.121",...]` + - becomes + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.121",...]` +- Update realtime task config for changed keys + - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. + +```json + + "firehose" : { + "type" : "kafka-0.8", + "consumerProps" : { + "zookeeper.connect": "localhost:2181", + "zookeeper.connection.timeout.ms": "15000", + "zookeeper.session.timeout.ms": "15000", + "zookeeper.sync.time.ms": "5000", + "group.id": "topic-pixel-local", + "fetch.message.max.bytes": "1048586", + "auto.offset.reset": "largest", + "auto.commit.enable": "false" + }, + "feed" : "druidtest", + "parser" : { + "timestampSpec" : { + "column" : "utcdt", + "format" : "iso" + }, + "data" : { + "format" : "json" + }, + "dimensionExclusions" : [ + "wp" + ] + } + }, + "plumber" : { + "type" : "realtime", + "windowPeriod" : "PT10m", + "segmentGranularity":"hour", + "basePersistDirectory" : "/tmp/realtime/basePersist", + "rejectionPolicyFactory": { + "type": "messageTime" + } + } +``` From 088c2386dcae2dceb1a717b946a0f5018d8dc342 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 28 Jul 2014 16:32:15 -0700 Subject: [PATCH 199/270] rewrite router async logic --- pom.xml | 19 +- server/pom.xml | 11 +- .../io/druid/client/RoutingDruidClient.java | 147 ----- .../http/AbstractHttpClientProvider.java | 107 ++++ .../guice/http/DruidHttpClientConfig.java | 49 ++ .../guice/{ => http}/HttpClientModule.java | 78 +-- .../guice/http/JettyHttpClientModule.java | 152 ++++++ .../druid/initialization/Initialization.java | 2 +- .../server/AsyncQueryForwardingServlet.java | 502 ++++++------------ .../java/io/druid/server/QueryResource.java | 10 - .../src/main/java/io/druid/cli/CliRouter.java | 8 +- .../cli/RouterJettyServerInitializer.java | 20 +- 12 files changed, 520 insertions(+), 585 deletions(-) delete mode 100644 server/src/main/java/io/druid/client/RoutingDruidClient.java create mode 100644 server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java create mode 100644 server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java rename server/src/main/java/io/druid/guice/{ => http}/HttpClientModule.java (58%) create mode 100644 server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java diff --git a/pom.xml b/pom.xml index 909fa04dd6d..7540bf6ad89 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid @@ -282,9 +283,9 @@ 1 - org.glassfish - javax.el - 3.0.0 + org.glassfish + javax.el + 3.0.0 com.jamesmurty.utils @@ -336,6 +337,16 @@ jetty-servlets 9.2.2.v20140723 + + org.eclipse.jetty + jetty-client + 9.2.2.v20140723 + + + org.eclipse.jetty + jetty-proxy + 9.2.2.v20140723 + joda-time joda-time diff --git a/server/pom.xml b/server/pom.xml index 15b9b76df68..80b07742526 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -101,6 +102,14 @@ org.eclipse.jetty jetty-server + + org.eclipse.jetty + jetty-client + + + org.eclipse.jetty + jetty-proxy + com.google.code.findbugs jsr305 diff --git a/server/src/main/java/io/druid/client/RoutingDruidClient.java b/server/src/main/java/io/druid/client/RoutingDruidClient.java deleted file mode 100644 index 3f6c2bf19f8..00000000000 --- a/server/src/main/java/io/druid/client/RoutingDruidClient.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright (C) 2012, 2013 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 io.druid.client; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMultimap; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.metamx.common.logger.Logger; -import com.metamx.http.client.HttpClient; -import com.metamx.http.client.response.HttpResponseHandler; -import io.druid.guice.annotations.Client; -import io.druid.query.Query; -import io.druid.server.QueryResource; -import io.druid.server.router.Router; -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.handler.codec.http.HttpHeaders; - -import javax.inject.Inject; -import java.io.IOException; -import java.net.URI; -import java.net.URL; -import java.util.concurrent.atomic.AtomicInteger; - -/** - */ -public class RoutingDruidClient -{ - private static final Logger log = new Logger(RoutingDruidClient.class); - - private final ObjectMapper objectMapper; - private final HttpClient httpClient; - - private final AtomicInteger openConnections; - private final boolean isSmile; - - @Inject - public RoutingDruidClient( - ObjectMapper objectMapper, - @Router HttpClient httpClient - ) - { - this.objectMapper = objectMapper; - this.httpClient = httpClient; - - this.isSmile = this.objectMapper.getFactory() instanceof SmileFactory; - this.openConnections = new AtomicInteger(); - } - - public int getNumOpenConnections() - { - return openConnections.get(); - } - - public ListenableFuture postQuery( - URI uri, - Query query, - HttpResponseHandler responseHandler - ) - { - final ListenableFuture future; - - try { - log.debug("Querying url[%s]", uri); - future = httpClient - .post(uri.toURL()) - .setContent(objectMapper.writeValueAsBytes(query)) - .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ? QueryResource.APPLICATION_SMILE : QueryResource.APPLICATION_JSON) - .go(responseHandler); - - openConnections.getAndIncrement(); - - Futures.addCallback( - future, - new FutureCallback() - { - @Override - public void onSuccess(FinalType result) - { - openConnections.getAndDecrement(); - } - - @Override - public void onFailure(Throwable t) - { - openConnections.getAndDecrement(); - } - } - ); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - - return future; - } - - public ListenableFuture get( - URI uri, - HttpResponseHandler responseHandler - ) - { - try { - return httpClient - .get(uri.toURL()) - .go(responseHandler); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - public ListenableFuture delete( - URI uri, - HttpResponseHandler responseHandler - ) - { - try { - return httpClient - .delete(uri.toURL()) - .go(responseHandler); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } -} diff --git a/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java b/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java new file mode 100644 index 00000000000..ffe3e2dd57a --- /dev/null +++ b/server/src/main/java/io/druid/guice/http/AbstractHttpClientProvider.java @@ -0,0 +1,107 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.http; + +import com.google.common.base.Supplier; +import com.google.inject.Binding; +import com.google.inject.Inject; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Provider; +import com.google.inject.TypeLiteral; +import com.metamx.common.lifecycle.Lifecycle; + +import javax.net.ssl.SSLContext; +import java.lang.annotation.Annotation; + +/** + */ +public abstract class AbstractHttpClientProvider implements Provider +{ + private final Key> configKey; + private final Key sslContextKey; + + private Provider> configProvider; + private Provider lifecycleProvider; + private Binding sslContextBinding; + + public AbstractHttpClientProvider() + { + configKey = Key.get( + new TypeLiteral>() + { + } + ); + sslContextKey = Key.get(SSLContext.class); + } + + public AbstractHttpClientProvider(Annotation annotation) + { + configKey = Key.get( + new TypeLiteral>() + { + }, annotation + ); + sslContextKey = Key.get(SSLContext.class, annotation); + } + + public AbstractHttpClientProvider(Class annotation) + { + configKey = Key.get( + new TypeLiteral>() + { + }, annotation + ); + sslContextKey = Key.get(SSLContext.class, annotation); + } + + @Inject + public void configure(Injector injector) + { + configProvider = injector.getProvider(configKey); + sslContextBinding = injector.getExistingBinding(sslContextKey); + lifecycleProvider = injector.getProvider(Lifecycle.class); + } + + public Key> getConfigKey() + { + return configKey; + } + + public Key getSslContextKey() + { + return sslContextKey; + } + + public Provider> getConfigProvider() + { + return configProvider; + } + + public Provider getLifecycleProvider() + { + return lifecycleProvider; + } + + public Binding getSslContextBinding() + { + return sslContextBinding; + } +} diff --git a/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java b/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java new file mode 100644 index 00000000000..cf85087c98c --- /dev/null +++ b/server/src/main/java/io/druid/guice/http/DruidHttpClientConfig.java @@ -0,0 +1,49 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.http; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Duration; +import org.joda.time.Period; + +import javax.validation.constraints.Min; + +/** + */ + +public class DruidHttpClientConfig +{ + @JsonProperty + @Min(0) + private int numConnections = 5; + + @JsonProperty + private Period readTimeout = new Period("PT15M"); + + public int getNumConnections() + { + return numConnections; + } + + public Duration getReadTimeout() + { + return readTimeout == null ? null : readTimeout.toStandardDuration(); + } +} diff --git a/server/src/main/java/io/druid/guice/HttpClientModule.java b/server/src/main/java/io/druid/guice/http/HttpClientModule.java similarity index 58% rename from server/src/main/java/io/druid/guice/HttpClientModule.java rename to server/src/main/java/io/druid/guice/http/HttpClientModule.java index 5ba51bfcfa6..2b40b4b1edf 100644 --- a/server/src/main/java/io/druid/guice/HttpClientModule.java +++ b/server/src/main/java/io/druid/guice/http/HttpClientModule.java @@ -17,28 +17,17 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package io.druid.guice; +package io.druid.guice.http; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Supplier; import com.google.inject.Binder; -import com.google.inject.Binding; -import com.google.inject.Inject; -import com.google.inject.Injector; -import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; -import com.google.inject.TypeLiteral; -import com.metamx.common.lifecycle.Lifecycle; import com.metamx.http.client.HttpClient; import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; import io.druid.guice.annotations.Global; -import org.joda.time.Duration; -import org.joda.time.Period; -import javax.net.ssl.SSLContext; -import javax.validation.constraints.Min; import java.lang.annotation.Annotation; /** @@ -80,15 +69,13 @@ public class HttpClientModule implements Module .annotatedWith(annotation) .toProvider(new HttpClientProvider(annotation)) .in(LazySingleton.class); - } - else if (annotationClazz != null) { + } else if (annotationClazz != null) { JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotationClazz); binder.bind(HttpClient.class) .annotatedWith(annotationClazz) .toProvider(new HttpClientProvider(annotationClazz)) .in(LazySingleton.class); - } - else { + } else { JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class); binder.bind(HttpClient.class) .toProvider(new HttpClientProvider()) @@ -96,76 +83,37 @@ public class HttpClientModule implements Module } } - public static class DruidHttpClientConfig + public static class HttpClientProvider extends AbstractHttpClientProvider { - @JsonProperty - @Min(0) - private int numConnections = 5; - - @JsonProperty - private Period readTimeout = new Period("PT15M"); - - public int getNumConnections() - { - return numConnections; - } - - public Duration getReadTimeout() - { - return readTimeout == null ? null : readTimeout.toStandardDuration(); - } - } - - public static class HttpClientProvider implements Provider - { - private final Key> configKey; - private final Key sslContextKey; - - private Provider> configProvider; - private Provider lifecycleProvider; - private Binding sslContextBinding; - public HttpClientProvider() { - configKey = Key.get(new TypeLiteral>(){}); - sslContextKey = Key.get(SSLContext.class); } public HttpClientProvider(Annotation annotation) { - configKey = Key.get(new TypeLiteral>(){}, annotation); - sslContextKey = Key.get(SSLContext.class, annotation); + super(annotation); } - public HttpClientProvider(Class annotation) + public HttpClientProvider(Class annotationClazz) { - configKey = Key.get(new TypeLiteral>(){}, annotation); - sslContextKey = Key.get(SSLContext.class, annotation); - } - - @Inject - public void configure(Injector injector) - { - configProvider = injector.getProvider(configKey); - sslContextBinding = injector.getExistingBinding(sslContextKey); - lifecycleProvider = injector.getProvider(Lifecycle.class); + super(annotationClazz); } @Override public HttpClient get() { - final DruidHttpClientConfig config = configProvider.get().get(); + final DruidHttpClientConfig config = getConfigProvider().get().get(); final HttpClientConfig.Builder builder = HttpClientConfig .builder() .withNumConnections(config.getNumConnections()) .withReadTimeout(config.getReadTimeout()); - if (sslContextBinding != null) { - builder.withSslContext(sslContextBinding.getProvider().get()); + if (getSslContextBinding() != null) { + builder.withSslContext(getSslContextBinding().getProvider().get()); } - return HttpClientInit.createClient(builder.build(), lifecycleProvider.get()); + return HttpClientInit.createClient(builder.build(), getLifecycleProvider().get()); } } } diff --git a/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java b/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java new file mode 100644 index 00000000000..4e77a198873 --- /dev/null +++ b/server/src/main/java/io/druid/guice/http/JettyHttpClientModule.java @@ -0,0 +1,152 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012, 2013 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 io.druid.guice.http; + +import com.google.api.client.repackaged.com.google.common.base.Throwables; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.metamx.common.lifecycle.Lifecycle; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; +import io.druid.guice.annotations.Global; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.util.ssl.SslContextFactory; + +import java.lang.annotation.Annotation; + +/** + */ +public class JettyHttpClientModule implements Module +{ + public static JettyHttpClientModule global() + { + return new JettyHttpClientModule("druid.global.http", Global.class); + } + + private final String propertyPrefix; + private Annotation annotation = null; + private Class annotationClazz = null; + + public JettyHttpClientModule(String propertyPrefix) + { + this.propertyPrefix = propertyPrefix; + } + + public JettyHttpClientModule(String propertyPrefix, Class annotation) + { + this.propertyPrefix = propertyPrefix; + this.annotationClazz = annotation; + } + + public JettyHttpClientModule(String propertyPrefix, Annotation annotation) + { + this.propertyPrefix = propertyPrefix; + this.annotation = annotation; + } + + @Override + public void configure(Binder binder) + { + if (annotation != null) { + JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotation); + binder.bind(HttpClient.class) + .annotatedWith(annotation) + .toProvider(new HttpClientProvider(annotation)) + .in(LazySingleton.class); + } else if (annotationClazz != null) { + JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class, annotationClazz); + binder.bind(HttpClient.class) + .annotatedWith(annotationClazz) + .toProvider(new HttpClientProvider(annotationClazz)) + .in(LazySingleton.class); + } else { + JsonConfigProvider.bind(binder, propertyPrefix, DruidHttpClientConfig.class); + binder.bind(HttpClient.class) + .toProvider(new HttpClientProvider()) + .in(LazySingleton.class); + } + } + + public static class HttpClientProvider extends AbstractHttpClientProvider + { + public HttpClientProvider() + { + } + + public HttpClientProvider(Annotation annotation) + { + super(annotation); + } + + public HttpClientProvider(Class annotation) + { + super(annotation); + } + + @Override + public HttpClient get() + { + final DruidHttpClientConfig config = getConfigProvider().get().get(); + + final HttpClient httpClient; + if (getSslContextBinding() != null) { + final SslContextFactory sslContextFactory = new SslContextFactory(); + sslContextFactory.setSslContext(getSslContextBinding().getProvider().get()); + httpClient = new HttpClient(sslContextFactory); + } else { + httpClient = new HttpClient(); + } + + httpClient.setIdleTimeout(config.getReadTimeout().getMillis()); + httpClient.setMaxConnectionsPerDestination(config.getNumConnections()); + + final Lifecycle lifecycle = getLifecycleProvider().get(); + + try { + lifecycle.addMaybeStartHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + httpClient.start(); + } + + @Override + public void stop() + { + try { + httpClient.stop(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + return httpClient; + } + } +} diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 4a5dd8ab20b..84feb11bbc3 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -40,7 +40,7 @@ import io.druid.guice.DruidProcessingModule; import io.druid.guice.DruidSecondaryModule; import io.druid.guice.ExtensionsConfig; import io.druid.guice.FirehoseModule; -import io.druid.guice.HttpClientModule; +import io.druid.guice.http.HttpClientModule; import io.druid.guice.IndexingServiceDiscoveryModule; import io.druid.guice.JacksonConfigManagerModule; import io.druid.guice.LifecycleModule; diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index f2b715a18f4..5d0981bb1ac 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -20,297 +20,40 @@ package io.druid.server; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.api.client.repackaged.com.google.common.base.Throwables; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; -import com.metamx.http.client.response.ClientResponse; -import com.metamx.http.client.response.HttpResponseHandler; -import io.druid.client.RoutingDruidClient; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; -import io.druid.query.DataSourceUtil; import io.druid.query.Query; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.handler.codec.http.HttpChunk; -import org.jboss.netty.handler.codec.http.HttpHeaders; -import org.jboss.netty.handler.codec.http.HttpResponse; +import io.druid.server.router.Router; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.api.Request; +import org.eclipse.jetty.client.util.BytesContentProvider; +import org.eclipse.jetty.http.HttpHeader; +import org.eclipse.jetty.http.HttpVersion; +import org.eclipse.jetty.proxy.AsyncProxyServlet; import org.joda.time.DateTime; -import javax.annotation.Nullable; import javax.servlet.AsyncContext; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; -import javax.servlet.annotation.WebServlet; -import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.net.URI; -import java.util.Map; -import java.util.UUID; +import java.util.Enumeration; +import java.util.concurrent.TimeUnit; /** * This class does async query processing and should be merged with QueryResource at some point */ -@WebServlet(asyncSupported = true) -public class AsyncQueryForwardingServlet extends HttpServlet +public class AsyncQueryForwardingServlet extends AsyncProxyServlet { private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); - private static final Joiner COMMA_JOIN = Joiner.on(","); - private final ObjectMapper jsonMapper; - private final ObjectMapper smileMapper; - private final QueryHostFinder hostFinder; - private final RoutingDruidClient routingDruidClient; - private final ServiceEmitter emitter; - private final RequestLogger requestLogger; - - public AsyncQueryForwardingServlet( - @Json ObjectMapper jsonMapper, - @Smile ObjectMapper smileMapper, - QueryHostFinder hostFinder, - RoutingDruidClient routingDruidClient, - ServiceEmitter emitter, - RequestLogger requestLogger - ) - { - this.jsonMapper = jsonMapper; - this.smileMapper = smileMapper; - this.hostFinder = hostFinder; - this.routingDruidClient = routingDruidClient; - this.emitter = emitter; - this.requestLogger = requestLogger; - } - - @Override - protected void doGet(HttpServletRequest req, HttpServletResponse res) - throws ServletException, IOException - { - final AsyncContext asyncContext = req.startAsync(); - asyncContext.setTimeout(0); - - final HttpResponseHandler responseHandler = - new PassthroughHttpResponseHandler(res); - - final URI uri = rewriteURI(hostFinder.getDefaultHost(), req); - asyncComplete( - res, - asyncContext, - jsonMapper, - routingDruidClient.get(uri, responseHandler) - ); - } - - @Override - protected void doDelete(HttpServletRequest req, HttpServletResponse res) throws ServletException, IOException - { - final AsyncContext asyncContext = req.startAsync(); - asyncContext.setTimeout(0); - - final HttpResponseHandler responseHandler = - new PassthroughHttpResponseHandler(res); - - final String host = hostFinder.getDefaultHost(); - - asyncComplete( - res, - asyncContext, - jsonMapper, - routingDruidClient.delete(rewriteURI(host, (HttpServletRequest) asyncContext.getRequest()), responseHandler) - ); - } - - @Override - protected void doPost(final HttpServletRequest req, final HttpServletResponse res) throws ServletException, IOException - { - final long start = System.currentTimeMillis(); - final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(req.getContentType()); - final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; - - try { - final Query inputQuery = objectMapper.readValue(req.getInputStream(), Query.class); - if (log.isDebugEnabled()) { - log.debug("Got query [%s]", inputQuery); - } - - final Query query; - if (inputQuery.getId() == null) { - query = inputQuery.withId(UUID.randomUUID().toString()); - } else { - query = inputQuery; - } - - URI rewrittenURI = rewriteURI(hostFinder.getHost(query), req); - - final AsyncContext asyncContext = req.startAsync(); - // let proxy http client timeout - asyncContext.setTimeout(0); - - ListenableFuture future = routingDruidClient.postQuery( - rewrittenURI, - query, - new PassthroughHttpResponseHandler(res) - ); - - Futures.addCallback( - future, - new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object o) - { - final long requestTime = System.currentTimeMillis() - start; - emitter.emit( - new ServiceMetricEvent.Builder() - .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) - .setUser3(String.valueOf(query.getContextPriority(0))) - .setUser4(query.getType()) - .setUser5(COMMA_JOIN.join(query.getIntervals())) - .setUser6(String.valueOf(query.hasFilters())) - .setUser7(req.getRemoteAddr()) - .setUser8(query.getId()) - .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) - .build("request/time", requestTime) - ); - - try { - requestLogger.log( - new RequestLogLine( - new DateTime(), - req.getRemoteAddr(), - query, - new QueryStats( - ImmutableMap.of( - "request/time", - requestTime, - "success", - true - ) - ) - ) - ); - } - catch (Exception e) { - log.error(e, "Unable to log query [%s]!", query); - } - } - - @Override - public void onFailure(Throwable throwable) - { - try { - final String errorMessage = throwable.getMessage(); - requestLogger.log( - new RequestLogLine( - new DateTime(), - req.getRemoteAddr(), - query, - new QueryStats( - ImmutableMap.of( - "success", - false, - "exception", - errorMessage == null ? "no message" : errorMessage) - ) - ) - ); - } - catch (IOException logError) { - log.error(logError, "Unable to log query [%s]!", query); - } - - log.makeAlert(throwable, "Exception handling request [%s]", query.getId()) - .addData("query", query) - .addData("peer", req.getRemoteAddr()) - .emit(); - } - } - ); - - asyncComplete( - res, - asyncContext, - objectMapper, - future - ); - } catch(IOException e) { - log.warn(e, "Exception parsing query"); - final String errorMessage = e.getMessage() == null ? "no error message" : e.getMessage(); - requestLogger.log( - new RequestLogLine( - new DateTime(), - req.getRemoteAddr(), - null, - new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) - ) - ); - res.setStatus(HttpServletResponse.SC_BAD_REQUEST); - objectMapper.writeValue( - res.getOutputStream(), - ImmutableMap.of("error", errorMessage) - ); - } catch(Exception e) { - handleException(res, objectMapper, e); - } - } - - private static void asyncComplete( - final HttpServletResponse res, - final AsyncContext asyncContext, - final ObjectMapper objectMapper, - ListenableFuture future - ) - { - Futures.addCallback( - future, - new FutureCallback() - { - @Override - public void onSuccess(@Nullable Object o) - { - asyncContext.complete(); - } - - @Override - public void onFailure(Throwable throwable) - { - log.error(throwable, "Error processing query response"); - try { - handleException(res, objectMapper, throwable); - } catch(Exception err) { - log.error(err, "Unable to handle exception response"); - } - asyncContext.complete(); - } - } - ); - } - - private URI rewriteURI(final String host, final HttpServletRequest req) - { - final StringBuilder uri = new StringBuilder("http://"); - uri.append(host); - uri.append(req.getRequestURI()); - final String queryString = req.getQueryString(); - if (queryString != null) { - uri.append("?").append(queryString); - } - return URI.create(uri.toString()); - } - - private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Throwable exception) throws IOException + private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Throwable exception) + throws IOException { if (!response.isCommitted()) { final String errorMessage = exception.getMessage() == null ? "null exception" : exception.getMessage(); @@ -325,89 +68,174 @@ public class AsyncQueryForwardingServlet extends HttpServlet response.flushBuffer(); } - private static class PassthroughHttpResponseHandler implements HttpResponseHandler + private final ObjectMapper jsonMapper; + private final ObjectMapper smileMapper; + private final QueryHostFinder hostFinder; + private final HttpClient httpClient; + private final RequestLogger requestLogger; + + public AsyncQueryForwardingServlet( + @Json ObjectMapper jsonMapper, + @Smile ObjectMapper smileMapper, + QueryHostFinder hostFinder, + @Router HttpClient httpClient, + RequestLogger requestLogger + ) { - private final HttpServletResponse response; + this.jsonMapper = jsonMapper; + this.smileMapper = smileMapper; + this.hostFinder = hostFinder; + this.httpClient = httpClient; + this.requestLogger = requestLogger; + } - public PassthroughHttpResponseHandler(HttpServletResponse response) throws IOException - { - this.response = response; + @Override + protected void service(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException + { + final boolean isSmile = QueryResource.APPLICATION_SMILE.equals(request.getContentType()); + final ObjectMapper objectMapper = isSmile ? smileMapper : jsonMapper; + + String host = hostFinder.getDefaultHost(); + Query inputQuery = null; + try { + inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); + if (inputQuery != null) { + host = hostFinder.getHost(inputQuery); + } } - - protected void copyStatusHeaders(HttpResponse clientResponse) - { - response.setStatus(clientResponse.getStatus().getCode()); - response.setContentType(clientResponse.headers().get(HttpHeaders.Names.CONTENT_TYPE)); - - FluentIterable.from(clientResponse.headers().entries()) - .filter(new Predicate>() - { - @Override - public boolean apply(@Nullable Map.Entry input) - { - return input.getKey().startsWith("X-Druid"); - } - } + catch (IOException e) { + log.warn(e, "Exception parsing query"); + final String errorMessage = e.getMessage() == null ? "no error message" : e.getMessage(); + requestLogger.log( + new RequestLogLine( + new DateTime(), + request.getRemoteAddr(), + null, + new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) ) - .transform( - new Function, Object>() - { - @Nullable - @Override - public Object apply(@Nullable Map.Entry input) - { - response.setHeader(input.getKey(), input.getValue()); - return null; - } - } - ) - .allMatch(Predicates.alwaysTrue()); + ); + response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + objectMapper.writeValue( + response.getOutputStream(), + ImmutableMap.of("error", errorMessage) + ); + } + catch (Exception e) { + handleException(response, objectMapper, e); } - @Override - public ClientResponse handleResponse(HttpResponse clientResponse) - { - copyStatusHeaders(clientResponse); + final int requestId = getRequestId(request); - try { - final ServletOutputStream outputStream = response.getOutputStream(); - ChannelBuffer buf = clientResponse.getContent(); - buf.readBytes(outputStream, buf.readableBytes()); - return ClientResponse.unfinished(outputStream); + URI rewrittenURI = rewriteURI(host, request); + + if (_log.isDebugEnabled()) { + StringBuffer uri = request.getRequestURL(); + if (request.getQueryString() != null) { + uri.append("?").append(request.getQueryString()); } - catch (Exception e) { - throw Throwables.propagate(e); + if (_log.isDebugEnabled()) { + _log.debug("{} rewriting: {} -> {}", requestId, uri, rewrittenURI); } } - @Override - public ClientResponse handleChunk( - ClientResponse clientResponse, HttpChunk chunk - ) - { - try { - ChannelBuffer buf = chunk.getContent(); - buf.readBytes(clientResponse.getObj(), buf.readableBytes()); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - return clientResponse; + if (rewrittenURI == null) { + onRewriteFailed(request, response); + return; } - @Override - public ClientResponse done(ClientResponse clientResponse) - { - return ClientResponse.finished(clientResponse.getObj()); + final Request proxyRequest = getHttpClient().newRequest(rewrittenURI) + .method(request.getMethod()) + .version(HttpVersion.fromString(request.getProtocol())); + + // Copy headers + boolean hasContent = request.getContentLength() > 0 || request.getContentType() != null; + for (Enumeration headerNames = request.getHeaderNames(); headerNames.hasMoreElements(); ) { + String headerName = headerNames.nextElement(); + + if (HttpHeader.TRANSFER_ENCODING.is(headerName)) { + hasContent = true; + } + + for (Enumeration headerValues = request.getHeaders(headerName); headerValues.hasMoreElements(); ) { + String headerValue = headerValues.nextElement(); + if (headerValue != null) { + proxyRequest.header(headerName, headerValue); + } + } } - @Override - public void exceptionCaught( - ClientResponse clientResponse, - Throwable e - ) - { - // exceptions are handled on future callback + // Add proxy headers + addViaHeader(proxyRequest); + + addXForwardedHeaders(proxyRequest, request); + + final AsyncContext asyncContext = request.startAsync(); + // We do not timeout the continuation, but the proxy request + asyncContext.setTimeout(0); + proxyRequest.timeout( + getTimeout(), TimeUnit.MILLISECONDS + ); + + if (hasContent && inputQuery != null) { + proxyRequest.content(new BytesContentProvider(jsonMapper.writeValueAsBytes(inputQuery))); } + + customizeProxyRequest(proxyRequest, request); + + if (_log.isDebugEnabled()) { + StringBuilder builder = new StringBuilder(request.getMethod()); + builder.append(" ").append(request.getRequestURI()); + String query = request.getQueryString(); + if (query != null) { + builder.append("?").append(query); + } + builder.append(" ").append(request.getProtocol()).append("\r\n"); + for (Enumeration headerNames = request.getHeaderNames(); headerNames.hasMoreElements(); ) { + String headerName = headerNames.nextElement(); + builder.append(headerName).append(": "); + for (Enumeration headerValues = request.getHeaders(headerName); headerValues.hasMoreElements(); ) { + String headerValue = headerValues.nextElement(); + if (headerValue != null) { + builder.append(headerValue); + } + if (headerValues.hasMoreElements()) { + builder.append(","); + } + } + builder.append("\r\n"); + } + builder.append("\r\n"); + + _log.debug( + "{} proxying to upstream:{}{}{}{}", + requestId, + System.lineSeparator(), + builder, + proxyRequest, + System.lineSeparator(), + proxyRequest.getHeaders().toString().trim() + ); + } + + proxyRequest.send(newProxyResponseListener(request, response)); + } + + @Override + protected HttpClient createHttpClient() throws ServletException + { + return httpClient; + } + + private URI rewriteURI(final String host, final HttpServletRequest req) + { + final StringBuilder uri = new StringBuilder("http://"); + + uri.append(host); + uri.append(req.getRequestURI()); + final String queryString = req.getQueryString(); + if (queryString != null) { + uri.append("?").append(queryString); + } + return URI.create(uri.toString()); } } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 0d38a33a33c..18ba190393f 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -148,16 +148,6 @@ public class QueryResource ); } - if ((boolean) query.getContextValue("b", false)) { - System.out.println("***NEW QUERY***"); - while (true) { - System.out.println("SLEEPING"); - Thread.sleep(10000); - } - } else if ((boolean) query.getContextValue("a", false)) { - return Response.ok("hi").build(); - } - if (log.isDebugEnabled()) { log.debug("Got query [%s]", query); } diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index c9a2f64fb7b..355e1f993a3 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -26,17 +26,15 @@ import com.google.inject.Provides; import com.google.inject.TypeLiteral; import com.metamx.common.logger.Logger; import io.airlift.command.Command; -import io.druid.client.RoutingDruidClient; import io.druid.curator.discovery.DiscoveryModule; import io.druid.curator.discovery.ServerDiscoveryFactory; import io.druid.curator.discovery.ServerDiscoverySelector; -import io.druid.guice.HttpClientModule; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; -import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Self; +import io.druid.guice.http.JettyHttpClientModule; import io.druid.server.initialization.JettyServerInitializer; import io.druid.server.router.CoordinatorRuleManager; import io.druid.server.router.QueryHostFinder; @@ -68,7 +66,7 @@ public class CliRouter extends ServerRunnable protected List getModules() { return ImmutableList.of( - new HttpClientModule("druid.router.http", Router.class), + new JettyHttpClientModule("druid.router.http", Router.class), new Module() { @Override @@ -81,12 +79,10 @@ public class CliRouter extends ServerRunnable binder.bind(TieredBrokerHostSelector.class).in(ManageLifecycle.class); binder.bind(QueryHostFinder.class).in(LazySingleton.class); - binder.bind(RoutingDruidClient.class).in(LazySingleton.class); binder.bind(new TypeLiteral>(){}) .toProvider(TieredBrokerSelectorStrategiesProvider.class) .in(LazySingleton.class); - binder.bind(JettyServerInitializer.class).to(RouterJettyServerInitializer.class).in(LazySingleton.class); LifecycleModule.register(binder, Server.class); diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index 8b329f035e9..b3ec2702d58 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -23,15 +23,14 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; -import com.metamx.emitter.service.ServiceEmitter; -import io.druid.client.RoutingDruidClient; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.server.AsyncQueryForwardingServlet; import io.druid.server.initialization.JettyServerInitializer; -import io.druid.server.initialization.ServerConfig; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; +import io.druid.server.router.Router; +import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -45,31 +44,25 @@ import org.eclipse.jetty.servlets.AsyncGzipFilter; */ public class RouterJettyServerInitializer implements JettyServerInitializer { - private final ServerConfig config; private final ObjectMapper jsonMapper; private final ObjectMapper smileMapper; private final QueryHostFinder hostFinder; - private final RoutingDruidClient routingDruidClient; - private final ServiceEmitter emitter; + private final HttpClient httpClient; private final RequestLogger requestLogger; @Inject public RouterJettyServerInitializer( - ServerConfig config, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, QueryHostFinder hostFinder, - RoutingDruidClient routingDruidClient, - ServiceEmitter emitter, + @Router HttpClient httpClient, RequestLogger requestLogger ) { - this.config = config; this.jsonMapper = jsonMapper; this.smileMapper = smileMapper; this.hostFinder = hostFinder; - this.routingDruidClient = routingDruidClient; - this.emitter = emitter; + this.httpClient = httpClient; this.requestLogger = requestLogger; } @@ -83,8 +76,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer jsonMapper, smileMapper, hostFinder, - routingDruidClient, - emitter, + httpClient, requestLogger ) ), "/druid/v2/*" From 382cde9cf27e684f63285ea96800178c5c33ebf1 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 09:51:53 -0700 Subject: [PATCH 200/270] less verbose logging for LQP --- .../server/coordinator/LoadQueuePeon.java | 72 ++++++++++--------- 1 file changed, 37 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index 8e3c2509697..e2577795a27 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -87,7 +87,7 @@ public class LoadQueuePeon private final Object lock = new Object(); - private volatile SegmentHolder currentlyLoading = null; + private volatile SegmentHolder currentlyProcessing = null; LoadQueuePeon( CuratorFramework curator, @@ -156,10 +156,10 @@ public class LoadQueuePeon ) { synchronized (lock) { - if ((currentlyLoading != null) && - currentlyLoading.getSegmentIdentifier().equals(segment.getIdentifier())) { + if ((currentlyProcessing != null) && + currentlyProcessing.getSegmentIdentifier().equals(segment.getIdentifier())) { if (callback != null) { - currentlyLoading.addCallback(callback); + currentlyProcessing.addCallback(callback); } return; } @@ -170,13 +170,13 @@ public class LoadQueuePeon synchronized (lock) { if (segmentsToLoad.contains(holder)) { if ((callback != null)) { - currentlyLoading.addCallback(callback); + currentlyProcessing.addCallback(callback); } return; } } - log.info("Asking server peon[%s] to load segment[%s]", basePath, segment); + log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getIdentifier()); queuedSize.addAndGet(segment.getSize()); segmentsToLoad.add(holder); doNext(); @@ -188,10 +188,10 @@ public class LoadQueuePeon ) { synchronized (lock) { - if ((currentlyLoading != null) && - currentlyLoading.getSegmentIdentifier().equals(segment.getIdentifier())) { + if ((currentlyProcessing != null) && + currentlyProcessing.getSegmentIdentifier().equals(segment.getIdentifier())) { if (callback != null) { - currentlyLoading.addCallback(callback); + currentlyProcessing.addCallback(callback); } return; } @@ -202,7 +202,7 @@ public class LoadQueuePeon synchronized (lock) { if (segmentsToDrop.contains(holder)) { if (callback != null) { - currentlyLoading.addCallback(callback); + currentlyProcessing.addCallback(callback); } return; } @@ -216,13 +216,13 @@ public class LoadQueuePeon private void doNext() { synchronized (lock) { - if (currentlyLoading == null) { + if (currentlyProcessing == null) { if (!segmentsToDrop.isEmpty()) { - currentlyLoading = segmentsToDrop.first(); - log.info("Server[%s] dropping [%s]", basePath, currentlyLoading); + currentlyProcessing = segmentsToDrop.first(); + log.info("Server[%s] dropping [%s]", basePath, currentlyProcessing.getSegmentIdentifier()); } else if (!segmentsToLoad.isEmpty()) { - currentlyLoading = segmentsToLoad.first(); - log.info("Server[%s] loading [%s]", basePath, currentlyLoading); + currentlyProcessing = segmentsToLoad.first(); + log.info("Server[%s] loading [%s]", basePath, currentlyProcessing.getSegmentIdentifier()); } else { return; } @@ -235,16 +235,16 @@ public class LoadQueuePeon { synchronized (lock) { try { - if (currentlyLoading == null) { + if (currentlyProcessing == null) { log.makeAlert("Crazy race condition! server[%s]", basePath) .emit(); actionCompleted(); doNext(); return; } - log.info("Server[%s] adding segment[%s]", basePath, currentlyLoading.getSegmentIdentifier()); - final String path = ZKPaths.makePath(basePath, currentlyLoading.getSegmentIdentifier()); - final byte[] payload = jsonMapper.writeValueAsBytes(currentlyLoading.getChangeRequest()); + log.info("Server[%s] processing segment[%s]", basePath, currentlyProcessing.getSegmentIdentifier()); + final String path = ZKPaths.makePath(basePath, currentlyProcessing.getSegmentIdentifier()); + final byte[] payload = jsonMapper.writeValueAsBytes(currentlyProcessing.getChangeRequest()); curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); zkWritingExecutor.schedule( @@ -255,7 +255,7 @@ public class LoadQueuePeon { try { if (curator.checkExists().forPath(path) != null) { - failAssign(new ISE("%s was never removed! Failing this assign!", path)); + failAssign(new ISE("%s was never removed! Failing this operation!", path)); } } catch (Exception e) { @@ -311,7 +311,9 @@ public class LoadQueuePeon ); } else { log.info( - "Server[%s] skipping doNext() because something is currently loading[%s].", basePath, currentlyLoading + "Server[%s] skipping doNext() because something is currently loading[%s].", + basePath, + currentlyProcessing.getSegmentIdentifier() ); } } @@ -319,29 +321,29 @@ public class LoadQueuePeon private void actionCompleted() { - if (currentlyLoading != null) { - switch (currentlyLoading.getType()) { + if (currentlyProcessing != null) { + switch (currentlyProcessing.getType()) { case LOAD: - segmentsToLoad.remove(currentlyLoading); - queuedSize.addAndGet(-currentlyLoading.getSegmentSize()); + segmentsToLoad.remove(currentlyProcessing); + queuedSize.addAndGet(-currentlyProcessing.getSegmentSize()); break; case DROP: - segmentsToDrop.remove(currentlyLoading); + segmentsToDrop.remove(currentlyProcessing); break; default: throw new UnsupportedOperationException(); } - currentlyLoading.executeCallbacks(); - currentlyLoading = null; + currentlyProcessing.executeCallbacks(); + currentlyProcessing = null; } } public void stop() { synchronized (lock) { - if (currentlyLoading != null) { - currentlyLoading.executeCallbacks(); - currentlyLoading = null; + if (currentlyProcessing != null) { + currentlyProcessing.executeCallbacks(); + currentlyProcessing = null; } if (!segmentsToDrop.isEmpty()) { @@ -366,14 +368,14 @@ public class LoadQueuePeon private void entryRemoved(String path) { synchronized (lock) { - if (currentlyLoading == null) { + if (currentlyProcessing == null) { log.warn("Server[%s] an entry[%s] was removed even though it wasn't loading!?", basePath, path); return; } - if (!ZKPaths.getNodeFromPath(path).equals(currentlyLoading.getSegmentIdentifier())) { + if (!ZKPaths.getNodeFromPath(path).equals(currentlyProcessing.getSegmentIdentifier())) { log.warn( "Server[%s] entry [%s] was removed even though it's not what is currently loading[%s]", - basePath, path, currentlyLoading + basePath, path, currentlyProcessing ); return; } @@ -387,7 +389,7 @@ public class LoadQueuePeon private void failAssign(Exception e) { synchronized (lock) { - log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyLoading); + log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyProcessing); failedAssignCount.getAndIncrement(); // Act like it was completed so that the coordinator gives it to someone else actionCompleted(); From 2ad1bd3f4455615e3642994ab836138ba1fa42d9 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 10:22:55 -0700 Subject: [PATCH 201/270] add back metrics --- .../java/io/druid/query/DataSourceUtil.java | 4 + .../server/AsyncQueryForwardingServlet.java | 170 +++++++++++++++--- .../cli/RouterJettyServerInitializer.java | 5 + 3 files changed, 152 insertions(+), 27 deletions(-) diff --git a/processing/src/main/java/io/druid/query/DataSourceUtil.java b/processing/src/main/java/io/druid/query/DataSourceUtil.java index 109db95ee04..7c056f1eabc 100644 --- a/processing/src/main/java/io/druid/query/DataSourceUtil.java +++ b/processing/src/main/java/io/druid/query/DataSourceUtil.java @@ -19,10 +19,14 @@ package io.druid.query; +import com.google.common.base.Joiner; + import java.util.List; public class DataSourceUtil { + public static final Joiner COMMA_JOIN = Joiner.on(","); + public static String getMetricName(DataSource dataSource) { final List names = dataSource.getNames(); diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index 5d0981bb1ac..b485fa2ce69 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -22,16 +22,22 @@ package io.druid.server; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; +import io.druid.query.DataSourceUtil; import io.druid.query.Query; import io.druid.server.log.RequestLogger; import io.druid.server.router.QueryHostFinder; import io.druid.server.router.Router; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.api.Request; +import org.eclipse.jetty.client.api.Response; +import org.eclipse.jetty.client.api.Result; import org.eclipse.jetty.client.util.BytesContentProvider; import org.eclipse.jetty.http.HttpHeader; +import org.eclipse.jetty.http.HttpMethod; import org.eclipse.jetty.http.HttpVersion; import org.eclipse.jetty.proxy.AsyncProxyServlet; import org.joda.time.DateTime; @@ -72,6 +78,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet private final ObjectMapper smileMapper; private final QueryHostFinder hostFinder; private final HttpClient httpClient; + private final ServiceEmitter emitter; private final RequestLogger requestLogger; public AsyncQueryForwardingServlet( @@ -79,6 +86,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet @Smile ObjectMapper smileMapper, QueryHostFinder hostFinder, @Router HttpClient httpClient, + ServiceEmitter emitter, RequestLogger requestLogger ) { @@ -86,6 +94,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet this.smileMapper = smileMapper; this.hostFinder = hostFinder; this.httpClient = httpClient; + this.emitter = emitter; this.requestLogger = requestLogger; } @@ -97,31 +106,37 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet String host = hostFinder.getDefaultHost(); Query inputQuery = null; - try { - inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); - if (inputQuery != null) { - host = hostFinder.getHost(inputQuery); + boolean hasContent = request.getContentLength() > 0 || request.getContentType() != null; + long startTime = System.currentTimeMillis(); + + // queries only exist for POST + if (request.getMethod().equalsIgnoreCase(HttpMethod.POST.asString())) { + try { + inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); + if (inputQuery != null) { + host = hostFinder.getHost(inputQuery); + } + } + catch (IOException e) { + log.warn(e, "Exception parsing query"); + final String errorMessage = e.getMessage() == null ? "no error message" : e.getMessage(); + requestLogger.log( + new RequestLogLine( + new DateTime(), + request.getRemoteAddr(), + null, + new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) + ) + ); + response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + objectMapper.writeValue( + response.getOutputStream(), + ImmutableMap.of("error", errorMessage) + ); + } + catch (Exception e) { + handleException(response, objectMapper, e); } - } - catch (IOException e) { - log.warn(e, "Exception parsing query"); - final String errorMessage = e.getMessage() == null ? "no error message" : e.getMessage(); - requestLogger.log( - new RequestLogLine( - new DateTime(), - request.getRemoteAddr(), - null, - new QueryStats(ImmutableMap.of("success", false, "exception", errorMessage)) - ) - ); - response.setStatus(HttpServletResponse.SC_BAD_REQUEST); - objectMapper.writeValue( - response.getOutputStream(), - ImmutableMap.of("error", errorMessage) - ); - } - catch (Exception e) { - handleException(response, objectMapper, e); } final int requestId = getRequestId(request); @@ -148,7 +163,6 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet .version(HttpVersion.fromString(request.getProtocol())); // Copy headers - boolean hasContent = request.getContentLength() > 0 || request.getContentType() != null; for (Enumeration headerNames = request.getHeaderNames(); headerNames.hasMoreElements(); ) { String headerName = headerNames.nextElement(); @@ -176,8 +190,12 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet getTimeout(), TimeUnit.MILLISECONDS ); - if (hasContent && inputQuery != null) { - proxyRequest.content(new BytesContentProvider(jsonMapper.writeValueAsBytes(inputQuery))); + if (hasContent) { + if (inputQuery != null) { + proxyRequest.content(new BytesContentProvider(jsonMapper.writeValueAsBytes(inputQuery))); + } else { + proxyRequest.content(proxyRequestContent(proxyRequest, request)); + } } customizeProxyRequest(proxyRequest, request); @@ -238,4 +256,102 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet } return URI.create(uri.toString()); } + + private class MetricsEmittingProxyResponseListener extends ProxyResponseListener + { + private final HttpServletRequest req; + private final HttpServletResponse res; + private final Query query; + private final long start; + + public MetricsEmittingProxyResponseListener( + HttpServletRequest request, + HttpServletResponse response, + Query query, + long start + ) + { + super(request, response); + + this.req = request; + this.res = response; + this.query = query; + this.start = start; + } + + @Override + public void onComplete(Result result) + { + final long requestTime = System.currentTimeMillis() - start; + emitter.emit( + new ServiceMetricEvent.Builder() + .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) + .setUser3(String.valueOf(query.getContextPriority(0))) + .setUser4(query.getType()) + .setUser5(DataSourceUtil.COMMA_JOIN.join(query.getIntervals())) + .setUser6(String.valueOf(query.hasFilters())) + .setUser7(req.getRemoteAddr()) + .setUser8(query.getId()) + .setUser9(query.getDuration().toPeriod().toStandardMinutes().toString()) + .build("request/time", requestTime) + ); + + try { + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "request/time", + requestTime, + "success", + true + ) + ) + ) + ); + } + catch (Exception e) { + log.error(e, "Unable to log query [%s]!", query); + } + + super.onComplete(result); + } + + @Override + public void onFailure(Response response, Throwable failure) + { + try { + final String errorMessage = failure.getMessage(); + requestLogger.log( + new RequestLogLine( + new DateTime(), + req.getRemoteAddr(), + query, + new QueryStats( + ImmutableMap.of( + "success", + false, + "exception", + errorMessage == null ? "no message" : errorMessage + ) + ) + ) + ); + } + catch (IOException logError) { + log.error(logError, "Unable to log query [%s]!", query); + } + + log.makeAlert(failure, "Exception handling request") + .addData("exception", failure.toString()) + .addData("query", query) + .addData("peer", req.getRemoteAddr()) + .emit(); + + super.onFailure(response, failure); + } + } } diff --git a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java index b3ec2702d58..72db25e758a 100644 --- a/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/RouterJettyServerInitializer.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; +import com.metamx.emitter.service.ServiceEmitter; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.server.AsyncQueryForwardingServlet; @@ -48,6 +49,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer private final ObjectMapper smileMapper; private final QueryHostFinder hostFinder; private final HttpClient httpClient; + private final ServiceEmitter emitter; private final RequestLogger requestLogger; @Inject @@ -56,6 +58,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer @Smile ObjectMapper smileMapper, QueryHostFinder hostFinder, @Router HttpClient httpClient, + ServiceEmitter emitter, RequestLogger requestLogger ) { @@ -63,6 +66,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer this.smileMapper = smileMapper; this.hostFinder = hostFinder; this.httpClient = httpClient; + this.emitter = emitter; this.requestLogger = requestLogger; } @@ -77,6 +81,7 @@ public class RouterJettyServerInitializer implements JettyServerInitializer smileMapper, hostFinder, httpClient, + emitter, requestLogger ) ), "/druid/v2/*" From 41db4b1063f287f64f4d8a6af53d239ee5c70623 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 10:28:51 -0700 Subject: [PATCH 202/270] actually use the metrics emitting listener --- .../server/AsyncQueryForwardingServlet.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index b485fa2ce69..ae379efb1e0 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -107,10 +107,11 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet String host = hostFinder.getDefaultHost(); Query inputQuery = null; boolean hasContent = request.getContentLength() > 0 || request.getContentType() != null; + boolean isQuery = request.getMethod().equalsIgnoreCase(HttpMethod.POST.asString()); long startTime = System.currentTimeMillis(); // queries only exist for POST - if (request.getMethod().equalsIgnoreCase(HttpMethod.POST.asString())) { + if (isQuery) { try { inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); if (inputQuery != null) { @@ -235,7 +236,11 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet ); } - proxyRequest.send(newProxyResponseListener(request, response)); + if (isQuery) { + proxyRequest.send(newMetricsEmittingProxyResponseListener(request, response, inputQuery, startTime)); + } else { + proxyRequest.send(newProxyResponseListener(request, response)); + } } @Override @@ -257,6 +262,17 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet return URI.create(uri.toString()); } + private Response.Listener newMetricsEmittingProxyResponseListener( + HttpServletRequest request, + HttpServletResponse response, + Query query, + long start + ) + { + return new MetricsEmittingProxyResponseListener(request, response, query, start); + } + + private class MetricsEmittingProxyResponseListener extends ProxyResponseListener { private final HttpServletRequest req; From d9c9190e2f6b26d2dc7e662b9a128d2dd8a8d6b2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 11:36:29 -0700 Subject: [PATCH 203/270] fix breakage in batch ingestion schema --- docs/content/Batch-ingestion.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index 057927e5596..f1c5a814987 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -83,7 +83,7 @@ The interval is the [ISO8601 interval](http://en.wikipedia.org/wiki/ISO_8601#Tim "type": "hashed" "targetPartitionSize": 5000000 }, - "metadataUpdateSpec": { + "updaterJobSpec": { "type": "db", "connectURI": "jdbc:mysql:\/\/localhost:7980\/test_db", "user": "username", From 0be5d27f2aa697cd1fb3e0e1512dab91b04ec701 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 11:40:35 -0700 Subject: [PATCH 204/270] another docs fix --- docs/content/Batch-ingestion.md | 2 +- docs/content/Tasks.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/Batch-ingestion.md b/docs/content/Batch-ingestion.md index f1c5a814987..2f53eb48b4e 100644 --- a/docs/content/Batch-ingestion.md +++ b/docs/content/Batch-ingestion.md @@ -272,7 +272,7 @@ The schema of the Hadoop Index Task contains a task "type" and a Hadoop Index Co |config|A Hadoop Index Config (see above).|yes| |hadoopCoordinates|The Maven `::` of Hadoop to use. The default is "org.apache.hadoop:hadoop-core:1.0.3".|no| -The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `metadataUpdateSpec`. The Indexing Service takes care of setting these fields internally. +The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `updaterJobSpec`. The Indexing Service takes care of setting these fields internally. To run the task: diff --git a/docs/content/Tasks.md b/docs/content/Tasks.md index 868e75efe88..6c5df35d25f 100644 --- a/docs/content/Tasks.md +++ b/docs/content/Tasks.md @@ -77,7 +77,7 @@ The Hadoop Index Task is used to index larger data sets that require the paralle |hadoopCoordinates|The Maven \:\:\ of Hadoop to use. The default is "org.apache.hadoop:hadoop-client:2.3.0".|no| -The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `metadataUpdateSpec`. The Indexing Service takes care of setting these fields internally. +The Hadoop Index Config submitted as part of an Hadoop Index Task is identical to the Hadoop Index Config used by the `HadoopBatchIndexer` except that three fields must be omitted: `segmentOutputPath`, `workingPath`, `updaterJobSpec`. The Indexing Service takes care of setting these fields internally. #### Using your own Hadoop distribution From 4a417cbcfcd8c8117145e79b395a961880f28881 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 11:49:06 -0700 Subject: [PATCH 205/270] clean up some logging in Zk coordinator --- .../server/coordination/BaseZkCoordinator.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java index 33d3118e304..bf77cdf9987 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -103,14 +103,14 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler switch (event.getType()) { case CHILD_ADDED: final String path = child.getPath(); - final DataSegmentChangeRequest segment = jsonMapper.readValue( + final DataSegmentChangeRequest request = jsonMapper.readValue( child.getData(), DataSegmentChangeRequest.class ); - log.info("New node[%s] with segmentClass[%s]", path, segment.getClass()); + log.info("New node[%s] with segmentClass[%s]", path, request.getClass()); try { - segment.go( + request.go( getDataSegmentChangeHandler(), new DataSegmentChangeCallback() { @@ -122,11 +122,17 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler try { if (!hasRun) { curator.delete().guaranteed().forPath(path); - log.info("Completed processing for node[%s]", path); + log.info("Completed processing and removing entry for node[%s]", path); hasRun = true; } } catch (Exception e) { + try { + curator.delete().guaranteed().forPath(path); + } + catch (Exception e1) { + log.info(e1, "Failed to delete node[%s], but ignoring exception.", path); + } throw Throwables.propagate(e); } } @@ -143,13 +149,13 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler log.makeAlert(e, "Segment load/unload: uncaught exception.") .addData("node", path) - .addData("nodeProperties", segment) + .addData("nodeProperties", request) .emit(); } break; case CHILD_REMOVED: - log.info("%s was removed", event.getData().getPath()); + log.info("[%s] was removed", event.getData().getPath()); break; default: log.info("Ignoring event[%s]", event); From b09517319d5aa07d56161b9c5936328625dd9230 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 13:23:12 -0700 Subject: [PATCH 206/270] remove debug code --- .../server/AsyncQueryForwardingServlet.java | 50 +------------------ 1 file changed, 1 insertion(+), 49 deletions(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index ae379efb1e0..e961cd453fe 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -107,7 +107,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet String host = hostFinder.getDefaultHost(); Query inputQuery = null; boolean hasContent = request.getContentLength() > 0 || request.getContentType() != null; - boolean isQuery = request.getMethod().equalsIgnoreCase(HttpMethod.POST.asString()); + boolean isQuery = request.getMethod().equals(HttpMethod.POST.asString()); long startTime = System.currentTimeMillis(); // queries only exist for POST @@ -140,20 +140,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet } } - final int requestId = getRequestId(request); - URI rewrittenURI = rewriteURI(host, request); - - if (_log.isDebugEnabled()) { - StringBuffer uri = request.getRequestURL(); - if (request.getQueryString() != null) { - uri.append("?").append(request.getQueryString()); - } - if (_log.isDebugEnabled()) { - _log.debug("{} rewriting: {} -> {}", requestId, uri, rewrittenURI); - } - } - if (rewrittenURI == null) { onRewriteFailed(request, response); return; @@ -201,41 +188,6 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet customizeProxyRequest(proxyRequest, request); - if (_log.isDebugEnabled()) { - StringBuilder builder = new StringBuilder(request.getMethod()); - builder.append(" ").append(request.getRequestURI()); - String query = request.getQueryString(); - if (query != null) { - builder.append("?").append(query); - } - builder.append(" ").append(request.getProtocol()).append("\r\n"); - for (Enumeration headerNames = request.getHeaderNames(); headerNames.hasMoreElements(); ) { - String headerName = headerNames.nextElement(); - builder.append(headerName).append(": "); - for (Enumeration headerValues = request.getHeaders(headerName); headerValues.hasMoreElements(); ) { - String headerValue = headerValues.nextElement(); - if (headerValue != null) { - builder.append(headerValue); - } - if (headerValues.hasMoreElements()) { - builder.append(","); - } - } - builder.append("\r\n"); - } - builder.append("\r\n"); - - _log.debug( - "{} proxying to upstream:{}{}{}{}", - requestId, - System.lineSeparator(), - builder, - proxyRequest, - System.lineSeparator(), - proxyRequest.getHeaders().toString().trim() - ); - } - if (isQuery) { proxyRequest.send(newMetricsEmittingProxyResponseListener(request, response, inputQuery, startTime)); } else { From 088e2c5af33210692a240b2b9dc1b0cdd3c6a6d6 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 13:36:28 -0700 Subject: [PATCH 207/270] address cr --- .../java/io/druid/server/AsyncQueryForwardingServlet.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index e961cd453fe..cb2faa7acee 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -58,7 +58,7 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet { private static final EmittingLogger log = new EmittingLogger(AsyncQueryForwardingServlet.class); - private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Throwable exception) + private static void handleException(HttpServletResponse response, ObjectMapper objectMapper, Exception exception) throws IOException { if (!response.isCommitted()) { @@ -72,6 +72,8 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet ); } response.flushBuffer(); + + throw new IOException(exception); } private final ObjectMapper jsonMapper; @@ -134,6 +136,8 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet response.getOutputStream(), ImmutableMap.of("error", errorMessage) ); + + throw e; } catch (Exception e) { handleException(response, objectMapper, e); From 25a5a0fd33233343d1398d4572e3b5682a6f93da Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 29 Jul 2014 13:54:36 -0700 Subject: [PATCH 208/270] fix unnecessary error messages --- .../java/io/druid/server/AsyncQueryForwardingServlet.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index cb2faa7acee..f284a276202 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -72,8 +72,6 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet ); } response.flushBuffer(); - - throw new IOException(exception); } private final ObjectMapper jsonMapper; @@ -132,15 +130,17 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet ) ); response.setStatus(HttpServletResponse.SC_BAD_REQUEST); + response.setContentType(QueryResource.APPLICATION_JSON); objectMapper.writeValue( response.getOutputStream(), ImmutableMap.of("error", errorMessage) ); - throw e; + return; } catch (Exception e) { handleException(response, objectMapper, e); + return; } } From 1ac84fcd009675c5d26dd032fcc573952a03d018 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 14:07:19 -0700 Subject: [PATCH 209/270] some more logging cleanup for zk coordination --- .../server/coordination/BaseZkCoordinator.java | 13 +++++++------ .../coordination/DataSegmentChangeRequest.java | 2 ++ .../coordination/SegmentChangeRequestDrop.java | 6 ++++++ .../coordination/SegmentChangeRequestLoad.java | 6 ++++++ .../coordination/SegmentChangeRequestNoop.java | 6 ++++++ 5 files changed, 27 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java index bf77cdf9987..d489d74b445 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -72,7 +72,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler return; } - log.info("Starting zkCoordinator for server[%s]", me); + log.info("Starting zkCoordinator for server[%s]", me.getName()); final String loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName()); final String servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName()); @@ -107,7 +107,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler child.getData(), DataSegmentChangeRequest.class ); - log.info("New node[%s] with segmentClass[%s]", path, request.getClass()); + log.info("New node[%s] with request[%s]", path, request.asString()); try { request.go( @@ -122,7 +122,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler try { if (!hasRun) { curator.delete().guaranteed().forPath(path); - log.info("Completed processing and removing entry for node[%s]", path); + log.info("Completed request and deleting node[%s]", path); hasRun = true; } } @@ -131,8 +131,9 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler curator.delete().guaranteed().forPath(path); } catch (Exception e1) { - log.info(e1, "Failed to delete node[%s], but ignoring exception.", path); + log.error(e1, "Failed to delete node[%s], but ignoring exception.", path); } + log.error(e, "Exception while removing node[%s]", path); throw Throwables.propagate(e); } } @@ -144,7 +145,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler curator.delete().guaranteed().forPath(path); } catch (Exception e1) { - log.info(e1, "Failed to delete node[%s], but ignoring exception.", path); + log.error(e1, "Failed to delete node[%s], but ignoring exception.", path); } log.makeAlert(e, "Segment load/unload: uncaught exception.") @@ -155,7 +156,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler break; case CHILD_REMOVED: - log.info("[%s] was removed", event.getData().getPath()); + log.info("node[%s] was removed", event.getData().getPath()); break; default: log.info("Ignoring event[%s]", event); diff --git a/server/src/main/java/io/druid/server/coordination/DataSegmentChangeRequest.java b/server/src/main/java/io/druid/server/coordination/DataSegmentChangeRequest.java index c5fa72e6bc0..be18720d064 100644 --- a/server/src/main/java/io/druid/server/coordination/DataSegmentChangeRequest.java +++ b/server/src/main/java/io/druid/server/coordination/DataSegmentChangeRequest.java @@ -33,4 +33,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; public interface DataSegmentChangeRequest { public void go(DataSegmentChangeHandler handler, DataSegmentChangeCallback callback); + + public String asString(); } diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java index 2514f22871e..a64a51e7d92 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java @@ -51,6 +51,12 @@ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest handler.removeSegment(segment, callback); } + @Override + public String asString() + { + return String.format("load %s", segment.getIdentifier()); + } + @Override public String toString() { diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java index 11eba9684c0..76d539dcc59 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java @@ -51,6 +51,12 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest return segment; } + @Override + public String asString() + { + return String.format("drop %s", segment.getIdentifier()); + } + @Override public String toString() { diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java index 7c51f3f0b6f..454d8617044 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java @@ -28,4 +28,10 @@ public class SegmentChangeRequestNoop implements DataSegmentChangeRequest { // do nothing } + + @Override + public String asString() + { + return "noop"; + } } From c46965c52ce058a1d7ed1567506abd7ea15e650a Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 14:13:37 -0700 Subject: [PATCH 210/270] [maven-release-plugin] prepare release druid-0.6.131 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 15 files changed, 18 insertions(+), 20 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 74e85d6107c..d0f02098705 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/common/pom.xml b/common/pom.xml index 9a198ed9cb3..783503e97c8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/examples/pom.xml b/examples/pom.xml index 0f5c345c3e4..59deede379c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index cb9dd2096a6..ce9c5d82eac 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/histogram/pom.xml b/histogram/pom.xml index 25b7979ae0d..94f6f17155c 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 47310f2d66f..542bd4a1675 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index b8325b3b1db..27fc2b7a6cd 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 955d942fa41..a377ca5fe12 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index b24b37598ee..df7e0d179bf 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/pom.xml b/pom.xml index 7540bf6ad89..5e30fe736c9 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.131-SNAPSHOT + 0.6.131 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.131 diff --git a/processing/pom.xml b/processing/pom.xml index c90dd961c25..63c4aa6bf15 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index aea2a950435..f4e96c96bba 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 6421a24401e..3631e62a3ba 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/server/pom.xml b/server/pom.xml index 80b07742526..8c529668b52 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 diff --git a/services/pom.xml b/services/pom.xml index af3501d21f4..1c3d2c69a62 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.131-SNAPSHOT + 0.6.131 From e1d87ed5ea9de4df777a06dd3d88065a8dd71106 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 14:13:41 -0700 Subject: [PATCH 211/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index d0f02098705..5a9e5ca67e5 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 783503e97c8..9f42c097757 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 59deede379c..dc6c87cb0c0 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index ce9c5d82eac..9746c242f75 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 94f6f17155c..cd2826ae4ad 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 542bd4a1675..5a2117bab18 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 27fc2b7a6cd..a6d9ac4e895 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index a377ca5fe12..bd424e7a8a3 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index df7e0d179bf..ec8b50dbeaf 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/pom.xml b/pom.xml index 5e30fe736c9..69d6f2f7db9 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.131 + 0.6.132-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 63c4aa6bf15..9b0d491d41e 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index f4e96c96bba..81f38e849ca 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 3631e62a3ba..318ea5abe13 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 8c529668b52..b715f4e9688 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 1c3d2c69a62..77a95f34b02 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.131 + 0.6.132-SNAPSHOT From dc38677454ced1c29d7fdf1a480dad863c8ca3b3 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 14:23:50 -0700 Subject: [PATCH 212/270] remove unneeded dependency --- pom.xml | 5 ----- server/pom.xml | 4 ---- 2 files changed, 9 deletions(-) diff --git a/pom.xml b/pom.xml index 69d6f2f7db9..48bc6ba8db1 100644 --- a/pom.xml +++ b/pom.xml @@ -336,11 +336,6 @@ jetty-servlets 9.2.2.v20140723 - - org.eclipse.jetty - jetty-client - 9.2.2.v20140723 - org.eclipse.jetty jetty-proxy diff --git a/server/pom.xml b/server/pom.xml index b715f4e9688..3bcfa6a9c86 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -101,10 +101,6 @@ org.eclipse.jetty jetty-server - - org.eclipse.jetty - jetty-client - org.eclipse.jetty jetty-proxy From 05a09d255071a28edee4027873b7b79885b5fcc8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Tue, 29 Jul 2014 15:13:23 -0700 Subject: [PATCH 213/270] allow overriding coordinator static resource --- .../coordinator/DruidCoordinatorConfig.java | 6 ++++++ .../main/java/io/druid/cli/CliCoordinator.java | 4 +++- .../cli/CoordinatorJettyServerInitializer.java | 18 +++++++++++++++++- 3 files changed, 26 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java index d53d53cf664..571e70ee3bd 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorConfig.java @@ -59,4 +59,10 @@ public abstract class DruidCoordinatorConfig { return new Duration(15 * 60 * 1000); } + + @Config("druid.coordinator.console.static") + public String getConsoleStatic() + { + return null; + } } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 1c551c3e402..f50eb538ce8 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -108,7 +108,9 @@ public class CliCoordinator extends ServerRunnable LifecycleModule.register(binder, DruidCoordinator.class); - binder.bind(JettyServerInitializer.class).toInstance(new CoordinatorJettyServerInitializer()); + binder.bind(JettyServerInitializer.class) + .to(CoordinatorJettyServerInitializer.class); + Jerseys.addResource(binder, BackwardsCompatibleInfoResource.class); Jerseys.addResource(binder, InfoResource.class); Jerseys.addResource(binder, BackwardsCompatibleCoordinatorResource.class); diff --git a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java index 5f1ad32edd7..099f742d560 100644 --- a/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java +++ b/services/src/main/java/io/druid/cli/CoordinatorJettyServerInitializer.java @@ -19,9 +19,11 @@ package io.druid.cli; +import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; import io.druid.server.coordinator.DruidCoordinator; +import io.druid.server.coordinator.DruidCoordinatorConfig; import io.druid.server.http.RedirectFilter; import io.druid.server.initialization.JettyServerInitializer; import org.eclipse.jetty.server.Handler; @@ -32,11 +34,20 @@ import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; import org.eclipse.jetty.servlets.GzipFilter; +import org.eclipse.jetty.util.resource.Resource; /** */ class CoordinatorJettyServerInitializer implements JettyServerInitializer { + private final DruidCoordinatorConfig config; + + @Inject + CoordinatorJettyServerInitializer(DruidCoordinatorConfig config) + { + this.config = config; + } + @Override public void initialize(Server server, Injector injector) { @@ -45,7 +56,12 @@ class CoordinatorJettyServerInitializer implements JettyServerInitializer ServletHolder holderPwd = new ServletHolder("default", DefaultServlet.class); root.addServlet(holderPwd, "/"); - root.setResourceBase(DruidCoordinator.class.getClassLoader().getResource("static").toExternalForm()); + if(config.getConsoleStatic() == null) { + root.setBaseResource(Resource.newClassPathResource("static")); + } else { + root.setResourceBase(config.getConsoleStatic()); + } + root.addFilter(new FilterHolder(injector.getInstance(RedirectFilter.class)), "/*", null); root.addFilter(GzipFilter.class, "/*", null); From 1586d60ec26b5aec2bc940602ec8f65e9ff25c72 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 16:14:29 -0700 Subject: [PATCH 214/270] fix bugs in log printing --- .../io/druid/server/coordination/BaseZkCoordinator.java | 6 +++--- .../druid/server/coordination/SegmentChangeRequestDrop.java | 2 +- .../druid/server/coordination/SegmentChangeRequestLoad.java | 2 +- .../druid/server/coordination/SegmentChangeRequestNoop.java | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java index d489d74b445..e751041747b 100644 --- a/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java +++ b/server/src/main/java/io/druid/server/coordination/BaseZkCoordinator.java @@ -107,7 +107,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler child.getData(), DataSegmentChangeRequest.class ); - log.info("New node[%s] with request[%s]", path, request.asString()); + log.info("New request[%s] with node[%s].", request.asString(), path); try { request.go( @@ -122,7 +122,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler try { if (!hasRun) { curator.delete().guaranteed().forPath(path); - log.info("Completed request and deleting node[%s]", path); + log.info("Completed request [%s]", request.asString()); hasRun = true; } } @@ -156,7 +156,7 @@ public abstract class BaseZkCoordinator implements DataSegmentChangeHandler break; case CHILD_REMOVED: - log.info("node[%s] was removed", event.getData().getPath()); + log.info("Node[%s] was removed", event.getData().getPath()); break; default: log.info("Ignoring event[%s]", event); diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java index a64a51e7d92..5da8ff90058 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestDrop.java @@ -54,7 +54,7 @@ public class SegmentChangeRequestDrop implements DataSegmentChangeRequest @Override public String asString() { - return String.format("load %s", segment.getIdentifier()); + return String.format("DROP: %s", segment.getIdentifier()); } @Override diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java index 76d539dcc59..3142c2804f8 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestLoad.java @@ -54,7 +54,7 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest @Override public String asString() { - return String.format("drop %s", segment.getIdentifier()); + return String.format("LOAD: %s", segment.getIdentifier()); } @Override diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java index 454d8617044..a5f254b25e3 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java @@ -32,6 +32,6 @@ public class SegmentChangeRequestNoop implements DataSegmentChangeRequest @Override public String asString() { - return "noop"; + return "NOOP"; } } From c89b5a385b9c9ad77988513e5c3f01bbbd74ec74 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 16:17:44 -0700 Subject: [PATCH 215/270] cleanup s3 pulling logs --- .../main/java/io/druid/storage/s3/S3DataSegmentPuller.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java index a40e524e94f..f9b68a05440 100644 --- a/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java +++ b/s3-extensions/src/main/java/io/druid/storage/s3/S3DataSegmentPuller.java @@ -104,7 +104,12 @@ public class S3DataSegmentPuller implements DataSegmentPuller } else { ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); } - log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); + log.info( + "Pull of file[%s/%s] completed in %,d millis", + s3Obj.getBucketName(), + s3Obj.getKey(), + System.currentTimeMillis() - startTime + ); return null; } catch (IOException e) { From 877bd98667dca59c773d15625ed5667540118aa8 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 16:20:10 -0700 Subject: [PATCH 216/270] [maven-release-plugin] prepare release druid-0.6.132 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 5a9e5ca67e5..ef78700c525 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/common/pom.xml b/common/pom.xml index 9f42c097757..b5dbe41a23c 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/examples/pom.xml b/examples/pom.xml index dc6c87cb0c0..3e5842cb1ba 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 9746c242f75..5f9e306496a 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/histogram/pom.xml b/histogram/pom.xml index cd2826ae4ad..7d88ddb363a 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 5a2117bab18..e99d03587ac 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a6d9ac4e895..37b97000df4 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index bd424e7a8a3..4ae30024563 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index ec8b50dbeaf..c2d812baebe 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/pom.xml b/pom.xml index 48bc6ba8db1..3562db2f4e4 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.132-SNAPSHOT + 0.6.132 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.132 diff --git a/processing/pom.xml b/processing/pom.xml index 9b0d491d41e..a4962b8e79c 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 81f38e849ca..1100af0935f 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 318ea5abe13..dd0cf8aab54 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/server/pom.xml b/server/pom.xml index 3bcfa6a9c86..f15adec550f 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 diff --git a/services/pom.xml b/services/pom.xml index 77a95f34b02..80cc4703049 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.132-SNAPSHOT + 0.6.132 From b5ecf000ad0c1de8f5f7ce89d708d7e9dc58e900 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 16:20:14 -0700 Subject: [PATCH 217/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index ef78700c525..c67e6691e3c 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index b5dbe41a23c..1075ea19829 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 3e5842cb1ba..8b5cdcaa70c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 5f9e306496a..c2930e63d2d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 7d88ddb363a..f8f4f76e467 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index e99d03587ac..4f618e02faf 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 37b97000df4..57efbb87068 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 4ae30024563..0bacc90faf1 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index c2d812baebe..0506952140f 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/pom.xml b/pom.xml index 3562db2f4e4..1346e021ade 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.132 + 0.6.133-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.132 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index a4962b8e79c..10ae308044c 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 1100af0935f..2f9afecd00c 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index dd0cf8aab54..eb323cb91e9 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index f15adec550f..80282644d25 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 80cc4703049..a92a96da56e 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.132 + 0.6.133-SNAPSHOT From f577e1e74b31a4a650f1c7b1a7b206c77e13e66c Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 17:23:32 -0700 Subject: [PATCH 218/270] fix guava version and firehose --- pom.xml | 2 +- .../realtime/firehose/EventReceiverFirehoseFactory.java | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 1346e021ade..256be1e0435 100644 --- a/pom.xml +++ b/pom.xml @@ -199,7 +199,7 @@ com.google.guava guava - 17.0 + 16.0.1 com.google.inject diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index c3512a42470..7b33688c1ce 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -91,9 +91,6 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory Date: Tue, 29 Jul 2014 17:38:19 -0700 Subject: [PATCH 219/270] prepare for next release --- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 7 +++---- docs/content/Realtime-Config.md | 4 ++-- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 4 ++-- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/overlord/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliPeon.java | 2 +- 15 files changed, 24 insertions(+), 25 deletions(-) diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 235df64a2ad..874436d5cb7 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -12,7 +12,7 @@ You can provision individual servers, loading Druid onto each machine (or buildi [Apache Whirr](http://whirr.apache.org/) is a set of libraries for launching cloud services. For Druid, Whirr serves as an easy way to launch a cluster in Amazon AWS by using simple commands and configuration files (called *recipes*). -**NOTE:** Whirr will install Druid 0.6.121. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. +**NOTE:** Whirr will install Druid 0.6.133. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. You'll need an AWS account, S3 Bucket and an EC2 key pair from that account so that Whirr can connect to the cloud via the EC2 API. If you haven't generated a key pair, see the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html) or see this [Whirr FAQ](http://whirr.apache.org/faq.html#how-do-i-find-my-cloud-credentials). diff --git a/docs/content/Examples.md b/docs/content/Examples.md index 58fb1f45e01..e4cf087c558 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.121 +git checkout druid-0.6.133 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.121-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 689a34b98a9..a8aa27ce7ae 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.121","io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -157,7 +157,6 @@ druid.indexer.logs.s3Prefix=prod/logs/v1 # Dedicate more resources to peons druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps druid.indexer.task.baseTaskDir=/mnt/persistent/task/ -druid.indexer.task.chathandler.type=announce druid.indexer.fork.property.druid.indexer.hadoopWorkingPath=/tmp/druid-indexing druid.indexer.fork.property.druid.computation.buffer.size=536870912 @@ -286,7 +285,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index a44c94fbdfa..6a396fac248 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.121","io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 33e0c8d13bd..d3a94e2f105 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 923f3f45e1b..2d0f4567f2a 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.121-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.121 +cd druid-services-0.6.133 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 56ca0d6305f..f1c28ddde0d 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index d0fc6a8fe45..61f0d035e92 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -354,9 +354,9 @@ Continuing from the Kafka 7 examples, to support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.121",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.133",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.121",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.133",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 0657595209b..c8db35c6070 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.121-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.121","io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index db87404ab88..2c5d1ef45e9 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.121-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.121 +cd druid-services-0.6.133 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index 3ed5b839494..b7fe986319c 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.121-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index fd0cbfc520a..92dbdcfbd54 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index 7a267d7114e..c2185520b64 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -4,7 +4,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 07c22756b6b..7c9b589f2a3 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.121","io.druid.extensions:druid-kafka-seven:0.6.121","io.druid.extensions:druid-rabbitmq:0.6.121"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133","io.druid.extensions:druid-rabbitmq:0.6.133"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index d85e974127d..4214f9130ed 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -111,7 +111,7 @@ public class CliPeon extends GuiceRunnable binder, "druid.indexer.task.chathandler.type", Key.get(ChatHandlerProvider.class), - Key.get(ServiceAnnouncingChatHandlerProvider.class) + Key.get(ServiceAnnouncingChatHandlerProvider.class, LazySingleton.class) ); final MapBinder handlerProviderBinder = PolyBind.optionBinder( binder, Key.get(ChatHandlerProvider.class) From 540a146eaaf21bf271135ab45f8d1cd42cb3a682 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 17:40:52 -0700 Subject: [PATCH 220/270] [maven-release-plugin] prepare release druid-0.6.133 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index c67e6691e3c..0d591595652 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/common/pom.xml b/common/pom.xml index 1075ea19829..db13938dd4c 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/examples/pom.xml b/examples/pom.xml index 8b5cdcaa70c..ac2995b03b0 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c2930e63d2d..48eaab7452b 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/histogram/pom.xml b/histogram/pom.xml index f8f4f76e467..835057e6666 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 4f618e02faf..15f02a1b533 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 57efbb87068..9b8f473fd40 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0bacc90faf1..6f338f6f1e8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 0506952140f..e856ffc78fe 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/pom.xml b/pom.xml index 256be1e0435..aa690801a4c 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.133-SNAPSHOT + 0.6.133 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.133 diff --git a/processing/pom.xml b/processing/pom.xml index 10ae308044c..eb702f371f6 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 2f9afecd00c..1547bd4011d 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index eb323cb91e9..90884521d77 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/server/pom.xml b/server/pom.xml index 80282644d25..65bf019a87e 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 diff --git a/services/pom.xml b/services/pom.xml index a92a96da56e..0ae6bf7e62f 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.133-SNAPSHOT + 0.6.133 From f7319620c4d4d57e2dafd5ed95d2b7ce26490174 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 17:40:56 -0700 Subject: [PATCH 221/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 0d591595652..71207e4bd79 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index db13938dd4c..bcf095ab809 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index ac2995b03b0..a492937ed99 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 48eaab7452b..bb5e5444856 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 835057e6666..6a240efa51b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 15f02a1b533..959a155bf9e 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9b8f473fd40..a12164c712d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 6f338f6f1e8..584c3f568c4 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index e856ffc78fe..67a6e73f6b2 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/pom.xml b/pom.xml index aa690801a4c..0543e3bbf59 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.133 + 0.6.134-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.133 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index eb702f371f6..64d5936c7a5 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 1547bd4011d..3c54f79babe 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 90884521d77..4db852c6bcc 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 65bf019a87e..5410ce5a220 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 0ae6bf7e62f..75df1dd4acd 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.133 + 0.6.134-SNAPSHOT From 7803ca07e7c9e291003fab3d1878945dc2a76eee Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 17:58:21 -0700 Subject: [PATCH 222/270] fix broken chat handler --- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 7 ++++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 4 ++-- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/overlord/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- services/src/main/java/io/druid/cli/CliPeon.java | 2 +- services/src/main/java/io/druid/guice/RealtimeModule.java | 2 +- 16 files changed, 26 insertions(+), 25 deletions(-) diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 874436d5cb7..cdfbd4915c7 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -12,7 +12,7 @@ You can provision individual servers, loading Druid onto each machine (or buildi [Apache Whirr](http://whirr.apache.org/) is a set of libraries for launching cloud services. For Druid, Whirr serves as an easy way to launch a cluster in Amazon AWS by using simple commands and configuration files (called *recipes*). -**NOTE:** Whirr will install Druid 0.6.133. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. +**NOTE:** Whirr will install Druid 0.6.134. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. You'll need an AWS account, S3 Bucket and an EC2 key pair from that account so that Whirr can connect to the cloud via the EC2 API. If you haven't generated a key pair, see the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html) or see this [Whirr FAQ](http://whirr.apache.org/faq.html#how-do-i-find-my-cloud-credentials). diff --git a/docs/content/Examples.md b/docs/content/Examples.md index e4cf087c558..f73fe9b64e2 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.133 +git checkout druid-0.6.134 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index a8aa27ce7ae..a2708cc252f 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -157,6 +157,7 @@ druid.indexer.logs.s3Prefix=prod/logs/v1 # Dedicate more resources to peons druid.indexer.runner.javaOpts=-server -Xmx3g -XX:+UseG1GC -XX:MaxGCPauseMillis=100 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps druid.indexer.task.baseTaskDir=/mnt/persistent/task/ +druid.indexer.task.chathandler.type=announce druid.indexer.fork.property.druid.indexer.hadoopWorkingPath=/tmp/druid-indexing druid.indexer.fork.property.druid.computation.buffer.size=536870912 @@ -285,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 6a396fac248..08fda6313b1 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index d3a94e2f105..aec85ee51d3 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 2d0f4567f2a..78b8505c8f0 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.133 +cd druid-services-0.6.134 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index f1c28ddde0d..5d7f87539a8 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index 61f0d035e92..c6b1321ba19 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -354,9 +354,9 @@ Continuing from the Kafka 7 examples, to support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.133",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.134",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.133",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.134",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index c8db35c6070..00ccea3ebb6 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 2c5d1ef45e9..9cd30606412 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.133 +cd druid-services-0.6.134 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index b7fe986319c..ad484b3be91 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.133-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index 92dbdcfbd54..ebed3b54e35 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index c2185520b64..1916bef644d 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -4,7 +4,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 7c9b589f2a3..ffdeeea452a 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.133","io.druid.extensions:druid-kafka-seven:0.6.133","io.druid.extensions:druid-rabbitmq:0.6.133"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134","io.druid.extensions:druid-rabbitmq:0.6.134"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/services/src/main/java/io/druid/cli/CliPeon.java b/services/src/main/java/io/druid/cli/CliPeon.java index 4214f9130ed..d4eb0228c9a 100644 --- a/services/src/main/java/io/druid/cli/CliPeon.java +++ b/services/src/main/java/io/druid/cli/CliPeon.java @@ -111,7 +111,7 @@ public class CliPeon extends GuiceRunnable binder, "druid.indexer.task.chathandler.type", Key.get(ChatHandlerProvider.class), - Key.get(ServiceAnnouncingChatHandlerProvider.class, LazySingleton.class) + Key.get(NoopChatHandlerProvider.class) ); final MapBinder handlerProviderBinder = PolyBind.optionBinder( binder, Key.get(ChatHandlerProvider.class) diff --git a/services/src/main/java/io/druid/guice/RealtimeModule.java b/services/src/main/java/io/druid/guice/RealtimeModule.java index 6770b650069..7bcce15823a 100644 --- a/services/src/main/java/io/druid/guice/RealtimeModule.java +++ b/services/src/main/java/io/druid/guice/RealtimeModule.java @@ -65,7 +65,7 @@ public class RealtimeModule implements Module binder, "druid.realtime.chathandler.type", Key.get(ChatHandlerProvider.class), - Key.get(ServiceAnnouncingChatHandlerProvider.class) + Key.get(NoopChatHandlerProvider.class) ); final MapBinder handlerProviderBinder = PolyBind.optionBinder( binder, Key.get(ChatHandlerProvider.class) From c8244d05d10d2b2e47a8b555081edcba7a70ba81 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 18:01:02 -0700 Subject: [PATCH 223/270] [maven-release-plugin] prepare release druid-0.6.134 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 71207e4bd79..e0274c342a4 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/common/pom.xml b/common/pom.xml index bcf095ab809..95016719a30 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/examples/pom.xml b/examples/pom.xml index a492937ed99..86ee6b85e2d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index bb5e5444856..d0a89db5ac9 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/histogram/pom.xml b/histogram/pom.xml index 6a240efa51b..b8438f97860 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 959a155bf9e..9f00cdf934f 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index a12164c712d..53b157c30fd 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 584c3f568c4..12d0d99a833 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 67a6e73f6b2..f3d1333c1e0 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/pom.xml b/pom.xml index 0543e3bbf59..2563c24ee80 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.134-SNAPSHOT + 0.6.134 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.134 diff --git a/processing/pom.xml b/processing/pom.xml index 64d5936c7a5..2d29827ff73 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 3c54f79babe..7c0d88e7e54 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 4db852c6bcc..ce401c08ff3 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/server/pom.xml b/server/pom.xml index 5410ce5a220..fc7f8e747c0 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 diff --git a/services/pom.xml b/services/pom.xml index 75df1dd4acd..9273ae8b161 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.134-SNAPSHOT + 0.6.134 From 0c898c58d1b20b39340fbee20921440ced161dc0 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 18:01:06 -0700 Subject: [PATCH 224/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index e0274c342a4..f1e94d8f2fa 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 95016719a30..17b3459f5c8 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 86ee6b85e2d..216aa383c24 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index d0a89db5ac9..7bbe86650b6 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index b8438f97860..a06826ce29b 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 9f00cdf934f..006c04af6ba 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 53b157c30fd..8a803661b26 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 12d0d99a833..acd65e7558e 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f3d1333c1e0..c26963d3333 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/pom.xml b/pom.xml index 2563c24ee80..47585008873 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.134 + 0.6.135-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.134 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 2d29827ff73..fd0b04bca65 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 7c0d88e7e54..4005edd42b0 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index ce401c08ff3..dcb9622c4c1 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index fc7f8e747c0..1b3127528d7 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 9273ae8b161..d19a65f60d4 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.134 + 0.6.135-SNAPSHOT From 39203009009cc25d4d0f4d142801a22938475bd6 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 21:44:07 -0700 Subject: [PATCH 225/270] fix router bug --- .../java/io/druid/server/AsyncQueryForwardingServlet.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java index f284a276202..91f90d37f91 100644 --- a/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java +++ b/server/src/main/java/io/druid/server/AsyncQueryForwardingServlet.java @@ -49,6 +49,7 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.net.URI; import java.util.Enumeration; +import java.util.UUID; import java.util.concurrent.TimeUnit; /** @@ -116,6 +117,9 @@ public class AsyncQueryForwardingServlet extends AsyncProxyServlet inputQuery = objectMapper.readValue(request.getInputStream(), Query.class); if (inputQuery != null) { host = hostFinder.getHost(inputQuery); + if (inputQuery.getId() == null) { + inputQuery = inputQuery.withId(UUID.randomUUID().toString()); + } } } catch (IOException e) { From ecbc9eda48422c5ade57a21209f717f6aba24a86 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 21:55:05 -0700 Subject: [PATCH 226/270] revert breakage to chat handlers --- .../realtime/firehose/EventReceiverFirehoseFactory.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index 7b33688c1ce..5b8311c665e 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -91,6 +91,9 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory event : events) { // Might throw an exception. We'd like that to happen now, instead of while adding to the row buffer. InputRow row = parser.parse(event); - rows.add(Rows.toCaseInsensitiveInputRow(row,row.getDimensions())); + rows.add(Rows.toCaseInsensitiveInputRow(row, row.getDimensions())); } try { From 625377fdd24083bcce5b4c2517e4337e464b996f Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 22:00:11 -0700 Subject: [PATCH 227/270] prep next rel --- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 4 ++-- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/overlord/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- 14 files changed, 23 insertions(+), 23 deletions(-) diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index cdfbd4915c7..09d28b190b4 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -12,7 +12,7 @@ You can provision individual servers, loading Druid onto each machine (or buildi [Apache Whirr](http://whirr.apache.org/) is a set of libraries for launching cloud services. For Druid, Whirr serves as an easy way to launch a cluster in Amazon AWS by using simple commands and configuration files (called *recipes*). -**NOTE:** Whirr will install Druid 0.6.134. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. +**NOTE:** Whirr will install Druid 0.6.135. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. You'll need an AWS account, S3 Bucket and an EC2 key pair from that account so that Whirr can connect to the cloud via the EC2 API. If you haven't generated a key pair, see the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html) or see this [Whirr FAQ](http://whirr.apache.org/faq.html#how-do-i-find-my-cloud-credentials). diff --git a/docs/content/Examples.md b/docs/content/Examples.md index f73fe9b64e2..db6435f78c0 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.134 +git checkout druid-0.6.135 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index a2708cc252f..9624485a4d7 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 08fda6313b1..6283da9ecf2 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index aec85ee51d3..8a54b13f7b1 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 78b8505c8f0..565336993e6 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.134 +cd druid-services-0.6.135 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 5d7f87539a8..3aa6b74f535 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index c6b1321ba19..49ddc3618a1 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -354,9 +354,9 @@ Continuing from the Kafka 7 examples, to support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.134",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.135",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.134",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.135",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 00ccea3ebb6..0aca4b74af1 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 9cd30606412..9cb1e1ed156 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.134 +cd druid-services-0.6.135 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index ad484b3be91..b125d5c177e 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.134-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index ebed3b54e35..283839e9d6c 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index 1916bef644d..44a7a3cdaee 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -4,7 +4,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index ffdeeea452a..76ea5625088 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.134","io.druid.extensions:druid-kafka-seven:0.6.134","io.druid.extensions:druid-rabbitmq:0.6.134"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135","io.druid.extensions:druid-rabbitmq:0.6.135"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop From 25eda62a0cf7ec71632b6d9f7db892811cdac9e2 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 22:03:09 -0700 Subject: [PATCH 228/270] [maven-release-plugin] prepare release druid-0.6.135 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index f1e94d8f2fa..2dbf1a3902b 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/common/pom.xml b/common/pom.xml index 17b3459f5c8..e2419808f26 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/examples/pom.xml b/examples/pom.xml index 216aa383c24..8f2e326c53f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 7bbe86650b6..b7c1bfb793b 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/histogram/pom.xml b/histogram/pom.xml index a06826ce29b..1f9587e4c8c 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 006c04af6ba..f81f9c299c9 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 8a803661b26..2cafc5dca8a 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index acd65e7558e..2fa996949fa 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index c26963d3333..b2b6d2786e7 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/pom.xml b/pom.xml index 47585008873..3d5d6956a7a 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.135-SNAPSHOT + 0.6.135 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.135 diff --git a/processing/pom.xml b/processing/pom.xml index fd0b04bca65..da50c8afa3a 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 4005edd42b0..07997cabc4d 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index dcb9622c4c1..7acaecac3fa 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/server/pom.xml b/server/pom.xml index 1b3127528d7..40878470838 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 diff --git a/services/pom.xml b/services/pom.xml index d19a65f60d4..933efd88534 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.135-SNAPSHOT + 0.6.135 From a8c8e4cb5142a805d2464f5f9e4226d2c0729512 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 29 Jul 2014 22:03:14 -0700 Subject: [PATCH 229/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 2dbf1a3902b..c49b1bf80be 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index e2419808f26..d01a68178ce 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 8f2e326c53f..033e1b0e8dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index b7c1bfb793b..633ebe927dc 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index 1f9587e4c8c..5b2be700289 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index f81f9c299c9..f90cc00f86c 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 2cafc5dca8a..5a602bd3c22 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 2fa996949fa..93a67938404 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index b2b6d2786e7..7d1d87f3a0d 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/pom.xml b/pom.xml index 3d5d6956a7a..92b9e55d8b5 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.135 + 0.6.136-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.135 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index da50c8afa3a..7a5819fe094 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 07997cabc4d..aadd8487deb 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 7acaecac3fa..0ab45df6952 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 40878470838..5bda13057a5 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 933efd88534..344bb25a029 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.135 + 0.6.136-SNAPSHOT From 147f5754da13c84918d00b6b399cc6867047c6eb Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 10:23:48 -0700 Subject: [PATCH 230/270] add missing doc --- docs/content/Broker-Config.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/Broker-Config.md b/docs/content/Broker-Config.md index 80eaf4ce404..7d40ba27b39 100644 --- a/docs/content/Broker-Config.md +++ b/docs/content/Broker-Config.md @@ -16,6 +16,7 @@ The broker module uses several of the default modules in [Configuration](Configu |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`| |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None| |`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| +|`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`groupBy`, `select`| #### Local Cache From 32bd7003ec810c292c90b743dfbdbcbcd55d3ebf Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 12:58:19 -0700 Subject: [PATCH 231/270] minor doc fix --- docs/content/Broker-Config.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/Broker-Config.md b/docs/content/Broker-Config.md index 7d40ba27b39..699bf1a49d7 100644 --- a/docs/content/Broker-Config.md +++ b/docs/content/Broker-Config.md @@ -16,7 +16,7 @@ The broker module uses several of the default modules in [Configuration](Configu |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`| |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None| |`druid.broker.cache.type`|`local`, `memcached`|The type of cache to use for queries.|`local`| -|`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`groupBy`, `select`| +|`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| #### Local Cache From 1c4d8c55dc4cc28badfb2ccb4ffe074e70c00f8d Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 13:56:29 -0700 Subject: [PATCH 232/270] finally track down and fix the node exists problem with not deleting noop requests --- .../io/druid/server/coordination/SegmentChangeRequestNoop.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java index a5f254b25e3..08f0ae334ab 100644 --- a/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java +++ b/server/src/main/java/io/druid/server/coordination/SegmentChangeRequestNoop.java @@ -26,7 +26,8 @@ public class SegmentChangeRequestNoop implements DataSegmentChangeRequest @Override public void go(DataSegmentChangeHandler handler, DataSegmentChangeCallback callback) { - // do nothing + // just execute the callback and do nothing else + callback.execute(); } @Override From e5ce9b4a2ad576f82f46339470b6c6c7e2222804 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 14:05:56 -0700 Subject: [PATCH 233/270] prepare for next release --- docs/content/Booting-a-production-cluster.md | 2 +- docs/content/Examples.md | 4 ++-- docs/content/Production-Cluster-Configuration.md | 6 +++--- docs/content/Realtime-Config.md | 4 ++-- docs/content/Simple-Cluster-Configuration.md | 2 +- docs/content/Tutorial:-A-First-Look-at-Druid.md | 4 ++-- docs/content/Tutorial:-Loading-Your-Data-Part-1.md | 2 +- docs/content/Tutorial:-Loading-Your-Data-Part-2.md | 4 ++-- docs/content/Tutorial:-The-Druid-Cluster.md | 6 +++--- docs/content/Tutorial:-Webstream.md | 4 ++-- docs/content/Twitter-Tutorial.md | 2 +- examples/config/historical/runtime.properties | 2 +- examples/config/overlord/runtime.properties | 2 +- examples/config/realtime/runtime.properties | 2 +- 14 files changed, 23 insertions(+), 23 deletions(-) diff --git a/docs/content/Booting-a-production-cluster.md b/docs/content/Booting-a-production-cluster.md index 09d28b190b4..2b0e5bd6f2b 100644 --- a/docs/content/Booting-a-production-cluster.md +++ b/docs/content/Booting-a-production-cluster.md @@ -12,7 +12,7 @@ You can provision individual servers, loading Druid onto each machine (or buildi [Apache Whirr](http://whirr.apache.org/) is a set of libraries for launching cloud services. For Druid, Whirr serves as an easy way to launch a cluster in Amazon AWS by using simple commands and configuration files (called *recipes*). -**NOTE:** Whirr will install Druid 0.6.135. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. +**NOTE:** Whirr will install Druid 0.6.136. Also, it doesn't work with JDK1.7.0_55. JDK1.7.0_45 recommended. You'll need an AWS account, S3 Bucket and an EC2 key pair from that account so that Whirr can connect to the cloud via the EC2 API. If you haven't generated a key pair, see the [AWS documentation](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-key-pairs.html) or see this [Whirr FAQ](http://whirr.apache.org/faq.html#how-do-i-find-my-cloud-credentials). diff --git a/docs/content/Examples.md b/docs/content/Examples.md index db6435f78c0..62ad7f39140 100644 --- a/docs/content/Examples.md +++ b/docs/content/Examples.md @@ -19,13 +19,13 @@ Clone Druid and build it: git clone https://github.com/metamx/druid.git druid cd druid git fetch --tags -git checkout druid-0.6.135 +git checkout druid-0.6.136 ./build.sh ``` ### Downloading the DSK (Druid Standalone Kit) -[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz) a stand-alone tarball and run it: +[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.136-bin.tar.gz) a stand-alone tarball and run it: ``` bash tar -xzf druid-services-0.X.X-bin.tar.gz diff --git a/docs/content/Production-Cluster-Configuration.md b/docs/content/Production-Cluster-Configuration.md index 9624485a4d7..80865798ced 100644 --- a/docs/content/Production-Cluster-Configuration.md +++ b/docs/content/Production-Cluster-Configuration.md @@ -57,7 +57,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/overlord -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.136"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -139,7 +139,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/middlemanager -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.136","io.druid.extensions:druid-kafka-seven:0.6.136"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod @@ -286,7 +286,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/historical -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.136"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Realtime-Config.md b/docs/content/Realtime-Config.md index 6283da9ecf2..8cd3970dc2e 100644 --- a/docs/content/Realtime-Config.md +++ b/docs/content/Realtime-Config.md @@ -27,7 +27,7 @@ druid.host=localhost druid.service=realtime druid.port=8083 -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.136"] druid.zk.service.host=localhost @@ -76,7 +76,7 @@ druid.host=#{IP_ADDR}:8080 druid.port=8080 druid.service=druid/prod/realtime -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.136","io.druid.extensions:druid-kafka-seven:0.6.136"] druid.zk.service.host=#{ZK_IPs} druid.zk.paths.base=/druid/prod diff --git a/docs/content/Simple-Cluster-Configuration.md b/docs/content/Simple-Cluster-Configuration.md index 8a54b13f7b1..48ddc70745b 100644 --- a/docs/content/Simple-Cluster-Configuration.md +++ b/docs/content/Simple-Cluster-Configuration.md @@ -28,7 +28,7 @@ Configuration: -Ddruid.zk.service.host=localhost --Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] +-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.136"] -Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid -Ddruid.db.connector.user=druid diff --git a/docs/content/Tutorial:-A-First-Look-at-Druid.md b/docs/content/Tutorial:-A-First-Look-at-Druid.md index 565336993e6..2c61087b1d1 100644 --- a/docs/content/Tutorial:-A-First-Look-at-Druid.md +++ b/docs/content/Tutorial:-A-First-Look-at-Druid.md @@ -49,7 +49,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu ### Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz). Download this file to a directory of your choosing. +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.136-bin.tar.gz). Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -60,7 +60,7 @@ tar -zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.135 +cd druid-services-0.6.136 ``` You should see a bunch of files: diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md index 3aa6b74f535..1d828199ef6 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-1.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-1.md @@ -91,7 +91,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.136"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md index 49ddc3618a1..13f28332218 100644 --- a/docs/content/Tutorial:-Loading-Your-Data-Part-2.md +++ b/docs/content/Tutorial:-Loading-Your-Data-Part-2.md @@ -354,9 +354,9 @@ Continuing from the Kafka 7 examples, to support Kafka 8, a couple changes need - Update realtime node's configs for Kafka 8 extensions - e.g. - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.135",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-seven:0.6.136",...]` - becomes - - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.135",...]` + - `druid.extensions.coordinates=[...,"io.druid.extensions:druid-kafka-eight:0.6.136",...]` - Update realtime task config for changed keys - `firehose.type`, `plumber.rejectionPolicyFactory`, and all of `firehose.consumerProps` changes. diff --git a/docs/content/Tutorial:-The-Druid-Cluster.md b/docs/content/Tutorial:-The-Druid-Cluster.md index 0aca4b74af1..55a0625d438 100644 --- a/docs/content/Tutorial:-The-Druid-Cluster.md +++ b/docs/content/Tutorial:-The-Druid-Cluster.md @@ -13,7 +13,7 @@ In this tutorial, we will set up other types of Druid nodes and external depende If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first. -You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz) +You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.136-bin.tar.gz) and untar the contents within by issuing: @@ -149,7 +149,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.136"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b @@ -240,7 +240,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.136","io.druid.extensions:druid-kafka-seven:0.6.136"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop diff --git a/docs/content/Tutorial:-Webstream.md b/docs/content/Tutorial:-Webstream.md index 9cb1e1ed156..ed1bd311f3f 100644 --- a/docs/content/Tutorial:-Webstream.md +++ b/docs/content/Tutorial:-Webstream.md @@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu h3. Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz) +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.136-bin.tar.gz) Download this file to a directory of your choosing. You can extract the awesomeness within by issuing: @@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz Not too lost so far right? That's great! If you cd into the directory: ``` -cd druid-services-0.6.135 +cd druid-services-0.6.136 ``` You should see a bunch of files: diff --git a/docs/content/Twitter-Tutorial.md b/docs/content/Twitter-Tutorial.md index b125d5c177e..58c2a734166 100644 --- a/docs/content/Twitter-Tutorial.md +++ b/docs/content/Twitter-Tutorial.md @@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source. # Download a Tarball -We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.135-bin.tar.gz). +We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.136-bin.tar.gz). Download this bad boy to a directory of your choosing. You can extract the awesomeness within by issuing: diff --git a/examples/config/historical/runtime.properties b/examples/config/historical/runtime.properties index 283839e9d6c..e5ff4e3cbed 100644 --- a/examples/config/historical/runtime.properties +++ b/examples/config/historical/runtime.properties @@ -4,7 +4,7 @@ druid.port=8081 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.136"] # Dummy read only AWS account (used to download example data) druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b diff --git a/examples/config/overlord/runtime.properties b/examples/config/overlord/runtime.properties index 44a7a3cdaee..c251665381e 100644 --- a/examples/config/overlord/runtime.properties +++ b/examples/config/overlord/runtime.properties @@ -4,7 +4,7 @@ druid.service=overlord druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.136"] druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid druid.db.connector.user=druid diff --git a/examples/config/realtime/runtime.properties b/examples/config/realtime/runtime.properties index 76ea5625088..619b37a17b6 100644 --- a/examples/config/realtime/runtime.properties +++ b/examples/config/realtime/runtime.properties @@ -4,7 +4,7 @@ druid.port=8083 druid.zk.service.host=localhost -druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.135","io.druid.extensions:druid-kafka-seven:0.6.135","io.druid.extensions:druid-rabbitmq:0.6.135"] +druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.136","io.druid.extensions:druid-kafka-seven:0.6.136","io.druid.extensions:druid-rabbitmq:0.6.136"] # Change this config to db to hand off to the rest of the Druid cluster druid.publish.type=noop From 66020fc2b6c988186c5a1974f484f2302744f1cd Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 14:08:34 -0700 Subject: [PATCH 234/270] [maven-release-plugin] prepare release druid-0.6.136 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index c49b1bf80be..ff594b46bea 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/common/pom.xml b/common/pom.xml index d01a68178ce..8642a540a69 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/examples/pom.xml b/examples/pom.xml index 033e1b0e8dd..4b8e1f293c8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 633ebe927dc..371640416c8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/histogram/pom.xml b/histogram/pom.xml index 5b2be700289..d635996c973 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index f90cc00f86c..5ca567a8227 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 5a602bd3c22..553bc14f37e 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 93a67938404..aa230e28914 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 7d1d87f3a0d..1ebcedba4df 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/pom.xml b/pom.xml index 92b9e55d8b5..c31cee6f2b5 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.136-SNAPSHOT + 0.6.136 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.131-SNAPSHOT + druid-0.6.136 diff --git a/processing/pom.xml b/processing/pom.xml index 7a5819fe094..475d202c331 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index aadd8487deb..40c2c64ce9f 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 0ab45df6952..9297a6866e6 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/server/pom.xml b/server/pom.xml index 5bda13057a5..c53ca3aae42 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 diff --git a/services/pom.xml b/services/pom.xml index 344bb25a029..889ab4bf65e 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.136-SNAPSHOT + 0.6.136 From bf8c24f88d672e892d030f17fd30bf151d640723 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 14:08:38 -0700 Subject: [PATCH 235/270] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- histogram/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index ff594b46bea..e8562ca18a3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 8642a540a69..01db6bbed33 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 4b8e1f293c8..62799c406c4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 371640416c8..f1c35628614 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/histogram/pom.xml b/histogram/pom.xml index d635996c973..ae05850b026 100644 --- a/histogram/pom.xml +++ b/histogram/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 5ca567a8227..60143f4a6a6 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 553bc14f37e..ce683b668e0 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index aa230e28914..f78036357c5 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 1ebcedba4df..5e037846335 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/pom.xml b/pom.xml index c31cee6f2b5..86d0781e23a 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.136 + 0.6.137-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.136 + druid-0.6.131-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index 475d202c331..c68115b4ea7 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 40c2c64ce9f..4fa96fc9aff 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 9297a6866e6..3f61b18ec04 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index c53ca3aae42..b3f56f7ebb1 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 889ab4bf65e..dcae58927cd 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.136 + 0.6.137-SNAPSHOT From 872a699142df889f62834c5a3db39462b27b8173 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 15:36:01 -0700 Subject: [PATCH 236/270] another log fix --- .../main/java/io/druid/server/coordinator/LoadQueuePeon.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java index e2577795a27..c7ee45ce492 100644 --- a/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java +++ b/server/src/main/java/io/druid/server/coordinator/LoadQueuePeon.java @@ -208,7 +208,7 @@ public class LoadQueuePeon } } - log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment); + log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier()); segmentsToDrop.add(holder); doNext(); } From 37f3cb38db1d9995623df850e7ec68cbae9e5275 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 17:08:08 -0700 Subject: [PATCH 237/270] fix to string error --- .../java/io/druid/query/timeboundary/TimeBoundaryQuery.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java index db9fa913bc9..5aae5d9fc67 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQuery.java @@ -199,7 +199,7 @@ public class TimeBoundaryQuery extends BaseQuery "dataSource='" + getDataSource() + '\'' + ", querySegmentSpec=" + getQuerySegmentSpec() + ", duration=" + getDuration() + - ", bound" + bound + + ", bound=" + bound + '}'; } From 17c600c691cc46fa22020e3602e42ce83f4b3c82 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 17:25:30 -0700 Subject: [PATCH 238/270] some doc fixes --- docs/content/toc.textile | 1 + docs/css/index.css | 17 ++++------------- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 5ca3d978fe4..6b2fa17c216 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -39,6 +39,7 @@ h2. Operations * "Extending Druid":./Modules.html * "Booting a Production Cluster":./Booting-a-production-cluster.html * "Performance FAQ":./Performance-FAQ.html +* "Logging":./Logging.html h2. Querying * "Querying":./Querying.html diff --git a/docs/css/index.css b/docs/css/index.css index d41ee794334..ead85a32b14 100644 --- a/docs/css/index.css +++ b/docs/css/index.css @@ -1,21 +1,12 @@ -.sub-text { - margin-top: 20px; - margin-bottom: 50px; -} - .main-marketing { - margin-bottom: 50px; + padding: 30px 0 30px 0; } .main-marketing a { - color: #000000; + color: #333; } -h2 { +.main-marketing h2 { + margin-top: 20px; font-weight: 400; - font-size: 30px; } - -.main-marketing img { - margin-bottom: 40px; -} \ No newline at end of file From 611dcb9d1c16702a2aabb57a287c57a0e79366d3 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 14:47:44 -0700 Subject: [PATCH 239/270] new docs on logging Conflicts: docs/content/toc.textile --- docs/content/Logging.md | 39 +++++++++++++++++++++++++++++++++++++++ docs/content/toc.textile | 4 +++- 2 files changed, 42 insertions(+), 1 deletion(-) create mode 100644 docs/content/Logging.md diff --git a/docs/content/Logging.md b/docs/content/Logging.md new file mode 100644 index 00000000000..d949aa7858d --- /dev/null +++ b/docs/content/Logging.md @@ -0,0 +1,39 @@ +--- +layout: doc_page +--- +Logging +========================== + +Druid nodes will emit logs that are useful for debugging to the console. Druid nodes also emit periodic metrics about their state. For more about metrics, see [Configuration](Configuration.html). Metric logs are printed to the console by default, and can be disabled with `-Ddruid.emitter.logging.logLevel=debug`. + +Druid uses [log4j](http://logging.apache.org/log4j/2.x/) for logging, and console logs can be configured by adding a log4j.xml file. Add this xml file to your classpath if you want to override default Druid log configuration. + +An example log4j.xml file is shown below: + +``` + + + + + + + + + + + + + + + + + + + + + + + + + +``` \ No newline at end of file diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 6b2fa17c216..0deb2d27bed 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -28,16 +28,18 @@ h2. Configuration * "Realtime":Realtime-Config.html h2. Data Ingestion +* "Ingestion FAQ":./Ingestion-FAQ.html * "Realtime":./Realtime-ingestion.html * "Batch":./Batch-ingestion.html * "Indexing Service":./Indexing-Service.html ** "Tasks":./Tasks.html * "Data Formats":./Data_formats.html -* "Ingestion FAQ":./Ingestion-FAQ.html h2. Operations +* "Performance FAQ":./Performance-FAQ.html * "Extending Druid":./Modules.html * "Booting a Production Cluster":./Booting-a-production-cluster.html +<<<<<<< HEAD * "Performance FAQ":./Performance-FAQ.html * "Logging":./Logging.html From d202eae3b5fe76c08cbd402ac7573d1fe6e5c776 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 31 Jul 2014 17:27:16 -0700 Subject: [PATCH 240/270] toc fix --- docs/content/toc.textile | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/content/toc.textile b/docs/content/toc.textile index 0deb2d27bed..6af23d2ce50 100644 --- a/docs/content/toc.textile +++ b/docs/content/toc.textile @@ -39,7 +39,6 @@ h2. Operations * "Performance FAQ":./Performance-FAQ.html * "Extending Druid":./Modules.html * "Booting a Production Cluster":./Booting-a-production-cluster.html -<<<<<<< HEAD * "Performance FAQ":./Performance-FAQ.html * "Logging":./Logging.html From 80a4f7c5b5e70c8014a3c5c4b8b113287b160b57 Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Fri, 1 Aug 2014 16:43:39 +0530 Subject: [PATCH 241/270] doc for IngestSegmentFirehose --- docs/content/Firehose.md | 32 +++++++++++++++++++++++++++++++- docs/content/Ingestion-FAQ.md | 5 +++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/docs/content/Firehose.md b/docs/content/Firehose.md index c87d0b792bf..ebe15b3632a 100644 --- a/docs/content/Firehose.md +++ b/docs/content/Firehose.md @@ -36,10 +36,40 @@ See [Examples](Examples.html). This firehose connects directly to the twitter sp See [Examples](Examples.html). This firehose creates a stream of random numbers. -#### RabbitMqFirehouse +#### RabbitMqFirehose This firehose ingests events from a define rabbit-mq queue. +#### IngestSegmentFirehose + +This Firehose can be used to read the data from existing druid segments. +It can be used ingest existing druid segments using a new schema and change the name, dimensions, metrics, rollup, etc. of the segment. +A sample ingest firehose spec is shown below - + +```json +{ + "type" : "ingestSegment", + "dataSource" : "wikipedia", + "interval" : "2013-01-01/2013-01-02", + "dimensions":[], + "metrics":[] +} +``` + +|property|description|required?| +|--------|-----------|---------| +|type|ingestSegment. Type of firehose|yes| +|dataSource|A String defining the data source to fetch rows from, very similar to a table in a relational database|yes| +|interval|A String representing ISO-8601 Interval. This defines the time range to fetch the data over.|yes| +|dimensions|The list of dimensions to select. If left empty, all dimensions are selected.|no| +|metrics|The list of metrics to select. If left empty, all metrics are returned.|no| +|filter| See [Filters](Filters.html)|yes| + + + + + + Parsing Data ------------ diff --git a/docs/content/Ingestion-FAQ.md b/docs/content/Ingestion-FAQ.md index 30e5de81a53..59bb8fb7a93 100644 --- a/docs/content/Ingestion-FAQ.md +++ b/docs/content/Ingestion-FAQ.md @@ -37,6 +37,11 @@ You can check the coordinator console located at `:/cluste You can check `:/druid/v2/datasources/?interval=0/3000` for the dimensions and metrics that have been created for your datasource. Make sure that the name of the aggregators you use in your query match one of these metrics. Also make sure that the query interval you specify match a valid time range where data exists. Note: the broker endpoint will only return valid results on historical segments. +## How can I Reindex existing data in Druid with schema changes? + +You can use IngestSegmentFirehose with index task to ingest existing druid segments using a new schema and change the name, dimensions, metrics, rollup, etc. of the segment. +See [Firehose](Firehose.html) for more details on IngestSegmentFirehose. + ## More information Getting data into Druid can definitely be difficult for first time users. Please don't hesitate to ask questions in our IRC channel or on our [google groups page](https://groups.google.com/forum/#!forum/druid-development). From 90ebeaa8d33a31a61468dd8acae33a3934d126fa Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 1 Aug 2014 09:42:52 -0700 Subject: [PATCH 242/270] fix broken approx histo doc --- docs/content/ApproxHisto.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/content/ApproxHisto.md b/docs/content/ApproxHisto.md index 4c7229b813e..70d7edafab7 100644 --- a/docs/content/ApproxHisto.md +++ b/docs/content/ApproxHisto.md @@ -1,3 +1,6 @@ +--- +layout: doc_page +--- ### ApproxHistogram aggregator This aggregator is based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf) to compute approximate histograms. From b44546ad8c302fb003d7616ec0d2e8ab4a2ed7b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 1 Aug 2014 09:47:12 -0700 Subject: [PATCH 243/270] query cancellation/timeout docs + missing items --- docs/content/Querying.md | 55 +++++++++++++++++++++++++++++++--------- 1 file changed, 43 insertions(+), 12 deletions(-) diff --git a/docs/content/Querying.md b/docs/content/Querying.md index 13f7b5a91ce..d20a6385fe9 100644 --- a/docs/content/Querying.md +++ b/docs/content/Querying.md @@ -108,21 +108,52 @@ Query Operators The following table summarizes query properties. -|query types|property|description|required?| -|-----------|--------|-----------|---------| -|timeseries, groupBy, search, timeBoundary|dataSource|query is applied to this data source|yes| -|timeseries, groupBy, search|intervals|range of time series to include in query|yes| -|timeseries, groupBy, search, timeBoundary|context|This is a key-value map that can allow the query to alter some of the behavior of a query. It is primarily used for debugging, for example if you include `"bySegment":true` in the map, you will get results associated with the data segment they came from.|no| -|timeseries, groupBy, search|filter|Specifies the filter (the "WHERE" clause in SQL) for the query. See [Filters](Filters.html)|no| -|timeseries, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. "hour"). See [Granularities](Granularities.html) for more information.|no| +Properties shared by all query types + +|property |description|required?| +|----------|-----------|---------| +|dataSource|query is applied to this data source|yes| +|intervals |range of time series to include in query|yes| +|context |This is a key-value map used to alter some of the behavior of a query. See [Query Context](#query-context) below|no| + + +|query type|property |description|required?| +|----------|----------|-----------|---------| +|timeseries, topN, groupBy, search|filter|Specifies the filter (the "WHERE" clause in SQL) for the query. See [Filters](Filters.html)|no| +|timeseries, topN, groupBy, search|granularity|the timestamp granularity to bucket results into (i.e. "hour"). See [Granularities](Granularities.html) for more information.|no| +|timeseries, topN, groupBy|aggregations|aggregations that combine values in a bucket. See [Aggregations](Aggregations.html).|yes| +|timeseries, topN, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post Aggregations.html).|yes| |groupBy|dimensions|constrains the groupings; if empty, then one value per time granularity bucket|yes| -|timeseries, groupBy|aggregations|aggregations that combine values in a bucket. See [Aggregations](Aggregations.html).|yes| -|timeseries, groupBy|postAggregations|aggregations of aggregations. See [Post Aggregations](Post Aggregations.html).|yes| |search|limit|maximum number of results (default is 1000), a system-level maximum can also be set via `com.metamx.query.search.maxSearchLimit`|no| |search|searchDimensions|Dimensions to apply the search query to. If not specified, it will search through all dimensions.|no| |search|query|The query portion of the search query. This is essentially a predicate that specifies if something matches.|yes| -Additional Information about Query Types ----------------------------------------- +Query Context +------------- -[TimeseriesQuery](TimeseriesQuery.html) +|property |default | description | +|--------------|---------------------|----------------------| +|timeout | `0` (no timeout) | Query timeout in milliseconds, beyond which unfinished queries will be cancelled | +|priority | `0` | Query Priority. Queries with higher priority get precedence for computational resources.| +|queryId | auto-generated | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query | +|useCache | `true` | Flag indicating whether to leverage the query cache for this query. This may be overriden in the broker or historical node configuration | +|populateCache | `true` | Flag indicating whether to save the results of the query to the query cache. Primarily used for debugging. This may be overriden in the broker or historical node configuration | +|bySegment | `false` | Return "by segment" results. Pimarily used for debugging, setting it to `true` returns results associated with the data segment they came from | +|finalize | `true` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` | + +Query Cancellation +------------------ + +Queries can be cancelled explicitely using their unique identifier. If the +query identifier is set at the time of query, or is known, the following +endpoint can be used on the broker or router to cancel the query. + +```sh +DELETE /druid/v2/{queryId} +``` + +For example, if the query ID is `abc123`, the query can be cancelled as follows: + +```sh +curl -X DELETE "http://host:port/druid/v2/abc123" +``` From 2667cd518819dad0636c87df44799ea02c53d977 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 1 Aug 2014 10:01:11 -0700 Subject: [PATCH 244/270] cleanup docs template for preview --- docs/_includes/page_footer.html | 42 ++------------------------------- docs/_includes/page_header.html | 13 ++-------- docs/_includes/site_head.html | 8 +++---- docs/_layouts/doc_page.html | 27 +++++++++------------ 4 files changed, 18 insertions(+), 72 deletions(-) diff --git a/docs/_includes/page_footer.html b/docs/_includes/page_footer.html index 1890df00f8b..71bab20d32d 100644 --- a/docs/_includes/page_footer.html +++ b/docs/_includes/page_footer.html @@ -1,48 +1,10 @@ -
+
- - diff --git a/docs/_includes/page_header.html b/docs/_includes/page_header.html index df3d7e1e5a4..623f5e1afc7 100644 --- a/docs/_includes/page_header.html +++ b/docs/_includes/page_header.html @@ -1,25 +1,16 @@ - -