From 77ec4df79718a218acd78a7d45cdfe87935eba83 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 3 Jun 2014 13:43:38 -0700 Subject: [PATCH 01/65] 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 6adc3011fad93d3b1aa7e313dc257873f62b04d0 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 18:25:10 -0700 Subject: [PATCH 02/65] fix post aggs with group by in #472 --- .../io/druid/query/groupby/GroupByQuery.java | 77 +++++++++++----- .../query/groupby/GroupByQueryEngine.java | 69 ++++++++------- .../groupby/GroupByQueryQueryToolChest.java | 19 ++-- .../timeseries/TimeseriesQueryEngine.java | 60 +++++++------ .../query/groupby/GroupByQueryRunnerTest.java | 87 +++++++++++++++++++ 5 files changed, 220 insertions(+), 92 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index 0a23b5704b7..d9431849ea2 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -292,11 +292,25 @@ public class GroupByQuery extends BaseQuery private List orderByColumnSpecs = Lists.newArrayList(); private int limit = Integer.MAX_VALUE; - private Builder() + public Builder() { } - private Builder(Builder builder) + public Builder(GroupByQuery query) + { + dataSource = query.getDataSource(); + querySegmentSpec = query.getQuerySegmentSpec(); + limitSpec = query.getOrderBy(); + dimFilter = query.getDimFilter(); + granularity = query.getGranularity(); + dimensions = query.getDimensions(); + aggregatorSpecs = query.getAggregatorSpecs(); + postAggregatorSpecs = query.getPostAggregatorSpecs(); + havingSpec = query.getHavingSpec(); + context = query.getContext(); + } + + public Builder(Builder builder) { dataSource = builder.dataSource; querySegmentSpec = builder.querySegmentSpec; @@ -515,36 +529,57 @@ public class GroupByQuery extends BaseQuery public String toString() { return "GroupByQuery{" + - "limitSpec=" + limitSpec + - ", dimFilter=" + dimFilter + - ", granularity=" + granularity + - ", dimensions=" + dimensions + - ", aggregatorSpecs=" + aggregatorSpecs + - ", postAggregatorSpecs=" + postAggregatorSpecs + - ", orderByLimitFn=" + orderByLimitFn + - '}'; + "limitSpec=" + limitSpec + + ", dimFilter=" + dimFilter + + ", granularity=" + granularity + + ", dimensions=" + dimensions + + ", aggregatorSpecs=" + aggregatorSpecs + + ", postAggregatorSpecs=" + postAggregatorSpecs + + ", orderByLimitFn=" + orderByLimitFn + + '}'; } @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - if (!super.equals(o)) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } GroupByQuery that = (GroupByQuery) o; - if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) + if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null) { return false; - if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false; - if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false; - if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false; - if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) return false; - if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) return false; - if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null) + } + if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) { return false; - if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null) + } + if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) { return false; + } + if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) { + return false; + } + if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) { + return false; + } + if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) { + return false; + } + if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null) { + return false; + } + if (postAggregatorSpecs != null + ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) + : that.postAggregatorSpecs != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index d9e52014f86..8a42a4e10fe 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -42,7 +42,6 @@ import io.druid.data.input.Row; import io.druid.guice.annotations.Global; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; -import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.DimExtractionFn; import io.druid.segment.Cursor; @@ -104,41 +103,41 @@ public class GroupByQueryEngine return Sequences.concat( Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(@Nullable final Cursor cursor) - { - return new BaseSequence( - new BaseSequence.IteratorMaker() + Sequences.map( + cursors, + new Function>() + { + @Override + public Sequence apply(@Nullable final Cursor cursor) { - @Override - public RowIterator make() - { - return new RowIterator(query, cursor, bufferHolder.get(), config.get()); - } + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public RowIterator make() + { + return new RowIterator(query, cursor, bufferHolder.get(), config.get()); + } - @Override - public void cleanup(RowIterator iterFromMake) - { - Closeables.closeQuietly(iterFromMake); - } + @Override + public void cleanup(RowIterator iterFromMake) + { + Closeables.closeQuietly(iterFromMake); + } + } + ); } - ); + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + Closeables.closeQuietly(bufferHolder); + } } - } - ), - new Closeable() - { - @Override - public void close() throws IOException - { - Closeables.closeQuietly(bufferHolder); - } - } - ) + ) ); } @@ -414,9 +413,9 @@ public class GroupByQueryEngine position += increments[i]; } - for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); - } + //for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + // theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); + //} return new MapBasedRow(timestamp, theEvent); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 0e00ceae46d..75190a66cda 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -43,7 +43,9 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; +import io.druid.query.aggregation.PostAggregator; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; @@ -59,7 +61,10 @@ public class GroupByQueryQueryToolChest extends QueryToolChest NO_MERGE_CONTEXT = ImmutableMap.of(GROUP_BY_MERGE_KEY, "false"); + private static final Map NO_MERGE_CONTEXT = ImmutableMap.of( + GROUP_BY_MERGE_KEY, + "false" + ); private final Supplier configSupplier; private GroupByQueryEngine engine; // For running the outer query around a subquery @@ -92,7 +97,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { - Sequence result; // If there's a subquery, merge subquery results and then apply the aggregator @@ -101,12 +105,17 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); + final GroupByQuery.Builder builder = new GroupByQuery.Builder(subquery); + for (PostAggregator postAggregator : subquery.getPostAggregatorSpecs()) { + builder.addAggregator(new DoubleSumAggregatorFactory(postAggregator.getName(), postAggregator.getName())); + } IncrementalIndexStorageAdapter adapter - = new IncrementalIndexStorageAdapter(makeIncrementalIndex(subquery, subqueryResult)); + = new IncrementalIndexStorageAdapter(makeIncrementalIndex(builder.build(), subqueryResult)); result = engine.process(query, adapter); } else { result = runner.run(query); @@ -128,7 +137,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest>() - { - private final List aggregatorSpecs = query.getAggregatorSpecs(); - private final List postAggregatorSpecs = query.getPostAggregatorSpecs(); + adapter, + query.getQuerySegmentSpec().getIntervals(), + Filters.convertDimensionFilters(query.getDimensionsFilter()), + query.getGranularity(), + new Function>() + { + private final List aggregatorSpecs = query.getAggregatorSpecs(); - @Override - public Result apply(Cursor cursor) - { - Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); + @Override + public Result apply(Cursor cursor) + { + Aggregator[] aggregators = QueryRunnerHelper.makeAggregators(cursor, aggregatorSpecs); - while (!cursor.isDone()) { - for (Aggregator aggregator : aggregators) { - aggregator.aggregate(); - } - cursor.advance(); - } + while (!cursor.isDone()) { + for (Aggregator aggregator : aggregators) { + aggregator.aggregate(); + } + cursor.advance(); + } - TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); + TimeseriesResultBuilder bob = new TimeseriesResultBuilder(cursor.getTime()); - for (Aggregator aggregator : aggregators) { - bob.addMetric(aggregator); - } + for (Aggregator aggregator : aggregators) { + bob.addMetric(aggregator); + } - Result retVal = bob.build(); + Result retVal = bob.build(); - // cleanup - for (Aggregator agg : aggregators) { - agg.close(); - } + // cleanup + for (Aggregator agg : aggregators) { + agg.close(); + } - return retVal; - } - } + return retVal; + } + } ); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 97e64a0ec0c..39cd21f81e1 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -44,6 +44,10 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ArithmeticPostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; +import io.druid.query.aggregation.post.FieldAccessPostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.ExtractionDimensionSpec; @@ -1028,6 +1032,89 @@ public class GroupByQueryRunnerTest Assert.assertFalse(results.iterator().hasNext()); } + @Test + public void testSubqueryWithPostAggregators() + { + + GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }")) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx_subagg", "index") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx_subpostagg", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"), + new ConstantPostAggregator("thousand", 1000, 1000) + ) + ) + + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_agg", "idx"), + new ConstantPostAggregator("ten_thousand", 10000, 10000) + ) + ) + + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + createExpectedRow("2011-04-01", "alias", "mezzanine", "rows", 3L, "idx", 13870.0), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + createExpectedRow("2011-04-01", "alias", "premium", "rows", 3L, "idx", 13900.0), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + + private Iterable runQuery(GroupByQuery query) { From 32b65d9c5b4fd07e8a96ed8663e44369ef96ccce Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 18:26:37 -0700 Subject: [PATCH 03/65] remove commented out code --- .../main/java/io/druid/query/groupby/GroupByQueryEngine.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index 8a42a4e10fe..e158fe078be 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -413,10 +413,6 @@ public class GroupByQueryEngine position += increments[i]; } - //for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { - // theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); - //} - return new MapBasedRow(timestamp, theEvent); } } From 187b2beb47ce960f6cb01712a760218267168e68 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 22:27:33 -0700 Subject: [PATCH 04/65] cleaner way of implementing things --- .../query/GroupByParallelQueryRunner.java | 1 - .../query/aggregation/AggregatorFactory.java | 19 ++++- .../aggregation/CountAggregatorFactory.java | 19 ++++- .../DoubleSumAggregatorFactory.java | 22 ++++-- .../HistogramAggregatorFactory.java | 32 +++++++-- .../JavaScriptAggregatorFactory.java | 16 +++++ .../aggregation/LongSumAggregatorFactory.java | 26 +++++-- .../aggregation/MaxAggregatorFactory.java | 6 ++ .../aggregation/MinAggregatorFactory.java | 6 ++ .../ToLowerCaseAggregatorFactory.java | 6 ++ .../CardinalityAggregatorFactory.java | 17 +++++ .../HyperUniquesAggregatorFactory.java | 6 ++ .../query/groupby/GroupByQueryEngine.java | 71 ++++++++++--------- .../query/groupby/GroupByQueryHelper.java | 6 +- .../groupby/GroupByQueryQueryToolChest.java | 30 +++++--- .../query/groupby/GroupByQueryRunnerTest.java | 7 +- 16 files changed, 216 insertions(+), 74 deletions(-) diff --git a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java index 51c663c6a2e..65925689125 100644 --- a/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java +++ b/processing/src/main/java/io/druid/query/GroupByParallelQueryRunner.java @@ -77,7 +77,6 @@ public class GroupByParallelQueryRunner implements QueryRunner @Override public Sequence run(final Query queryParam) { - final GroupByQuery query = (GroupByQuery) queryParam; final Pair> indexAccumulatorPair = GroupByQueryHelper.createIndexAccumulatorPair( query, diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index 87077e45b31..e236e683d08 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -26,9 +26,9 @@ import java.util.List; /** * Processing related interface - * + *

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

