From 77ec4df79718a218acd78a7d45cdfe87935eba83 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 3 Jun 2014 13:43:38 -0700 Subject: [PATCH 01/72] 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 5bd4ad3f0f29d6b9e8a955f0b582eaacb6238582 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 11:29:49 -0700 Subject: [PATCH 02/72] 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 03/72] 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 d4a47fe6e8f86d3bceb825935960ffaa38c92de5 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 13:45:32 -0700 Subject: [PATCH 04/72] 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 05/72] 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 06/72] 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 07/72] 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 08/72] 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 09/72] 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 10/72] 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 11/72] 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 12/72] 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 13/72] 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 14/72] [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 15/72] [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 16/72] 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 17/72] [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 18/72] [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 19/72] 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 20/72] 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 21/72] 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 22/72] 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 23/72] 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 24/72] 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 25/72] 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 26/72] 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 27/72] [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 28/72] [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 29/72] 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 30/72] 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 31/72] [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 32/72] [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 33/72] 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 34/72] 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 35/72] 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 36/72] [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 37/72] [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 38/72] 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 39/72] 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 40/72] 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 41/72] 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 42/72] 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 43/72] 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 44/72] 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 45/72] 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 46/72] 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 47/72] 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 48/72] 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 49/72] 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 50/72] 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 51/72] 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 52/72] 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 53/72] 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 54/72] 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 55/72] 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 56/72] 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 57/72] 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 58/72] 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 59/72] 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 60/72] 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 61/72] 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 62/72] 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 63/72] 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 64/72] 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 65/72] 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 66/72] 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 67/72] 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 68/72] 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 69/72] 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 c646648319632ee972a2b4f25421f47a2774faba Mon Sep 17 00:00:00 2001 From: fjy Date: Wed, 2 Jul 2014 10:07:04 -0700 Subject: [PATCH 70/72] 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 518ab473f3ed1046ea5a907bfc0e712109ab4fdb Mon Sep 17 00:00:00 2001 From: nishantmonu51 Date: Thu, 3 Jul 2014 09:58:12 +0530 Subject: [PATCH 71/72] 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 72/72] 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 ) {