* This is useful as an abstraction to allow Aggregator classes to be written in terms of MetricSelector objects * without making any assumptions about how they are pulling values out of the base data. That is, the data is * provided to the Aggregator through the MetricSelector object, so whatever creates that object gets to choose how @@ -37,7 +37,9 @@ import java.util.List; public interface AggregatorFactory { public Aggregator factorize(ColumnSelectorFactory metricFactory); + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory); + public Comparator getComparator(); /** @@ -48,6 +50,7 @@ public interface AggregatorFactory * * @param lhs The left hand side of the combine * @param rhs The right hand side of the combine + * * @return an object representing the combination of lhs and rhs, this can be a new object or a mutation of the inputs */ public Object combine(Object lhs, Object rhs); @@ -61,11 +64,19 @@ public interface AggregatorFactory */ public AggregatorFactory getCombiningFactory(); + /** + * Gets a list of aggregator factories using the actual column names of the data + * + * @return Base AggregatorFactories for different fields of this AggregatorFactory + */ + public List getBaseFactories(); + /** * A method that knows how to "deserialize" the object from whatever form it might have been put into * in order to transfer via JSON. * * @param object the object to deserialize + * * @return the deserialized object */ public Object deserialize(Object object); @@ -75,13 +86,17 @@ public interface AggregatorFactory * intermediate format than their final resultant output. * * @param object the object to be finalized + * * @return the finalized value that should be returned for the initial query */ public Object finalizeComputation(Object object); public String getName(); + public List requiredFields(); + public byte[] getCacheKey(); + public String getTypeName(); /** diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index e47999e8719..1d1e2e149d7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; import io.druid.segment.ColumnSelectorFactory; +import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -76,6 +77,12 @@ public class CountAggregatorFactory implements AggregatorFactory return new LongSumAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new CountAggregatorFactory(name)); + } + @Override public Object deserialize(Object object) { @@ -136,12 +143,18 @@ public class CountAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } CountAggregatorFactory that = (CountAggregatorFactory) o; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index c7f3eba75f4..7e395c3c3ca 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -85,6 +85,12 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory return new DoubleSumAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new DoubleSumAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { @@ -158,13 +164,21 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } DoubleSumAggregatorFactory that = (DoubleSumAggregatorFactory) o; - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java index 060d40d2798..76809eb39a4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -56,7 +56,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory this.name = name; this.fieldName = fieldName; - this.breaksList = (breaksList == null) ? Lists.newArrayList() :breaksList; + this.breaksList = (breaksList == null) ? Lists.newArrayList() : breaksList; this.breaks = new float[this.breaksList.size()]; for (int i = 0; i < this.breaksList.size(); ++i) { this.breaks[i] = this.breaksList.get(i); @@ -100,6 +100,12 @@ public class HistogramAggregatorFactory implements AggregatorFactory return new HistogramAggregatorFactory(name, name, breaksList); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new HistogramAggregatorFactory(fieldName, fieldName, breaksList)); + } + @Override public Object deserialize(Object object) { @@ -183,15 +189,27 @@ public class HistogramAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } HistogramAggregatorFactory that = (HistogramAggregatorFactory) o; - if (!Arrays.equals(breaks, that.breaks)) return false; - if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) return false; - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (!Arrays.equals(breaks, that.breaks)) { + return false; + } + if (breaksList != null ? !breaksList.equals(that.breaksList) : that.breaksList != null) { + return false; + } + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 0c1e4d2ad9d..ac85b75b8dc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -140,6 +140,22 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory return new JavaScriptAggregatorFactory(name, Lists.newArrayList(name), fnCombine, fnReset, fnCombine); } + @Override + public List getBaseFactories() + { + return Lists.transform( + fieldNames, + new com.google.common.base.Function() + { + @Override + public AggregatorFactory apply(String input) + { + return new JavaScriptAggregatorFactory(name, Arrays.asList(input), fnAggregate, fnReset, fnCombine); + } + } + ); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 50ef5130756..0eaae184e9f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -31,11 +31,11 @@ import java.util.Comparator; import java.util.List; /** -*/ + */ public class LongSumAggregatorFactory implements AggregatorFactory { private static final byte CACHE_TYPE_ID = 0x1; - + private final String fieldName; private final String name; @@ -85,6 +85,12 @@ public class LongSumAggregatorFactory implements AggregatorFactory return new LongSumAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { @@ -154,13 +160,21 @@ public class LongSumAggregatorFactory implements AggregatorFactory @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } LongSumAggregatorFactory that = (LongSumAggregatorFactory) o; - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) { + return false; + } + if (name != null ? !name.equals(that.name) : that.name != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index b731c4319e7..78abafa3d76 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -82,6 +82,12 @@ public class MaxAggregatorFactory implements AggregatorFactory return new MaxAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new MaxAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index d3956c94b52..6c373eded4b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -82,6 +82,12 @@ public class MinAggregatorFactory implements AggregatorFactory return new MinAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new MinAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java index 6c559ba8ec6..b649f1fa5b3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java @@ -65,6 +65,12 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory return baseAggregatorFactory.getCombiningFactory(); } + @Override + public List getBaseFactories() + { + return baseAggregatorFactory.getBaseFactories(); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 2978dba9d9f..eacfede07ea 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -38,6 +38,7 @@ import org.apache.commons.codec.binary.Base64; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -145,6 +146,22 @@ public class CardinalityAggregatorFactory implements AggregatorFactory return new CardinalityAggregatorFactory(name, fieldNames, byRow); } + @Override + public List getBaseFactories() + { + return Lists.transform( + fieldNames, + new Function() + { + @Override + public AggregatorFactory apply(String input) + { + return new CardinalityAggregatorFactory(name, Arrays.asList(input), byRow); + } + } + ); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index fd4e36fb781..2193be3419e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -131,6 +131,12 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory return new HyperUniquesAggregatorFactory(name, name); } + @Override + public List getBaseFactories() + { + return Arrays.asList(new HyperUniquesAggregatorFactory(fieldName, fieldName)); + } + @Override public Object deserialize(Object object) { diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java index e158fe078be..d9e52014f86 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryEngine.java @@ -42,6 +42,7 @@ import io.druid.data.input.Row; import io.druid.guice.annotations.Global; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.DimExtractionFn; import io.druid.segment.Cursor; @@ -103,41 +104,41 @@ public class GroupByQueryEngine return Sequences.concat( Sequences.withBaggage( - Sequences.map( - cursors, - new Function>() - { - @Override - public Sequence apply(@Nullable final Cursor cursor) - { - return new BaseSequence( - new BaseSequence.IteratorMaker() - { - @Override - public RowIterator make() - { - return new RowIterator(query, cursor, bufferHolder.get(), config.get()); - } - - @Override - public void cleanup(RowIterator iterFromMake) - { - Closeables.closeQuietly(iterFromMake); - } - } - ); - } - } - ), - new Closeable() + Sequences.map( + cursors, + new Function>() + { + @Override + public Sequence apply(@Nullable final Cursor cursor) { - @Override - public void close() throws IOException - { - Closeables.closeQuietly(bufferHolder); - } + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public RowIterator make() + { + return new RowIterator(query, cursor, bufferHolder.get(), config.get()); + } + + @Override + public void cleanup(RowIterator iterFromMake) + { + Closeables.closeQuietly(iterFromMake); + } + } + ); } - ) + } + ), + new Closeable() + { + @Override + public void close() throws IOException + { + Closeables.closeQuietly(bufferHolder); + } + } + ) ); } @@ -413,6 +414,10 @@ public class GroupByQueryEngine position += increments[i]; } + for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) { + theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent)); + } + return new MapBasedRow(timestamp, theEvent); } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 00298f18ba0..44fb82b7c6e 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -53,7 +53,7 @@ public class GroupByQueryHelper new Function() { @Override - public AggregatorFactory apply(@Nullable AggregatorFactory input) + public AggregatorFactory apply(AggregatorFactory input) { return input.getCombiningFactory(); } @@ -64,7 +64,7 @@ public class GroupByQueryHelper new Function() { @Override - public String apply(@Nullable DimensionSpec input) + public String apply(DimensionSpec input) { return input.getOutputName(); } @@ -90,7 +90,7 @@ public class GroupByQueryHelper return accumulated; } }; - return new Pair>(index, accumulator); + return new Pair<>(index, accumulator); } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 75190a66cda..681871cfd93 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -24,6 +24,7 @@ import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; import com.metamx.common.Pair; @@ -51,6 +52,8 @@ import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; +import javax.annotation.Nullable; +import java.util.List; import java.util.Map; /** @@ -100,7 +103,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest result; // If there's a subquery, merge subquery results and then apply the aggregator - DataSource dataSource = query.getDataSource(); + final DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { GroupByQuery subquery; try { @@ -109,19 +112,24 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); - final GroupByQuery.Builder builder = new GroupByQuery.Builder(subquery); - for (PostAggregator postAggregator : subquery.getPostAggregatorSpecs()) { - builder.addAggregator(new DoubleSumAggregatorFactory(postAggregator.getName(), postAggregator.getName())); + final Sequence subqueryResult = mergeGroupByResults(subquery, runner); + final List aggs = Lists.newArrayList(); + for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { + aggs.addAll(aggregatorFactory.getBaseFactories()); } - IncrementalIndexStorageAdapter adapter - = new IncrementalIndexStorageAdapter(makeIncrementalIndex(builder.build(), subqueryResult)); - result = engine.process(query, adapter); + final GroupByQuery innerQuery = new GroupByQuery.Builder(query).setAggregatorSpecs(aggs) + .setInterval(subquery.getIntervals()) + .setPostAggregatorSpecs(Lists.newArrayList()) + .build(); + + final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter( + makeIncrementalIndex(innerQuery, subqueryResult) + ); + return engine.process(query, adapter); } else { result = runner.run(query); + return postAggregate(query, makeIncrementalIndex(query, result)); } - - return postAggregate(query, makeIncrementalIndex(query, result)); } @@ -161,7 +169,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeSequences(Sequence> seqOfSequences) { - return new ConcatSequence(seqOfSequences); + return new ConcatSequence<>(seqOfSequences); } @Override diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 39cd21f81e1..6b44645ac81 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -1035,8 +1035,7 @@ public class GroupByQueryRunnerTest @Test public void testSubqueryWithPostAggregators() { - - GroupByQuery subquery = GroupByQuery + final GroupByQuery subquery = GroupByQuery .builder() .setDataSource(QueryRunnerTestHelper.dataSource) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -1062,7 +1061,7 @@ public class GroupByQueryRunnerTest .setGranularity(QueryRunnerTestHelper.dayGran) .build(); - GroupByQuery query = GroupByQuery + final GroupByQuery query = GroupByQuery .builder() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) @@ -1076,7 +1075,7 @@ public class GroupByQueryRunnerTest .setPostAggregatorSpecs( Arrays.asList( new ArithmeticPostAggregator( - "idx", "+", Arrays.asList( + "idx", "+", Arrays.asList( new FieldAccessPostAggregator("the_idx_agg", "idx"), new ConstantPostAggregator("ten_thousand", 10000, 10000) ) From 5955ecf032d678aa27095bd5193e24ed865b32b4 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 9 Jun 2014 22:31:30 -0700 Subject: [PATCH 05/65] fix bugs wtih js and card aggs --- .../io/druid/query/aggregation/JavaScriptAggregatorFactory.java | 2 +- .../aggregation/cardinality/CardinalityAggregatorFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index ac85b75b8dc..90175c54b88 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -150,7 +150,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new JavaScriptAggregatorFactory(name, Arrays.asList(input), fnAggregate, fnReset, fnCombine); + return new JavaScriptAggregatorFactory(input, Arrays.asList(input), fnAggregate, fnReset, fnCombine); } } ); diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index eacfede07ea..4c79b076533 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -156,7 +156,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new CardinalityAggregatorFactory(name, Arrays.asList(input), byRow); + return new CardinalityAggregatorFactory(input, Arrays.asList(input), byRow); } } ); From cde86d81e20fd14255eb95fe7e39f058657e3b04 Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 10 Jun 2014 15:50:13 -0700 Subject: [PATCH 06/65] make having and limitspec work for nested queries --- .../io/druid/query/groupby/GroupByQuery.java | 41 +++++---- .../groupby/GroupByQueryQueryToolChest.java | 30 +++--- .../groupby/orderby/DefaultLimitSpec.java | 65 +++++++++---- .../query/groupby/orderby/LimitSpec.java | 8 +- .../query/groupby/orderby/NoopLimitSpec.java | 6 ++ .../query/groupby/GroupByQueryRunnerTest.java | 92 +++++++++++++++++++ 6 files changed, 192 insertions(+), 50 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java index d9431849ea2..98ac83e32cd 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQuery.java @@ -72,7 +72,7 @@ public class GroupByQuery extends BaseQuery private final List aggregatorSpecs; private final List postAggregatorSpecs; - private final Function, Sequence> orderByLimitFn; + private final Function, Sequence> limitFn; @JsonCreator public GroupByQuery( @@ -85,8 +85,9 @@ public class GroupByQuery extends BaseQuery @JsonProperty("postAggregations") List postAggregatorSpecs, @JsonProperty("having") HavingSpec havingSpec, @JsonProperty("limitSpec") LimitSpec limitSpec, - @JsonProperty("orderBy") LimitSpec orderBySpec, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + // Backwards compatible + @JsonProperty("orderBy") LimitSpec orderBySpec ) { super(dataSource, querySegmentSpec, context); @@ -129,7 +130,7 @@ public class GroupByQuery extends BaseQuery ); } - orderByLimitFn = postProcFn; + limitFn = postProcFn; } /** @@ -146,7 +147,7 @@ public class GroupByQuery extends BaseQuery List postAggregatorSpecs, HavingSpec havingSpec, LimitSpec orderBySpec, - Function, Sequence> orderByLimitFn, + Function, Sequence> limitFn, Map context ) { @@ -159,7 +160,7 @@ public class GroupByQuery extends BaseQuery this.postAggregatorSpecs = postAggregatorSpecs; this.havingSpec = havingSpec; this.limitSpec = orderBySpec; - this.orderByLimitFn = orderByLimitFn; + this.limitFn = limitFn; } @JsonProperty("filter") @@ -199,7 +200,7 @@ public class GroupByQuery extends BaseQuery } @JsonProperty - public LimitSpec getOrderBy() + public LimitSpec getLimitSpec() { return limitSpec; } @@ -218,7 +219,7 @@ public class GroupByQuery extends BaseQuery public Sequence applyLimit(Sequence results) { - return orderByLimitFn.apply(results); + return limitFn.apply(results); } @Override @@ -234,7 +235,7 @@ public class GroupByQuery extends BaseQuery postAggregatorSpecs, havingSpec, limitSpec, - orderByLimitFn, + limitFn, computeOverridenContext(contextOverride) ); } @@ -252,7 +253,7 @@ public class GroupByQuery extends BaseQuery postAggregatorSpecs, havingSpec, limitSpec, - orderByLimitFn, + limitFn, getContext() ); } @@ -270,7 +271,7 @@ public class GroupByQuery extends BaseQuery postAggregatorSpecs, havingSpec, limitSpec, - orderByLimitFn, + limitFn, getContext() ); } @@ -300,7 +301,7 @@ public class GroupByQuery extends BaseQuery { dataSource = query.getDataSource(); querySegmentSpec = query.getQuerySegmentSpec(); - limitSpec = query.getOrderBy(); + limitSpec = query.getLimitSpec(); dimFilter = query.getDimFilter(); granularity = query.getGranularity(); dimensions = query.getDimensions(); @@ -504,7 +505,11 @@ public class GroupByQuery extends BaseQuery { final LimitSpec theLimitSpec; if (limitSpec == null) { - theLimitSpec = new DefaultLimitSpec(orderByColumnSpecs, limit); + if (orderByColumnSpecs.isEmpty() && limit == Integer.MAX_VALUE) { + theLimitSpec = new NoopLimitSpec(); + } else { + theLimitSpec = new DefaultLimitSpec(orderByColumnSpecs, limit); + } } else { theLimitSpec = limitSpec; } @@ -518,9 +523,9 @@ public class GroupByQuery extends BaseQuery aggregatorSpecs, postAggregatorSpecs, havingSpec, - null, theLimitSpec, - context + context, + null ); } } @@ -535,7 +540,7 @@ public class GroupByQuery extends BaseQuery ", dimensions=" + dimensions + ", aggregatorSpecs=" + aggregatorSpecs + ", postAggregatorSpecs=" + postAggregatorSpecs + - ", orderByLimitFn=" + orderByLimitFn + + ", limitFn=" + limitFn + '}'; } @@ -572,7 +577,7 @@ public class GroupByQuery extends BaseQuery if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) { return false; } - if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null) { + if (limitFn != null ? !limitFn.equals(that.limitFn) : that.limitFn != null) { return false; } if (postAggregatorSpecs != null @@ -595,7 +600,7 @@ public class GroupByQuery extends BaseQuery result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0); result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0); result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0); - result = 31 * result + (orderByLimitFn != null ? orderByLimitFn.hashCode() : 0); + result = 31 * result + (limitFn != null ? limitFn.hashCode() : 0); return result; } } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 681871cfd93..09cf7e431a8 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -44,7 +44,6 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryToolChest; import io.druid.query.SubqueryQueryRunner; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.incremental.IncrementalIndex; @@ -52,7 +51,6 @@ import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.Interval; import org.joda.time.Minutes; -import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -89,7 +87,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest run(Query input) { - if (Boolean.valueOf((String) input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { + if (Boolean.valueOf(input.getContextValue(GROUP_BY_MERGE_KEY, "true"))) { return mergeGroupByResults(((GroupByQuery) input).withOverriddenContext(NO_MERGE_CONTEXT), runner); } else { return runner.run(input); @@ -100,8 +98,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest mergeGroupByResults(final GroupByQuery query, QueryRunner runner) { - Sequence result; - // If there's a subquery, merge subquery results and then apply the aggregator final DataSource dataSource = query.getDataSource(); if (dataSource instanceof QueryDataSource) { @@ -117,25 +113,30 @@ public class GroupByQueryQueryToolChest extends QueryToolChestnewArrayList()) - .build(); + + // We need the inner incremental index to have all the columns required by the outer query + final GroupByQuery innerQuery = new GroupByQuery.Builder(query) + .setAggregatorSpecs(aggs) + .setInterval(subquery.getIntervals()) + .setPostAggregatorSpecs(Lists.newArrayList()) + .build(); + + final GroupByQuery outerQuery = new GroupByQuery.Builder(query) + .setLimitSpec(query.getLimitSpec().merge(subquery.getLimitSpec())) + .build(); final IncrementalIndexStorageAdapter adapter = new IncrementalIndexStorageAdapter( makeIncrementalIndex(innerQuery, subqueryResult) ); - return engine.process(query, adapter); + return outerQuery.applyLimit(engine.process(outerQuery, adapter)); } else { - result = runner.run(query); - return postAggregate(query, makeIncrementalIndex(query, result)); + return query.applyLimit(postAggregate(query, makeIncrementalIndex(query, runner.run(query)))); } } - private Sequence postAggregate(final GroupByQuery query, IncrementalIndex index) { - Sequence sequence = Sequences.map( + return Sequences.map( Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs())), new Function() { @@ -151,7 +152,6 @@ public class GroupByQueryQueryToolChest extends QueryToolChest rows) diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java index eda54ea0dc3..3d78e112cb5 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -87,12 +87,17 @@ public class DefaultLimitSpec implements LimitSpec if (limit == Integer.MAX_VALUE) { return new SortingFn(ordering); - } - else { + } else { return new TopNFunction(ordering, limit); } } + @Override + public LimitSpec merge(LimitSpec other) + { + return this; + } + private Ordering makeComparator( List dimensions, List aggs, List postAggs ) @@ -200,12 +205,18 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } LimitingFn that = (LimitingFn) o; - if (limit != that.limit) return false; + if (limit != that.limit) { + return false; + } return true; } @@ -232,12 +243,18 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } SortingFn sortingFn = (SortingFn) o; - if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) return false; + if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) { + return false; + } return true; } @@ -273,13 +290,21 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } TopNFunction that = (TopNFunction) o; - if (limit != that.limit) return false; - if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) return false; + if (limit != that.limit) { + return false; + } + if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) { + return false; + } return true; } @@ -296,13 +321,21 @@ public class DefaultLimitSpec implements LimitSpec @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } DefaultLimitSpec that = (DefaultLimitSpec) o; - if (limit != that.limit) return false; - if (columns != null ? !columns.equals(that.columns) : that.columns != null) return false; + if (limit != that.limit) { + return false; + } + if (columns != null ? !columns.equals(that.columns) : that.columns != null) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java index 0d07f1f91c9..fa50d62016c 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/LimitSpec.java @@ -38,5 +38,11 @@ import java.util.List; }) public interface LimitSpec { - public Function, Sequence> build(List dimensions, List aggs, List postAggs); + public Function, Sequence> build( + List dimensions, + List aggs, + List postAggs + ); + + public LimitSpec merge(LimitSpec other); } diff --git a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java index d975e24a65f..e71038d4918 100644 --- a/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java +++ b/processing/src/main/java/io/druid/query/groupby/orderby/NoopLimitSpec.java @@ -41,6 +41,12 @@ public class NoopLimitSpec implements LimitSpec return Functions.identity(); } + @Override + public LimitSpec merge(LimitSpec other) + { + return other; + } + @Override public String toString() { diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 6b44645ac81..511b042e3e6 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -56,6 +56,7 @@ import io.druid.query.filter.JavaScriptDimFilter; import io.druid.query.filter.RegexDimFilter; import io.druid.query.groupby.having.EqualToHavingSpec; import io.druid.query.groupby.having.GreaterThanHavingSpec; +import io.druid.query.groupby.having.HavingSpec; import io.druid.query.groupby.having.OrHavingSpec; import io.druid.query.groupby.orderby.DefaultLimitSpec; import io.druid.query.groupby.orderby.LimitSpec; @@ -1113,6 +1114,97 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testSubqueryWithPostAggregatorsAndHaving() + { + final GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("quality", "function(dim){ return true; }")) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx_subagg", "index") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx_subpostagg", + "+", + Arrays.asList( + new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"), + new ConstantPostAggregator("thousand", 1000, 1000) + ) + ) + + ) + ) + .setHavingSpec( + new HavingSpec() + { + @Override + public boolean eval(Row row) + { + return (row.getFloatMetric("idx_subpostagg") < 3800); + } + } + ) + .addOrderByColumn("alias") + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_agg", "idx"), + new ConstantPostAggregator("ten_thousand", 10000, 10000) + ) + ) + + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } private Iterable runQuery(GroupByQuery query) { From 22e368e79b9d3d370cde224b798c0c9c15789d93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 11:22:41 -0700 Subject: [PATCH 07/65] add dimension/column selector to incremental index --- .../segment/incremental/IncrementalIndex.java | 97 +++++++++++++++++-- 1 file changed, 90 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 057a4b95ff2..8ab7f06f7b8 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -46,6 +46,7 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.TimestampColumnSelector; +import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; @@ -53,6 +54,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; import java.util.LinkedHashMap; @@ -186,7 +188,7 @@ public class IncrementalIndex implements Iterable dims = newDims; } - TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims); + final TimeAndDims key = new TimeAndDims(Math.max(gran.truncate(row.getTimestampFromEpoch()), minTimestamp), dims); Aggregator[] aggs = facts.get(key); if (aggs == null) { @@ -248,8 +250,34 @@ public class IncrementalIndex implements Iterable }; } - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + final List dimensionValues = in.getDimension(columnName); + if (dimensionValues != null) { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Object.class; + } + @Override + public Object get() + { + final String[] dimVals = dimensionValues.toArray(new String[]{}); + if (dimVals.length == 1) { + return dimVals[0]; + } + else if (dimVals.length == 0) { + return null; + } + else { + return dimVals; + } + } + }; + } + + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); if (serde == null) { throw new ISE("Don't know how to handle type[%s]", typeName); } @@ -273,12 +301,67 @@ public class IncrementalIndex implements Iterable } @Override - public DimensionSelector makeDimensionSelector(String dimension) + public DimensionSelector makeDimensionSelector(final String dimension) { - // we should implement this, but this is going to be rewritten soon anyways - throw new UnsupportedOperationException( - "Incremental index aggregation does not support dimension selectors" - ); + final String dimensionName = dimension.toLowerCase(); + final List dimensionValues = in.getDimension(dimensionName); + if (dimensionValues == null) { + return null; + } + + final IncrementalIndex.DimDim dimValLookup = getDimension(dimensionName); + final int maxId = dimValLookup.size(); + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + final ArrayList vals = Lists.newArrayList(); + for (String dimVal : dimensionValues) { + int id = dimValLookup.getId(dimVal); + vals.add(id); + } + + return new IndexedInts() + { + @Override + public int size() + { + return vals.size(); + } + + @Override + public int get(int index) + { + return vals.get(index); + } + + @Override + public Iterator iterator() + { + return vals.iterator(); + } + }; + } + + @Override + public int getValueCardinality() + { + return maxId; + } + + @Override + public String lookupName(int id) + { + return dimValLookup.getValue(id); + } + + @Override + public int lookupId(String name) + { + return dimValLookup.getId(name); + } + }; } } From 403962fe0b8e06235d61d511ee87bb226f9de0ab Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 12 Jun 2014 11:36:53 -0700 Subject: [PATCH 08/65] add some failing tests for groupBy --- .../JavaScriptAggregatorFactory.java | 2 +- .../CardinalityAggregatorFactory.java | 2 +- .../segment/incremental/IncrementalIndex.java | 15 +-- .../query/groupby/GroupByQueryRunnerTest.java | 112 ++++++++++++++++++ 4 files changed, 120 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 90175c54b88..9f84c320702 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -150,7 +150,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new JavaScriptAggregatorFactory(input, Arrays.asList(input), fnAggregate, fnReset, fnCombine); + return new JavaScriptAggregatorFactory(input, fieldNames, fnAggregate, fnReset, fnCombine); } } ); diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 4c79b076533..e8c414913f4 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -156,7 +156,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory apply(String input) { - return new CardinalityAggregatorFactory(input, Arrays.asList(input), byRow); + return new CardinalityAggregatorFactory(input, fieldNames, byRow); } } ); diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 8ab7f06f7b8..1c14d1a0447 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -237,7 +237,7 @@ public class IncrementalIndex implements Iterable return new ObjectColumnSelector() { @Override - public Class classOfObject() + public Class classOfObject() { return Float.TYPE; } @@ -250,12 +250,11 @@ public class IncrementalIndex implements Iterable }; } - final List dimensionValues = in.getDimension(columnName); - if (dimensionValues != null) { + if (getDimension(columnName) != null) { return new ObjectColumnSelector() { @Override - public Class classOfObject() + public Class classOfObject() { return Object.class; } @@ -263,14 +262,12 @@ public class IncrementalIndex implements Iterable @Override public Object get() { - final String[] dimVals = dimensionValues.toArray(new String[]{}); + final String[] dimVals = in.getDimension(columnName).toArray(new String[]{}); if (dimVals.length == 1) { return dimVals[0]; - } - else if (dimVals.length == 0) { + } else if (dimVals.length == 0) { return null; - } - else { + } else { return dimVals; } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 511b042e3e6..5cfc101dcd9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -42,6 +42,7 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryToolChest; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -1206,6 +1207,117 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testSubqueryWithEverything() + { + final GroupByQuery subquery = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "alias"))) + .setDimFilter(new JavaScriptDimFilter("provider", "function(dim){ return true; }")) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + new LongSumAggregatorFactory("idx_subagg", "index"), + new JavaScriptAggregatorFactory( + "js_agg", + Arrays.asList("index", "provider"), + "function(index, dim){return index + dim.length;}", + "function(){return 0;}", + "function(a,b){return a + b;}" + ) + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx_subpostagg", + "+", + Arrays.asList( + new FieldAccessPostAggregator("the_idx_subagg", "idx_subagg"), + new ConstantPostAggregator("thousand", 1000, 1000) + ) + ) + + ) + ) + .setHavingSpec( + new HavingSpec() + { + @Override + public boolean eval(Row row) + { + return (row.getFloatMetric("idx_subpostagg") < 3800); + } + } + ) + .addOrderByColumn("alias") + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + final GroupByQuery query = GroupByQuery + .builder() + .setDataSource(subquery) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("alias", "alias"))) + .setAggregatorSpecs( + Arrays.asList( + new LongSumAggregatorFactory("rows", "rows"), + new LongSumAggregatorFactory("idx", "idx_subpostagg"), + new DoubleSumAggregatorFactory("js_outer_agg", "js_agg") + ) + ) + .setPostAggregatorSpecs( + Arrays.asList( + new ArithmeticPostAggregator( + "idx", "+", Arrays.asList( + new FieldAccessPostAggregator("the_idx_agg", "idx"), + new ConstantPostAggregator("ten_thousand", 10000, 10000) + ) + ) + + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Arrays.asList( + new OrderByColumnSpec( + "alias", + OrderByColumnSpec.Direction.DESCENDING + ) + ), + 5 + ) + ) + .setGranularity(QueryRunnerTestHelper.dayGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), + createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), + + createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), + createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), + createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), + createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), + createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), + createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), + createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), + createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), + createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + ); + + // Subqueries are handled by the ToolChest + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + private Iterable runQuery(GroupByQuery query) { From e84bcca40f868b0738f60fc49ccaf3e39d969366 Mon Sep 17 00:00:00 2001 From: fjy Date: Thu, 12 Jun 2014 13:36:40 -0700 Subject: [PATCH 09/65] rename method --- .../java/io/druid/query/aggregation/AggregatorFactory.java | 6 +++--- .../io/druid/query/aggregation/CountAggregatorFactory.java | 2 +- .../druid/query/aggregation/DoubleSumAggregatorFactory.java | 2 +- .../druid/query/aggregation/HistogramAggregatorFactory.java | 2 +- .../query/aggregation/JavaScriptAggregatorFactory.java | 2 +- .../druid/query/aggregation/LongSumAggregatorFactory.java | 2 +- .../io/druid/query/aggregation/MaxAggregatorFactory.java | 2 +- .../io/druid/query/aggregation/MinAggregatorFactory.java | 2 +- .../query/aggregation/ToLowerCaseAggregatorFactory.java | 4 ++-- .../cardinality/CardinalityAggregatorFactory.java | 2 +- .../hyperloglog/HyperUniquesAggregatorFactory.java | 2 +- .../io/druid/query/groupby/GroupByQueryQueryToolChest.java | 2 +- 12 files changed, 15 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java index e236e683d08..ad355c71b6b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -65,11 +65,11 @@ public interface AggregatorFactory public AggregatorFactory getCombiningFactory(); /** - * Gets a list of aggregator factories using the actual column names of the data + * Gets a list of all columns that this AggregatorFactory will scan * - * @return Base AggregatorFactories for different fields of this AggregatorFactory + * @return AggregatorFactories for the columns to scan of the parent AggregatorFactory */ - public List getBaseFactories(); + public List getRequiredColumns(); /** * A method that knows how to "deserialize" the object from whatever form it might have been put into diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java index 1d1e2e149d7..9549015f687 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -78,7 +78,7 @@ public class CountAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new CountAggregatorFactory(name)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java index 7e395c3c3ca..083a16589dc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -86,7 +86,7 @@ public class DoubleSumAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new DoubleSumAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java index 76809eb39a4..422d8279a2e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -101,7 +101,7 @@ public class HistogramAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new HistogramAggregatorFactory(fieldName, fieldName, breaksList)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java index 9f84c320702..fc5603f2fe9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -141,7 +141,7 @@ public class JavaScriptAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Lists.transform( fieldNames, diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java index 0eaae184e9f..963d9458beb 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -86,7 +86,7 @@ public class LongSumAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new LongSumAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index 78abafa3d76..56e6de5f56e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -83,7 +83,7 @@ public class MaxAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new MaxAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index 6c373eded4b..e7256d0ccb9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -83,7 +83,7 @@ public class MinAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new MinAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java index b649f1fa5b3..a884bb9be4d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/ToLowerCaseAggregatorFactory.java @@ -66,9 +66,9 @@ public class ToLowerCaseAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { - return baseAggregatorFactory.getBaseFactories(); + return baseAggregatorFactory.getRequiredColumns(); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index e8c414913f4..12da44bea07 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -147,7 +147,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Lists.transform( fieldNames, diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index 2193be3419e..26449929703 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -132,7 +132,7 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory } @Override - public List getBaseFactories() + public List getRequiredColumns() { return Arrays.asList(new HyperUniquesAggregatorFactory(fieldName, fieldName)); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 09cf7e431a8..025af21a2d7 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -111,7 +111,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest subqueryResult = mergeGroupByResults(subquery, runner); final List aggs = Lists.newArrayList(); for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) { - aggs.addAll(aggregatorFactory.getBaseFactories()); + aggs.addAll(aggregatorFactory.getRequiredColumns()); } // We need the inner incremental index to have all the columns required by the outer query From 150480bdf98e82bb781b67d127370ca15312fe46 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 12:44:26 -0700 Subject: [PATCH 10/65] complexmetricserde workaround for groupby --- .../query/groupby/GroupByQueryHelper.java | 2 +- .../segment/incremental/IncrementalIndex.java | 95 +++++++++---------- 2 files changed, 45 insertions(+), 52 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 44fb82b7c6e..7041fa970da 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -83,7 +83,7 @@ public class GroupByQueryHelper @Override public IncrementalIndex accumulate(IncrementalIndex accumulated, Row in) { - if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions)) > config.getMaxResults()) { + if (accumulated.add(Rows.toCaseInsensitiveInputRow(in, dimensions), false) > config.getMaxResults()) { throw new ISE("Computation exceeds maxRows limit[%s]", config.getMaxResults()); } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 1c14d1a0447..25aa47463ed 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -135,17 +135,29 @@ public class IncrementalIndex implements Iterable ); } + public int add(InputRow row) + { + // this is an ugly workaround to call ComplexMetricExtractor.extractValue at ingestion time + return add(row, true); + } + /** * Adds a new row. The row might correspond with another row that already exists, in which case this will * update that row instead of inserting a new one. *

- * This is *not* thread-safe. Calls to add() should always happen on the same thread. + * + * Calls to add() are thread safe. + * + * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that + * should not deserialize input columns using ComplexMetricSerde for aggregators that return complex metrics. * * @param row the row of data to add + * @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input + * value for aggregators that return metrics other than float. * * @return the number of rows in the data set after adding the InputRow */ - public int add(InputRow row) + public int add(InputRow row, final boolean deserializeComplexMetrics) { row = spatialDimensionRowFormatter.formatRow(row); @@ -233,25 +245,7 @@ public class IncrementalIndex implements Iterable final String typeName = agg.getTypeName(); final String columnName = column.toLowerCase(); - if (typeName.equals("float")) { - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return Float.TYPE; - } - - @Override - public Float get() - { - return in.getFloatMetric(columnName); - } - }; - } - - if (getDimension(columnName) != null) { - return new ObjectColumnSelector() + final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() { @Override public Class classOfObject() @@ -262,39 +256,38 @@ public class IncrementalIndex implements Iterable @Override public Object get() { - final String[] dimVals = in.getDimension(columnName).toArray(new String[]{}); - if (dimVals.length == 1) { - return dimVals[0]; - } else if (dimVals.length == 0) { - return null; - } else { - return dimVals; - } + return in.getRaw(columnName); + } + }; + + if(!deserializeComplexMetrics) { + return rawColumnSelector; + } else { + if (typeName.equals("float")) { + return rawColumnSelector; + } + + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + if (serde == null) { + throw new ISE("Don't know how to handle type[%s]", typeName); + } + + final ComplexMetricExtractor extractor = serde.getExtractor(); + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return extractor.extractedClass(); + } + + @Override + public Object get() + { + return extractor.extractValue(in, columnName); } }; } - - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); - if (serde == null) { - throw new ISE("Don't know how to handle type[%s]", typeName); - } - - final ComplexMetricExtractor extractor = serde.getExtractor(); - - return new ObjectColumnSelector() - { - @Override - public Class classOfObject() - { - return extractor.extractedClass(); - } - - @Override - public Object get() - { - return extractor.extractValue(in, columnName); - } - }; } @Override From fd47efd59e695cb6ecb97af0829bbcfe3b6a0dc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 15:14:30 -0700 Subject: [PATCH 11/65] fix cardinality combining factory --- .../aggregation/cardinality/CardinalityAggregatorFactory.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 12da44bea07..10443c828cd 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -32,6 +32,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; +import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; import org.apache.commons.codec.binary.Base64; @@ -143,7 +144,7 @@ public class CardinalityAggregatorFactory implements AggregatorFactory @Override public AggregatorFactory getCombiningFactory() { - return new CardinalityAggregatorFactory(name, fieldNames, byRow); + return new HyperUniquesAggregatorFactory(name, name); } @Override From bfa3caa58949b5b987d3fea4911b8677cbc4fa58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 15:16:07 -0700 Subject: [PATCH 12/65] js/cardinality/hyperUnique happy groupBy family --- .../HyperUniquesAggregatorFactory.java | 10 ++-- .../query/groupby/GroupByQueryHelper.java | 2 + .../segment/incremental/IncrementalIndex.java | 23 +++----- .../io/druid/query/QueryRunnerTestHelper.java | 6 ++ .../query/groupby/GroupByQueryRunnerTest.java | 58 ++++++++++++------- 5 files changed, 61 insertions(+), 38 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index 26449929703..e4597c71a0b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -73,12 +73,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory return Aggregators.noopAggregator(); } - if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) { + final Class classOfObject = selector.classOfObject(); + if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) { return new HyperUniquesAggregator(name, selector); } throw new IAE( - "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject() + "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject ); } @@ -91,12 +92,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory return Aggregators.noopBufferAggregator(); } - if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) { + final Class classOfObject = selector.classOfObject(); + if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) { return new HyperUniquesBufferAggregator(selector); } throw new IAE( - "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject() + "Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject ); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java index 7041fa970da..b7183b12354 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryHelper.java @@ -24,12 +24,14 @@ import com.google.common.collect.Lists; import com.metamx.common.ISE; import com.metamx.common.Pair; import com.metamx.common.guava.Accumulator; +import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.data.input.Rows; import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.dimension.DimensionSpec; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; import javax.annotation.Nullable; import java.util.List; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java index 25aa47463ed..4f6f6a0f67f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -248,7 +248,7 @@ public class IncrementalIndex implements Iterable final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() { @Override - public Class classOfObject() + public Class classOfObject() { return Object.class; } @@ -294,22 +294,17 @@ public class IncrementalIndex implements Iterable public DimensionSelector makeDimensionSelector(final String dimension) { final String dimensionName = dimension.toLowerCase(); - final List dimensionValues = in.getDimension(dimensionName); - if (dimensionValues == null) { - return null; - } - - final IncrementalIndex.DimDim dimValLookup = getDimension(dimensionName); - final int maxId = dimValLookup.size(); return new DimensionSelector() { @Override public IndexedInts getRow() { + final List dimensionValues = in.getDimension(dimensionName); final ArrayList vals = Lists.newArrayList(); - for (String dimVal : dimensionValues) { - int id = dimValLookup.getId(dimVal); - vals.add(id); + if (dimensionValues != null) { + for (int i = 0; i < dimensionValues.size(); ++i) { + vals.add(i); + } } return new IndexedInts() @@ -337,19 +332,19 @@ public class IncrementalIndex implements Iterable @Override public int getValueCardinality() { - return maxId; + throw new UnsupportedOperationException("value cardinality is unknown in incremental index"); } @Override public String lookupName(int id) { - return dimValLookup.getValue(id); + return in.getDimension(dimensionName).get(id); } @Override public int lookupId(String name) { - return dimValLookup.getId(name); + return in.getDimension(dimensionName).indexOf(name); } }; } diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index 55c29752ac1..deeaff563e8 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -28,6 +28,7 @@ import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; @@ -110,6 +111,11 @@ public class QueryRunnerTestHelper "uniques", "quality_uniques" ); + public static final CardinalityAggregatorFactory qualityCardinality = new CardinalityAggregatorFactory( + "cardinality", + Arrays.asList("quality"), + false + ); public static final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null); public static final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows"); public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index"); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index a43cec90f77..d718c0775d9 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -217,6 +217,36 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, results, ""); } + @Test + public void testGroupByWithCardinality() + { + GroupByQuery query = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.qualityCardinality + ) + ) + .setGranularity(QueryRunnerTestHelper.allGran) + .build(); + + List expectedResults = Arrays.asList( + createExpectedRow( + "2011-04-01", + "rows", + 26L, + "cardinality", + QueryRunnerTestHelper.UNIQUES_9 + ) + ); + + Iterable results = runQuery(query); + TestHelper.assertExpectedObjects(expectedResults, results, ""); + } + @Test public void testGroupByWithDimExtractionFn() { @@ -1209,7 +1239,7 @@ public class GroupByQueryRunnerTest } @Test - public void testSubqueryWithEverything() + public void testSubqueryWithMultiColumnAggregators() { final GroupByQuery subquery = GroupByQuery .builder() @@ -1220,11 +1250,11 @@ public class GroupByQueryRunnerTest .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.rowsCount, - new LongSumAggregatorFactory("idx_subagg", "index"), + new DoubleSumAggregatorFactory("idx_subagg", "index"), new JavaScriptAggregatorFactory( "js_agg", Arrays.asList("index", "provider"), - "function(index, dim){return index + dim.length;}", + "function(current, index, dim){return current + index + dim.length;}", "function(){return 0;}", "function(a,b){return a + b;}" ) @@ -1295,23 +1325,11 @@ public class GroupByQueryRunnerTest .build(); List expectedResults = Arrays.asList( - createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0), - createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0), - createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0), - createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0), - createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0), - createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0), - createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0), - - createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0), - createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0), - createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0), - createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0), - createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0), - createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0), - createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0), - createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0), - createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0) + createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0, "js_outer_agg", 123.92274475097656), + createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0, "js_outer_agg", 82.62254333496094), + createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0, "js_outer_agg", 125.58358001708984), + createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0, "js_outer_agg", 124.13470458984375), + createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0, "js_outer_agg", 162.74722290039062) ); // Subqueries are handled by the ToolChest From 8f7c82d35152f9f869506b9495e768feb05fcb31 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 15:44:04 -0700 Subject: [PATCH 13/65] avoid closing yielder prematurely --- .../DruidDefaultSerializersModule.java | 4 +- .../java/io/druid/server/QueryResource.java | 46 +++++++++++-------- 2 files changed, 30 insertions(+), 20 deletions(-) diff --git a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java index 6184221a1db..a0cc2b87f73 100644 --- a/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/io/druid/jackson/DruidDefaultSerializersModule.java @@ -132,17 +132,17 @@ public class DruidDefaultSerializersModule extends SimpleModule public void serialize(Yielder yielder, final JsonGenerator jgen, SerializerProvider provider) throws IOException, JsonProcessingException { - jgen.writeStartArray(); try { + jgen.writeStartArray(); while (!yielder.isDone()) { final Object o = yielder.get(); jgen.writeObject(o); yielder = yielder.next(null); } + jgen.writeEndArray(); } finally { yielder.close(); } - jgen.writeEndArray(); } } ); diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 33bdd519c83..8300b028ff7 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -147,28 +147,29 @@ public class QueryResource log.debug("Got query [%s]", query); } - Sequence results = query.run(texasRanger); - - if (results == null) { + Sequence res = query.run(texasRanger); + final Sequence results; + if (res == null) { results = Sequences.empty(); + } else { + results = res; } - try ( - final Yielder yielder = results.toYielder( - null, - new YieldingAccumulator() - { - @Override - public Object accumulate(Object accumulated, Object in) - { - yield(); - return in; - } - } - ) - ) { - long requestTime = System.currentTimeMillis() - start; + final Yielder yielder = results.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Object accumulate(Object accumulated, Object in) + { + yield(); + return in; + } + } + ); + try { + long requestTime = System.currentTimeMillis() - start; emitter.emit( new ServiceMetricEvent.Builder() .setUser2(DataSourceUtil.getMetricName(query.getDataSource())) @@ -202,6 +203,7 @@ public class QueryResource @Override public void write(OutputStream outputStream) throws IOException, WebApplicationException { + // json serializer will always close the yielder jsonWriter.writeValue(outputStream, yielder); outputStream.close(); } @@ -211,6 +213,14 @@ public class QueryResource .header("X-Druid-Query-Id", queryId) .build(); } + catch (Exception e) { + // make sure to close yieder if anything happened before starting to serialize the response. + yielder.close(); + } + finally { + // do not close yielder here, since we do not want to close the yielder prior to + // StreamingOutput having iterated over all the results + } } catch (QueryInterruptedException e) { try { From b2419b1530764ce72ea47f12f89208f9b2293488 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Thu, 12 Jun 2014 17:09:21 -0700 Subject: [PATCH 14/65] fix missing return statement --- server/src/main/java/io/druid/server/QueryResource.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 8300b028ff7..7d7d39b402d 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -216,6 +216,7 @@ public class QueryResource catch (Exception e) { // make sure to close yieder if anything happened before starting to serialize the response. yielder.close(); + throw Throwables.propagate(e); } finally { // do not close yielder here, since we do not want to close the yielder prior to From 4c4047165f626e9dbcae42bdcd00c916a8ec8069 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 11:28:48 -0700 Subject: [PATCH 15/65] fix query cancellation test race condition --- .../io/druid/query/ChainedExecutionQueryRunnerTest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index f2555dd7214..a8c464a8703 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -243,7 +243,9 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); Assert.assertTrue(runner2.hasStarted); - Assert.assertFalse(runner3.hasStarted); + Assert.assertTrue(runner1.interrupted); + Assert.assertTrue(runner2.interrupted); + Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); Assert.assertFalse(runner1.hasCompleted); Assert.assertFalse(runner2.hasCompleted); Assert.assertFalse(runner3.hasCompleted); @@ -256,6 +258,7 @@ public class ChainedExecutionQueryRunnerTest private final CountDownLatch latch; private boolean hasStarted = false; private boolean hasCompleted = false; + private boolean interrupted = false; public DyingQueryRunner(CountDownLatch latch) { @@ -268,6 +271,7 @@ public class ChainedExecutionQueryRunnerTest hasStarted = true; latch.countDown(); if (Thread.interrupted()) { + interrupted = true; throw new QueryInterruptedException("I got killed"); } @@ -276,6 +280,7 @@ public class ChainedExecutionQueryRunnerTest Thread.sleep(500); } catch (InterruptedException e) { + interrupted = true; throw new QueryInterruptedException("I got killed"); } From 5bd4ad3f0f29d6b9e8a955f0b582eaacb6238582 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 11:29:49 -0700 Subject: [PATCH 16/65] 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 17/65] fix bad merge --- .../java/io/druid/query/ChainedExecutionQueryRunnerTest.java | 2 +- server/src/main/java/io/druid/server/QueryResource.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index 90d9c447a96..a8c464a8703 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -108,7 +108,7 @@ public class ChainedExecutionQueryRunnerTest .build() ); - Future resultFuture = Executors.newSingleThreadExecutor().submit( + Future resultFuture = Executors.newFixedThreadPool(1).submit( new Runnable() { @Override diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 4d9188f48f9..4787b17465a 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.common.base.Charsets; import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteStreams; import com.google.inject.Inject; From 294a9ba78f0b86fb8e369ee138d77277a1f33c05 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 11:42:03 -0700 Subject: [PATCH 18/65] fix same race condition in other test --- .../java/io/druid/query/ChainedExecutionQueryRunnerTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java index a8c464a8703..3a87e5c648d 100644 --- a/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/ChainedExecutionQueryRunnerTest.java @@ -139,7 +139,9 @@ public class ChainedExecutionQueryRunnerTest Assert.assertTrue(future.isCancelled()); Assert.assertTrue(runner1.hasStarted); Assert.assertTrue(runner2.hasStarted); - Assert.assertFalse(runner3.hasStarted); + Assert.assertTrue(runner1.interrupted); + Assert.assertTrue(runner2.interrupted); + Assert.assertTrue(!runner3.hasStarted || runner3.interrupted); Assert.assertFalse(runner1.hasCompleted); Assert.assertFalse(runner2.hasCompleted); Assert.assertFalse(runner3.hasCompleted); From 93f97a5d865d4184b2a2fc2ca8e10df183d38eb4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 13:20:14 -0700 Subject: [PATCH 19/65] fix bysegment-query-runner --- .../io/druid/query/BySegmentQueryRunner.java | 37 +++---------------- 1 file changed, 6 insertions(+), 31 deletions(-) diff --git a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java index 44094d0216a..355f7538b9f 100644 --- a/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java +++ b/processing/src/main/java/io/druid/query/BySegmentQueryRunner.java @@ -28,6 +28,7 @@ import com.metamx.common.guava.Yielders; import com.metamx.common.guava.YieldingAccumulator; import org.joda.time.DateTime; +import java.util.Arrays; import java.util.List; /** @@ -55,15 +56,10 @@ public class BySegmentQueryRunner implements QueryRunner { if (query.getContextBySegment(false)) { final Sequence baseSequence = base.run(query); - return new Sequence() - { - @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) - { - List results = Sequences.toList(baseSequence, Lists.newArrayList()); - return accumulator.accumulate( - initValue, + final List results = Sequences.toList(baseSequence, Lists.newArrayList()); + return Sequences.simple( + Arrays.asList( (T) new Result>( timestamp, new BySegmentResultValueClass( @@ -72,29 +68,8 @@ public class BySegmentQueryRunner implements QueryRunner query.getIntervals().get(0) ) ) - ); - } - - @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) - { - List results = Sequences.toList(baseSequence, Lists.newArrayList()); - - final OutType retVal = accumulator.accumulate( - initValue, - (T) new Result>( - timestamp, - new BySegmentResultValueClass( - results, - segmentIdentifier, - query.getIntervals().get(0) - ) - ) - ); - - return Yielders.done(retVal, null); - } - }; + ) + ); } return base.run(query); From 394d0cfbca4207ba45873448c77818573fb6a07d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:45:43 -0700 Subject: [PATCH 20/65] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From f6d319dac2d1772d0f81d7f72b8a61c7f0f3ef66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:46:08 -0700 Subject: [PATCH 21/65] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..96dfbca7ed3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..19badad6ea7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..01acc892db9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..393a1fad858 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..0876ffd3ff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..43822fc8df3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..98f5c2c79f8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..a7c848dbeb6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..4b811976653 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.122 + 0.6.123-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..cbc1e2753cf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..e56a55f4192 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..969a275696d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..ad805bc2df3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..8b418ce6972 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT From 6505f82053965c0bdbfbe0fc3279b6b18f735bc6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:54:23 -0700 Subject: [PATCH 22/65] [maven-release-plugin] rollback the release of druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 5 +++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +++-- services/pom.xml | 2 +- 14 files changed, 18 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 96dfbca7ed3..86ecc759553 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 19badad6ea7..4e828108927 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 01acc892db9..5b9cbc47847 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 393a1fad858..46b6a44ce6d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0876ffd3ff5..6decbae2e15 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 43822fc8df3..9db9badd919 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98f5c2c79f8..5840e57e491 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a7c848dbeb6..25bfd488014 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/pom.xml b/pom.xml index 4b811976653..6717674b368 100644 --- a/pom.xml +++ b/pom.xml @@ -18,12 +18,13 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT druid druid diff --git a/processing/pom.xml b/processing/pom.xml index cbc1e2753cf..f23eb94be46 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e56a55f4192..19a463d66e9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 969a275696d..1f6ba89a2cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ad805bc2df3..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 8b418ce6972..f8ed887cba2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT From 1159e2e41569fd88e121049b0536417218884c88 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:56:12 -0700 Subject: [PATCH 23/65] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From bd18fef47432255396e84be26a3aa7d70eb9f672 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 14:57:31 -0700 Subject: [PATCH 24/65] [maven-release-plugin] rollback the release of druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 ++++--- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +++-- services/pom.xml | 2 +- 14 files changed, 19 insertions(+), 17 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..86ecc759553 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..4e828108927 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..5b9cbc47847 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..46b6a44ce6d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..6decbae2e15 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..9db9badd919 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..5840e57e491 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..25bfd488014 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..6717674b368 100644 --- a/pom.xml +++ b/pom.xml @@ -18,19 +18,20 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122 + 0.6.122-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..f23eb94be46 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..19a463d66e9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..1f6ba89a2cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..f8ed887cba2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.122-SNAPSHOT From 83b2434cad4d017497f0c26140c246480e2e464f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:01:09 -0700 Subject: [PATCH 25/65] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From 695a2946a9f4fd1c13d4d3f83942f0abff8a3228 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:01:13 -0700 Subject: [PATCH 26/65] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..96dfbca7ed3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..19badad6ea7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..01acc892db9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..393a1fad858 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..0876ffd3ff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..43822fc8df3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..98f5c2c79f8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..a7c848dbeb6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..4b811976653 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.122 + 0.6.123-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..cbc1e2753cf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..e56a55f4192 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..969a275696d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..ad805bc2df3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..8b418ce6972 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT From b7b28ad2f90e53d8316b5afa066c80a4da1bf08d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:03:03 -0700 Subject: [PATCH 27/65] [maven-release-plugin] rollback the release of druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 5 +++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 +++-- services/pom.xml | 2 +- 14 files changed, 18 insertions(+), 16 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 96dfbca7ed3..86ecc759553 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index 19badad6ea7..4e828108927 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 01acc892db9..5b9cbc47847 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 393a1fad858..46b6a44ce6d 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0876ffd3ff5..6decbae2e15 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 43822fc8df3..9db9badd919 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98f5c2c79f8..5840e57e491 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a7c848dbeb6..25bfd488014 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/pom.xml b/pom.xml index 4b811976653..6717674b368 100644 --- a/pom.xml +++ b/pom.xml @@ -18,12 +18,13 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT druid druid diff --git a/processing/pom.xml b/processing/pom.xml index cbc1e2753cf..f23eb94be46 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e56a55f4192..19a463d66e9 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 969a275696d..1f6ba89a2cb 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index ad805bc2df3..de7da66f3ac 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -28,7 +29,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index 8b418ce6972..f8ed887cba2 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.122-SNAPSHOT From bcc6e779196bc13b1792d0b24ffc2646be72a452 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 15:05:14 -0700 Subject: [PATCH 28/65] [maven-release-plugin] prepare release druid-0.6.122 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 7 +++---- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 5 ++--- services/pom.xml | 2 +- 14 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 86ecc759553..6780a7cea50 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/common/pom.xml b/common/pom.xml index 4e828108927..f7a757234e1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/examples/pom.xml b/examples/pom.xml index 5b9cbc47847..9573b243f4a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 46b6a44ce6d..c5ddc9e1638 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 6decbae2e15..624b6b60fc4 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 9db9badd919..f7cd2e0902d 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 5840e57e491..0616affa696 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 25bfd488014..f7816e09134 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/pom.xml b/pom.xml index 6717674b368..70736b7a8f7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,20 +18,19 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid pom - 0.6.122-SNAPSHOT + 0.6.122 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.122 diff --git a/processing/pom.xml b/processing/pom.xml index f23eb94be46..aa87d013530 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 19a463d66e9..636afd81cd2 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 1f6ba89a2cb..c21a323701e 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/server/pom.xml b/server/pom.xml index de7da66f3ac..2ec2a597f57 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,8 +18,7 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 io.druid druid-server @@ -29,7 +28,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 diff --git a/services/pom.xml b/services/pom.xml index f8ed887cba2..f3fe12dbe5b 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122-SNAPSHOT + 0.6.122 From 0fd57dbc75e74a0b5887770eff730ef43f3aa6f2 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 13 Jun 2014 15:05:18 -0700 Subject: [PATCH 29/65] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 6780a7cea50..96dfbca7ed3 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index f7a757234e1..19badad6ea7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 9573b243f4a..01acc892db9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index c5ddc9e1638..393a1fad858 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 624b6b60fc4..0876ffd3ff5 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index f7cd2e0902d..43822fc8df3 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 0616affa696..98f5c2c79f8 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index f7816e09134..a7c848dbeb6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/pom.xml b/pom.xml index 70736b7a8f7..4b811976653 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.122 + 0.6.123-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.122 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index aa87d013530..cbc1e2753cf 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 636afd81cd2..e56a55f4192 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index c21a323701e..969a275696d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 2ec2a597f57..ad805bc2df3 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index f3fe12dbe5b..8b418ce6972 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.122 + 0.6.123-SNAPSHOT From 903f1a695ff456533eb7fcede833914c0036c2c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:16:36 -0700 Subject: [PATCH 30/65] [maven-release-plugin] prepare release druid-0.6.123 --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 96dfbca7ed3..08989c9ff5f 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/common/pom.xml b/common/pom.xml index 19badad6ea7..c26f423f5ba 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/examples/pom.xml b/examples/pom.xml index 01acc892db9..0f1cbe02d83 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 393a1fad858..0004336cdf8 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 0876ffd3ff5..19ac02b1834 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 43822fc8df3..617c9222b10 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 98f5c2c79f8..1aa7a84bc33 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index a7c848dbeb6..5a139be19c6 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/pom.xml b/pom.xml index 4b811976653..632077b5edf 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.123-SNAPSHOT + 0.6.123 druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.107-SNAPSHOT + druid-0.6.123 diff --git a/processing/pom.xml b/processing/pom.xml index cbc1e2753cf..db7a06c4e68 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index e56a55f4192..3702c7d0d6a 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index 969a275696d..a8f4d7c358d 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/server/pom.xml b/server/pom.xml index ad805bc2df3..bb81e8aba72 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 diff --git a/services/pom.xml b/services/pom.xml index 8b418ce6972..bd186d66c98 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123-SNAPSHOT + 0.6.123 From a5c533dcab9c156e498b587f845714f33232c2f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Xavier=20L=C3=A9aut=C3=A9?= Date: Fri, 13 Jun 2014 15:17:00 -0700 Subject: [PATCH 31/65] [maven-release-plugin] prepare for next development iteration --- cassandra-storage/pom.xml | 2 +- common/pom.xml | 2 +- examples/pom.xml | 2 +- hdfs-storage/pom.xml | 2 +- indexing-hadoop/pom.xml | 2 +- indexing-service/pom.xml | 2 +- kafka-eight/pom.xml | 2 +- kafka-seven/pom.xml | 2 +- pom.xml | 4 ++-- processing/pom.xml | 2 +- rabbitmq/pom.xml | 2 +- s3-extensions/pom.xml | 2 +- server/pom.xml | 2 +- services/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/cassandra-storage/pom.xml b/cassandra-storage/pom.xml index 08989c9ff5f..7729770ef86 100644 --- a/cassandra-storage/pom.xml +++ b/cassandra-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/common/pom.xml b/common/pom.xml index c26f423f5ba..4a515fc0675 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/examples/pom.xml b/examples/pom.xml index 0f1cbe02d83..177c43086cb 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/hdfs-storage/pom.xml b/hdfs-storage/pom.xml index 0004336cdf8..d0aa54ac33f 100644 --- a/hdfs-storage/pom.xml +++ b/hdfs-storage/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 19ac02b1834..063a60bc0f3 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 617c9222b10..2aa51f4f21c 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/kafka-eight/pom.xml b/kafka-eight/pom.xml index 1aa7a84bc33..74cf5d1bfc6 100644 --- a/kafka-eight/pom.xml +++ b/kafka-eight/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/kafka-seven/pom.xml b/kafka-seven/pom.xml index 5a139be19c6..b30908f6d83 100644 --- a/kafka-seven/pom.xml +++ b/kafka-seven/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/pom.xml b/pom.xml index 632077b5edf..1aa12d28497 100644 --- a/pom.xml +++ b/pom.xml @@ -23,14 +23,14 @@ io.druid druid pom - 0.6.123 + 0.6.124-SNAPSHOT druid druid scm:git:ssh://git@github.com/metamx/druid.git scm:git:ssh://git@github.com/metamx/druid.git http://www.github.com/metamx/druid - druid-0.6.123 + druid-0.6.107-SNAPSHOT diff --git a/processing/pom.xml b/processing/pom.xml index db7a06c4e68..3cfcf495797 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/rabbitmq/pom.xml b/rabbitmq/pom.xml index 3702c7d0d6a..507e95d266f 100644 --- a/rabbitmq/pom.xml +++ b/rabbitmq/pom.xml @@ -9,7 +9,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/s3-extensions/pom.xml b/s3-extensions/pom.xml index a8f4d7c358d..2e7da9309d8 100644 --- a/s3-extensions/pom.xml +++ b/s3-extensions/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index bb81e8aba72..c6f55aa8a7a 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT diff --git a/services/pom.xml b/services/pom.xml index bd186d66c98..9db334a63ce 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -27,7 +27,7 @@ io.druid druid - 0.6.123 + 0.6.124-SNAPSHOT From d4a47fe6e8f86d3bceb825935960ffaa38c92de5 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 16 Jun 2014 13:45:32 -0700 Subject: [PATCH 32/65] 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 33/65] 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 34/65] 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 35/65] 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 36/65] 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 37/65] 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 38/65] 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 39/65] 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 2a88f8c4ce44ed2fc32a992596df218a2d979c80 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Jun 2014 16:36:05 -0700 Subject: [PATCH 40/65] 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 41/65] [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 42/65] [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 43/65] 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 44/65] [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 45/65] [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 46/65] 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 47/65] 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 48/65] 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 49/65] 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 50/65] 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 51/65] 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 52/65] 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 53/65] 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 54/65] [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 55/65] [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 56/65] 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 57/65] 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 58/65] [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 59/65] [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 60/65] 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 61/65] 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 62/65] 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 63/65] [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 64/65] [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 65/65] 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