From 885b512a1c8c309cdfb29069fbcd3256a1304c07 Mon Sep 17 00:00:00 2001 From: fjy Date: Mon, 11 Jan 2016 19:05:51 -0800 Subject: [PATCH 1/2] Double and Int column support --- .../theta/EmptySketchAggregator.java | 12 + .../theta/EmptySketchBufferAggregator.java | 12 + .../datasketches/theta/SketchAggregator.java | 12 + .../theta/SketchAggregatorFactory.java | 6 + .../theta/SketchBufferAggregator.java | 12 + .../ApproximateHistogramAggregator.java | 12 + ...ApproximateHistogramAggregatorFactory.java | 6 + .../ApproximateHistogramBufferAggregator.java | 12 + ...ApproximateHistogramFoldingAggregator.java | 12 + ...imateHistogramFoldingBufferAggregator.java | 12 + .../io/druid/indexer/IndexGeneratorJob.java | 12 + .../java/io/druid/indexer/InputRowSerde.java | 8 + .../druid/indexer/hadoop/SegmentInputRow.java | 12 + pom.xml | 2 +- .../io/druid/jackson/AggregatorsModule.java | 6 +- .../java/io/druid/query/DruidMetrics.java | 2 +- .../druid/query/aggregation/Aggregator.java | 5 +- .../query/aggregation/AggregatorFactory.java | 6 + .../druid/query/aggregation/Aggregators.java | 25 +- .../query/aggregation/BufferAggregator.java | 77 ++- .../query/aggregation/CountAggregator.java | 12 + .../aggregation/CountAggregatorFactory.java | 6 + .../aggregation/CountBufferAggregator.java | 13 +- .../aggregation/DoubleMaxAggregator.java | 18 +- .../DoubleMaxAggregatorFactory.java | 12 +- .../DoubleMaxBufferAggregator.java | 18 +- .../aggregation/DoubleMinAggregator.java | 19 +- .../DoubleMinAggregatorFactory.java | 12 +- .../DoubleMinBufferAggregator.java | 19 +- .../aggregation/DoubleSumAggregator.java | 17 +- .../DoubleSumAggregatorFactory.java | 12 +- .../DoubleSumBufferAggregator.java | 20 +- .../query/aggregation/FilteredAggregator.java | 12 + .../FilteredAggregatorFactory.java | 6 + .../aggregation/FilteredBufferAggregator.java | 12 + .../query/aggregation/FloatSumAggregator.java | 118 +++++ .../FloatSumAggregatorFactory.java | 200 +++++++ .../aggregation/FloatSumBufferAggregator.java | 86 ++++ .../aggregation/HistogramAggregator.java | 12 + .../HistogramAggregatorFactory.java | 6 + .../HistogramBufferAggregator.java | 12 + .../query/aggregation/IntSumAggregator.java | 118 +++++ .../aggregation/IntSumAggregatorFactory.java | 200 +++++++ .../aggregation/IntSumBufferAggregator.java | 86 ++++ .../aggregation/JavaScriptAggregator.java | 12 + .../JavaScriptAggregatorFactory.java | 6 + .../JavaScriptBufferAggregator.java | 18 +- .../query/aggregation/LongMaxAggregator.java | 12 + .../aggregation/LongMaxAggregatorFactory.java | 6 + .../aggregation/LongMaxBufferAggregator.java | 12 + .../query/aggregation/LongMinAggregator.java | 12 + .../aggregation/LongMinAggregatorFactory.java | 6 + .../aggregation/LongMinBufferAggregator.java | 12 + .../query/aggregation/LongSumAggregator.java | 12 + .../aggregation/LongSumAggregatorFactory.java | 6 + .../aggregation/LongSumBufferAggregator.java | 13 +- .../cardinality/CardinalityAggregator.java | 12 + .../CardinalityAggregatorFactory.java | 6 + .../CardinalityBufferAggregator.java | 13 +- .../hyperloglog/HyperUniquesAggregator.java | 12 + .../HyperUniquesAggregatorFactory.java | 26 +- .../HyperUniquesBufferAggregator.java | 13 +- .../druid/query/metadata/SegmentAnalyzer.java | 32 ++ .../druid/segment/ColumnSelectorFactory.java | 2 + .../druid/segment/DoubleColumnSelector.java | 27 + .../segment/DoubleMetricColumnSerializer.java | 81 +++ .../main/java/io/druid/segment/IndexIO.java | 10 + .../java/io/druid/segment/IndexMerger.java | 8 +- .../io/druid/segment/IntColumnSelector.java | 27 + .../segment/IntMetricColumnSerializer.java | 81 +++ .../java/io/druid/segment/MetricHolder.java | 83 ++- .../QueryableIndexIndexableAdapter.java | 16 +- .../segment/QueryableIndexStorageAdapter.java | 110 +++- .../segment/column/ColumnCapabilities.java | 1 + .../column/ColumnCapabilitiesImpl.java | 6 + .../druid/segment/column/GenericColumn.java | 17 + .../column/IndexedDoublesGenericColumn.java | 126 +++++ .../column/IndexedFloatsGenericColumn.java | 31 +- .../column/IndexedIntsGenericColumn.java | 126 +++++ .../column/IndexedLongsGenericColumn.java | 31 +- .../io/druid/segment/column/ValueType.java | 2 + .../CompressedDoubleBufferObjectStrategy.java | 72 +++ .../CompressedDoublesIndexedSupplier.java | 317 ++++++++++++ .../CompressedDoublesSupplierSerializer.java | 120 +++++ .../CompressedIntsSupplierSerializer.java | 113 ++++ .../io/druid/segment/data/IndexedDoubles.java | 34 ++ .../segment/incremental/IncrementalIndex.java | 49 +- .../IncrementalIndexStorageAdapter.java | 54 ++ .../incremental/OffheapIncrementalIndex.java | 486 ------------------ .../incremental/OnheapIncrementalIndex.java | 12 + .../SpatialDimensionRowTransformer.java | 28 +- .../druid/segment/serde/ColumnPartSerde.java | 2 + .../serde/DoubleGenericColumnPartSerde.java | 84 +++ .../serde/DoubleGenericColumnSupplier.java | 49 ++ .../serde/IntGenericColumnPartSerde.java | 84 +++ .../serde/IntGenericColumnSupplier.java | 44 ++ .../io/druid/query/QueryRunnerTestHelper.java | 23 +- .../aggregation/DoubleMaxAggregationTest.java | 16 +- .../aggregation/DoubleMinAggregationTest.java | 16 +- .../aggregation/DoubleSumAggregatorTest.java | 12 +- .../aggregation/FilteredAggregatorTest.java | 76 ++- .../JavaScriptAggregatorBenchmark.java | 18 +- .../aggregation/JavaScriptAggregatorTest.java | 26 +- .../aggregation/MetricSelectorUtils.java | 19 + .../aggregation/TestDoubleColumnSelector.java | 47 ++ .../query/groupby/GroupByQueryRunnerTest.java | 78 +-- .../test/java/io/druid/segment/TestIndex.java | 3 +- .../incremental/IncrementalIndexTest.java | 19 - .../firehose/WikipediaIrcDecoder.java | 49 +- .../CombiningFirehoseFactoryTest.java | 12 + .../segment/realtime/RealtimeManagerTest.java | 12 + .../plumber/RealtimePlumberSchoolTest.java | 12 + .../segment/realtime/plumber/SinkTest.java | 24 + .../shard/HashBasedNumberedShardSpecTest.java | 12 + 114 files changed, 3618 insertions(+), 739 deletions(-) create mode 100644 processing/src/main/java/io/druid/query/aggregation/FloatSumAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/FloatSumAggregatorFactory.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/FloatSumBufferAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/IntSumAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/IntSumAggregatorFactory.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/IntSumBufferAggregator.java create mode 100644 processing/src/main/java/io/druid/segment/DoubleColumnSelector.java create mode 100644 processing/src/main/java/io/druid/segment/DoubleMetricColumnSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/IntColumnSelector.java create mode 100644 processing/src/main/java/io/druid/segment/IntMetricColumnSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/column/IndexedDoublesGenericColumn.java create mode 100644 processing/src/main/java/io/druid/segment/column/IndexedIntsGenericColumn.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedDoublesSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedIntsSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/IndexedDoubles.java delete mode 100644 processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java create mode 100644 processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java create mode 100644 processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java create mode 100644 processing/src/main/java/io/druid/segment/serde/IntGenericColumnPartSerde.java create mode 100644 processing/src/main/java/io/druid/segment/serde/IntGenericColumnSupplier.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelector.java diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java index 02da6e064d5e..038d7c541d14 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java @@ -58,6 +58,18 @@ public long getLong() throw new UnsupportedOperationException("Not implemented"); } + @Override + public int getInt() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public String getName() { diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java index 5073c1b79b00..d492115367a4 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java @@ -57,6 +57,18 @@ public long getLong(ByteBuffer buf, int position) throw new UnsupportedOperationException("Not implemented"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public void close() { diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java index 8e1e7643f321..85afb1d30bfc 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -89,6 +89,18 @@ public long getLong() throw new UnsupportedOperationException("Not implemented"); } + @Override + public int getInt() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public String getName() { diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java index 9b5597a90ea8..6263d0450aae 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -159,6 +159,12 @@ public Object getAggregatorStartValue() return Sketches.updateSketchBuilder().build(size); } + @Override + public boolean isComplex() + { + return true; + } + @Override public List requiredFields() { diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java index 2958b1846371..ff1b626df1b5 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java @@ -110,6 +110,18 @@ public long getLong(ByteBuffer buf, int position) throw new UnsupportedOperationException("Not implemented"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public void close() { diff --git a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java index 88be52f31eaf..66e8153095a7 100644 --- a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java +++ b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java @@ -95,6 +95,18 @@ public long getLong() throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getLong()"); } + @Override + public double getDouble() + { + throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getDouble()"); + } + + @Override + public int getInt() + { + throw new UnsupportedOperationException("ApproximateHistogramAggregator does not support getInt()"); + } + @Override public String getName() { diff --git a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index 890ba457f1d0..8afec492d8d9 100644 --- a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -238,6 +238,12 @@ public Object getAggregatorStartValue() return new ApproximateHistogram(resolution); } + @Override + public boolean isComplex() + { + return true; + } + @Override public boolean equals(Object o) { diff --git a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java index 297fde021d09..d72acdbf21bf 100644 --- a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java +++ b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java @@ -94,6 +94,18 @@ public long getLong(ByteBuffer buf, int position) throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getLong()"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getInt()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getDouble()"); + } + @Override public void close() { diff --git a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java index ee04f45b8adf..84ebc1648c9f 100644 --- a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java +++ b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java @@ -93,6 +93,18 @@ public long getLong() throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getLong()"); } + @Override + public int getInt() + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getInt()"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getDouble()"); + } + @Override public String getName() { diff --git a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java index bd52b603badb..ab19294b7139 100644 --- a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java +++ b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java @@ -97,6 +97,18 @@ public long getLong(ByteBuffer buf, int position) throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getLong()"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getInt()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getDouble()"); + } + @Override public void close() { 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 980f981396ef..0854e2769082 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -396,6 +396,18 @@ public long getLongMetric(String metric) return row.getLongMetric(metric); } + @Override + public int getIntMetric(String metric) + { + return row.getIntMetric(metric); + } + + @Override + public double getDoubleMetric(String metric) + { + return row.getDoubleMetric(metric); + } + @Override public int compareTo(Row o) { diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java index b179faefb730..5b21942a7010 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java @@ -99,6 +99,10 @@ public InputRow get() out.writeFloat(agg.getFloat()); } else if (t.equals("long")) { WritableUtils.writeVLong(out, agg.getLong()); + } else if (t.equals("int")) { + out.writeInt(agg.getInt()); + } else if (t.equals("double")) { + out.writeDouble(agg.getDouble()); } else { //its a complex metric Object val = agg.get(); @@ -199,6 +203,10 @@ public static final InputRow fromBytes(byte[] data, AggregatorFactory[] aggs) event.put(metric, in.readFloat()); } else if (type.equals("long")) { event.put(metric, WritableUtils.readVLong(in)); + } else if (type.equals("int")) { + event.put(metric, in.readInt()); + } else if (type.equals("double")) { + event.put(metric, in.readDouble()); } else { ComplexMetricSerde serde = getComplexMetricSerde(type); byte[] value = readBytes(in); diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java index f3a1adf7dc45..759b01947924 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java @@ -81,6 +81,18 @@ public long getLongMetric(String metric) return delegate.getLongMetric(metric); } + @Override + public double getDoubleMetric(String metric) + { + return delegate.getDoubleMetric(metric); + } + + @Override + public int getIntMetric(String metric) + { + return delegate.getIntMetric(metric); + } + @Override public int compareTo(Row row) { diff --git a/pom.xml b/pom.xml index 005628a60e62..97ab41e902c7 100644 --- a/pom.xml +++ b/pom.xml @@ -69,7 +69,7 @@ 2.9.1 9.2.5.v20141112 1.19 - 0.3.15 + 0.3.16-SNAPSHOT 2.4.6 2.4.1 diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 2e9582da89c6..ca3e45606c1f 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -29,7 +29,9 @@ import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; +import io.druid.query.aggregation.FloatSumAggregator; import io.druid.query.aggregation.HistogramAggregatorFactory; +import io.druid.query.aggregation.IntSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongMaxAggregatorFactory; import io.druid.query.aggregation.LongMinAggregatorFactory; @@ -74,7 +76,9 @@ public AggregatorsModule() @JsonSubTypes.Type(name = "histogram", value = HistogramAggregatorFactory.class), @JsonSubTypes.Type(name = "hyperUnique", value = HyperUniquesAggregatorFactory.class), @JsonSubTypes.Type(name = "cardinality", value = CardinalityAggregatorFactory.class), - @JsonSubTypes.Type(name = "filtered", value = FilteredAggregatorFactory.class) + @JsonSubTypes.Type(name = "filtered", value = FilteredAggregatorFactory.class), + @JsonSubTypes.Type(name = "intSum", value = IntSumAggregatorFactory.class), + @JsonSubTypes.Type(name = "floatSum", value = FloatSumAggregator.class) }) public static interface AggregatorFactoryMixin { diff --git a/processing/src/main/java/io/druid/query/DruidMetrics.java b/processing/src/main/java/io/druid/query/DruidMetrics.java index b9915da54455..708e27e814ba 100644 --- a/processing/src/main/java/io/druid/query/DruidMetrics.java +++ b/processing/src/main/java/io/druid/query/DruidMetrics.java @@ -54,7 +54,7 @@ public static int findNumComplexAggs(List aggs) int retVal = 0; for (AggregatorFactory agg : aggs) { // This needs to change when we have support column types better - if (!agg.getTypeName().equals("float") && !agg.getTypeName().equals("long")) { + if (agg.isComplex()) { retVal++; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java index 0cc56b458c55..9cbc6c588176 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java @@ -37,8 +37,9 @@ public interface Aggregator { void reset(); Object get(); float getFloat(); + long getLong(); + int getInt(); + double getDouble(); String getName(); void close(); - - long getLong(); } 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 81ea7dd185eb..65440d5a7c01 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -112,4 +112,10 @@ public interface AggregatorFactory * @return the starting value for a corresponding aggregator. */ public Object getAggregatorStartValue(); + + /** + * Returns if the aggregator is doing complex or simple (primitive) aggregations + * @return if the aggregator is complex + */ + public boolean isComplex(); } diff --git a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java index 717e02053627..6b9f7dfd7dd7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java @@ -53,6 +53,18 @@ public float getFloat() return 0; } + @Override + public double getDouble() + { + return 0; + } + + @Override + public int getInt() + { + return 0; + } + @Override public String getName() { @@ -101,13 +113,24 @@ public float getFloat(ByteBuffer buf, int position) return 0; } - @Override public long getLong(ByteBuffer buf, int position) { return 0L; } + @Override + public int getInt(ByteBuffer buf, int position) + { + return 0; + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return 0d; + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java index bbdf9100bbab..5f1f97342510 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -25,7 +25,7 @@ * A BufferAggregator is an object that can aggregate metrics into a ByteBuffer. Its aggregation-related methods * (namely, aggregate(...) and get(...)) only take the ByteBuffer and position because it is assumed that the Aggregator * was given something (one or more MetricSelector(s)) in its constructor that it can use to get at the next bit of data. - * + *

* Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls * to aggregate(...). */ @@ -33,79 +33,118 @@ public interface BufferAggregator { /** * Initializes the buffer location - * + *

* Implementations of this method must initialize the byte buffer at the given position - * + *

* Implementations must not change the position, limit or mark of the given buffer - * + *

* This method must not exceed the number of bytes returned by {@link AggregatorFactory#getMaxIntermediateSize()} * in the corresponding {@link AggregatorFactory} * - * @param buf byte buffer to initialize + * @param buf byte buffer to initialize * @param position offset within the byte buffer for initialization */ void init(ByteBuffer buf, int position); /** * Aggregates metric values into the given aggregate byte representation - * + *

* Implementations of this method must read in the aggregate value from the buffer at the given position, * aggregate the next element of data and write the updated aggregate value back into the buffer. - * + *

* Implementations must not change the position, limit or mark of the given buffer * - * @param buf byte buffer storing the byte array representation of the aggregate + * @param buf byte buffer storing the byte array representation of the aggregate * @param position offset within the byte buffer at which the current aggregate value is stored */ void aggregate(ByteBuffer buf, int position); /** * Returns the intermediate object representation of the given aggregate. - * + *

* Converts the given byte buffer representation into an intermediate aggregate Object - * + *

* Implementations must not change the position, limit or mark of the given buffer * - * @param buf byte buffer storing the byte array representation of the aggregate + * @param buf byte buffer storing the byte array representation of the aggregate * @param position offset within the byte buffer at which the aggregate value is stored + * * @return the Object representation of the aggregate */ Object get(ByteBuffer buf, int position); /** * Returns the float representation of the given aggregate byte array - * + *

* Converts the given byte buffer representation into the intermediate aggregate value. - * + *

* Implementations must not change the position, limit or mark of the given buffer - * + *

* Implementations are only required to support this method if they are aggregations which * have an {@link AggregatorFactory#getTypeName()} of "float". * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. * - * @param buf byte buffer storing the byte array representation of the aggregate + * @param buf byte buffer storing the byte array representation of the aggregate * @param position offset within the byte buffer at which the aggregate value is stored + * * @return the float representation of the aggregate */ float getFloat(ByteBuffer buf, int position); /** * Returns the long representation of the given aggregate byte array - * + *

* Converts the given byte buffer representation into the intermediate aggregate value. - * + *

* Implementations must not change the position, limit or mark of the given buffer - * + *

* Implementations are only required to support this method if they are aggregations which * have an {@link AggregatorFactory#getTypeName()} of "long". * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. * - * @param buf byte buffer storing the byte array representation of the aggregate + * @param buf byte buffer storing the byte array representation of the aggregate * @param position offset within the byte buffer at which the aggregate value is stored + * * @return the long representation of the aggregate */ long getLong(ByteBuffer buf, int position); + /** + * Returns the int representation of the given aggregate byte array + *

+ * Converts the given byte buffer representation into the intermediate aggregate value. + *

+ * Implementations must not change the position, limit or mark of the given buffer + *

+ * Implementations are only required to support this method if they are aggregations which + * have an {@link AggregatorFactory#getTypeName()} of "long". + * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the aggregate value is stored + * + * @return the int representation of the aggregate + */ + int getInt(ByteBuffer buf, int position); + + /** + * Returns the double representation of the given aggregate byte array + *

+ * Converts the given byte buffer representation into the intermediate aggregate value. + *

+ * Implementations must not change the position, limit or mark of the given buffer + *

+ * Implementations are only required to support this method if they are aggregations which + * have an {@link AggregatorFactory#getTypeName()} of "long". + * If unimplemented, throwing an {@link UnsupportedOperationException} is common and recommended. + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the aggregate value is stored + * + * @return the double representation of the aggregate + */ + double getDouble(ByteBuffer buf, int position); + /** * Release any resources used by the aggregator */ diff --git a/processing/src/main/java/io/druid/query/aggregation/CountAggregator.java b/processing/src/main/java/io/druid/query/aggregation/CountAggregator.java index 4c283760bb5f..42ca15783b4b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregator.java @@ -70,6 +70,18 @@ public long getLong() return count; } + @Override + public int getInt() + { + return (int) count; + } + + @Override + public double getDouble() + { + return (double) count; + } + @Override public String getName() { 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 4bf2018f3b40..04f6c0f6626d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountAggregatorFactory.java @@ -132,6 +132,12 @@ public Object getAggregatorStartValue() return 0; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java index ec5804e4100a..7f7f9c01ab3c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java @@ -50,13 +50,24 @@ public float getFloat(ByteBuffer buf, int position) return buf.getLong(position); } - @Override public long getLong(ByteBuffer buf, int position) { return buf.getLong(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return buf.getInt(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java index ce1156115edd..4c241ff11323 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java @@ -19,7 +19,7 @@ package io.druid.query.aggregation; -import io.druid.segment.FloatColumnSelector; +import io.druid.segment.DoubleColumnSelector; import java.util.Comparator; @@ -34,12 +34,12 @@ static double combineValues(Object lhs, Object rhs) return Math.max(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue()); } - private final FloatColumnSelector selector; + private final DoubleColumnSelector selector; private final String name; private double max; - public DoubleMaxAggregator(String name, FloatColumnSelector selector) + public DoubleMaxAggregator(String name, DoubleColumnSelector selector) { this.name = name; this.selector = selector; @@ -77,6 +77,18 @@ public long getLong() return (long) max; } + @Override + public int getInt() + { + return (int) max; + } + + @Override + public double getDouble() + { + return max; + } + @Override public String getName() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java index 50dd2a49fb44..6113c7134e99 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java @@ -56,13 +56,13 @@ public DoubleMaxAggregatorFactory( @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - return new DoubleMaxAggregator(name, metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleMaxAggregator(name, metricFactory.makeDoubleColumnSelector(fieldName)); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new DoubleMaxBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleMaxBufferAggregator(metricFactory.makeDoubleColumnSelector(fieldName)); } @Override @@ -135,7 +135,7 @@ public byte[] getCacheKey() @Override public String getTypeName() { - return "float"; + return "double"; } @Override @@ -150,6 +150,12 @@ public Object getAggregatorStartValue() return Double.NEGATIVE_INFINITY; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java index 3a79da6eefb7..bd83c79ff038 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java @@ -19,7 +19,7 @@ package io.druid.query.aggregation; -import io.druid.segment.FloatColumnSelector; +import io.druid.segment.DoubleColumnSelector; import java.nio.ByteBuffer; @@ -27,9 +27,9 @@ */ public class DoubleMaxBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; + private final DoubleColumnSelector selector; - public DoubleMaxBufferAggregator(FloatColumnSelector selector) + public DoubleMaxBufferAggregator(DoubleColumnSelector selector) { this.selector = selector; } @@ -64,6 +64,18 @@ public long getLong(ByteBuffer buf, int position) return (long) buf.getDouble(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return (int) buf.getDouble(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java index bca2b3770a79..b1258eb7b052 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java @@ -19,8 +19,9 @@ package io.druid.query.aggregation; -import io.druid.segment.FloatColumnSelector; +import io.druid.segment.DoubleColumnSelector; +import java.nio.ByteBuffer; import java.util.Comparator; /** @@ -34,12 +35,12 @@ static double combineValues(Object lhs, Object rhs) return Math.min(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue()); } - private final FloatColumnSelector selector; + private final DoubleColumnSelector selector; private final String name; private double min; - public DoubleMinAggregator(String name, FloatColumnSelector selector) + public DoubleMinAggregator(String name, DoubleColumnSelector selector) { this.name = name; this.selector = selector; @@ -77,6 +78,18 @@ public long getLong() return (long) min; } + @Override + public int getInt() + { + return (int) min; + } + + @Override + public double getDouble() + { + return (double) min; + } + @Override public String getName() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java index af98615a9a93..d42eea205400 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java @@ -56,13 +56,13 @@ public DoubleMinAggregatorFactory( @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - return new DoubleMinAggregator(name, metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleMinAggregator(name, metricFactory.makeDoubleColumnSelector(fieldName)); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new DoubleMinBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleMinBufferAggregator(metricFactory.makeDoubleColumnSelector(fieldName)); } @Override @@ -135,7 +135,7 @@ public byte[] getCacheKey() @Override public String getTypeName() { - return "float"; + return "double"; } @Override @@ -150,6 +150,12 @@ public Object getAggregatorStartValue() return Double.POSITIVE_INFINITY; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java index f4c3f475002e..bb6be285129d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java @@ -19,7 +19,7 @@ package io.druid.query.aggregation; -import io.druid.segment.FloatColumnSelector; +import io.druid.segment.DoubleColumnSelector; import java.nio.ByteBuffer; @@ -27,9 +27,9 @@ */ public class DoubleMinBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; + private final DoubleColumnSelector selector; - public DoubleMinBufferAggregator(FloatColumnSelector selector) + public DoubleMinBufferAggregator(DoubleColumnSelector selector) { this.selector = selector; } @@ -58,13 +58,24 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getDouble(position); } - @Override public long getLong(ByteBuffer buf, int position) { return (long) buf.getDouble(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return (int) buf.getDouble(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregator.java index 9a8e73745f89..466f2d4c19ff 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregator.java @@ -21,6 +21,7 @@ import com.google.common.collect.Ordering; import com.google.common.primitives.Doubles; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.util.Comparator; @@ -43,12 +44,12 @@ static double combineValues(Object lhs, Object rhs) return ((Number) lhs).doubleValue() + ((Number) rhs).doubleValue(); } - private final FloatColumnSelector selector; + private final DoubleColumnSelector selector; private final String name; private double sum; - public DoubleSumAggregator(String name, FloatColumnSelector selector) + public DoubleSumAggregator(String name, DoubleColumnSelector selector) { this.name = name; this.selector = selector; @@ -86,6 +87,18 @@ public long getLong() return (long) sum; } + @Override + public int getInt() + { + return (int) sum; + } + + @Override + public double getDouble() + { + return sum; + } + @Override public String getName() { 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 0cf3e5a5ef07..c4d1de22f600 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -58,14 +58,14 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) { return new DoubleSumAggregator( name, - metricFactory.makeFloatColumnSelector(fieldName) + metricFactory.makeDoubleColumnSelector(fieldName) ); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new DoubleSumBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleSumBufferAggregator(metricFactory.makeDoubleColumnSelector(fieldName)); } @Override @@ -138,7 +138,7 @@ public byte[] getCacheKey() @Override public String getTypeName() { - return "float"; + return "double"; } @Override @@ -153,6 +153,12 @@ public Object getAggregatorStartValue() return 0; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java index e67a64d7c686..d033be30d093 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; @@ -27,10 +28,10 @@ */ public class DoubleSumBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; + private final DoubleColumnSelector selector; public DoubleSumBufferAggregator( - FloatColumnSelector selector + DoubleColumnSelector selector ) { this.selector = selector; @@ -45,7 +46,7 @@ public void init(ByteBuffer buf, int position) @Override public void aggregate(ByteBuffer buf, int position) { - buf.putDouble(position, buf.getDouble(position) + (double) selector.get()); + buf.putDouble(position, buf.getDouble(position) + selector.get()); } @Override @@ -60,13 +61,24 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getDouble(position); } - @Override public long getLong(ByteBuffer buf, int position) { return (long) buf.getDouble(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return (int) buf.getDouble(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java index 8a3f56c271b8..007b0f7f4374 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregator.java @@ -64,6 +64,18 @@ public long getLong() return delegate.getLong(); } + @Override + public int getInt() + { + return delegate.getInt(); + } + + @Override + public double getDouble() + { + return delegate.getDouble(); + } + @Override public String getName() { diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java index 9fa42c9c31c8..117aab9d84c7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -143,6 +143,12 @@ public Object getAggregatorStartValue() return delegate.getAggregatorStartValue(); } + @Override + public boolean isComplex() + { + return false; + } + @JsonProperty public AggregatorFactory getAggregator() { diff --git a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java index bd53197804e3..0f4b4707db18 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredBufferAggregator.java @@ -66,6 +66,18 @@ public float getFloat(ByteBuffer buf, int position) return delegate.getFloat(buf, position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return delegate.getInt(buf, position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return delegate.getDouble(buf, position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/FloatSumAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FloatSumAggregator.java new file mode 100644 index 000000000000..98a654856b82 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FloatSumAggregator.java @@ -0,0 +1,118 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import com.google.common.collect.Ordering; +import com.google.common.primitives.Floats; +import io.druid.segment.FloatColumnSelector; + +import java.util.Comparator; + +/** + */ +public class FloatSumAggregator implements Aggregator +{ + static final Comparator COMPARATOR = new Ordering() + { + @Override + public int compare(Object o, Object o1) + { + return Floats.compare(((Number) o).floatValue(), ((Number) o1).floatValue()); + } + }.nullsFirst(); + + static double combineValues(Object lhs, Object rhs) + { + return ((Number) lhs).floatValue() + ((Number) rhs).floatValue(); + } + + private final FloatColumnSelector selector; + private final String name; + + private float sum; + + public FloatSumAggregator(String name, FloatColumnSelector selector) + { + this.name = name; + this.selector = selector; + + this.sum = 0; + } + + @Override + public void aggregate() + { + sum += selector.get(); + } + + @Override + public void reset() + { + sum = 0; + } + + @Override + public Object get() + { + return sum; + } + + @Override + public float getFloat() + { + return sum; + } + + @Override + public long getLong() + { + return (long) sum; + } + + @Override + public int getInt() + { + return (int) sum; + } + + @Override + public double getDouble() + { + return (double) sum; + } + + @Override + public String getName() + { + return this.name; + } + + @Override + public Aggregator clone() + { + return new FloatSumAggregator(name, selector); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/FloatSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/FloatSumAggregatorFactory.java new file mode 100644 index 000000000000..3e54348ac5e2 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FloatSumAggregatorFactory.java @@ -0,0 +1,200 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Floats; +import com.metamx.common.StringUtils; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class FloatSumAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0xB; + + private final String fieldName; + private final String name; + + @JsonCreator + public FloatSumAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new FloatSumAggregator( + name, + metricFactory.makeFloatColumnSelector(fieldName) + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new FloatSumBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + } + + @Override + public Comparator getComparator() + { + return FloatSumAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return FloatSumAggregator.combineValues(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new FloatSumAggregatorFactory(name, name); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new FloatSumAggregatorFactory(fieldName, fieldName)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Float.parseFloat((String) object); + } + return object; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "float"; + } + + @Override + public int getMaxIntermediateSize() + { + return Floats.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return 0; + } + + @Override + public boolean isComplex() + { + return false; + } + + @Override + public String toString() + { + return "FloatSumAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + FloatSumAggregatorFactory that = (FloatSumAggregatorFactory) 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; + } + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/FloatSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/FloatSumBufferAggregator.java new file mode 100644 index 000000000000..7cdee8aa76ab --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/FloatSumBufferAggregator.java @@ -0,0 +1,86 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import io.druid.segment.FloatColumnSelector; + +import java.nio.ByteBuffer; + +/** + */ +public class FloatSumBufferAggregator implements BufferAggregator +{ + private final FloatColumnSelector selector; + + public FloatSumBufferAggregator( + FloatColumnSelector selector + ) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + buf.putFloat(position, 0.0f); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putFloat(position, buf.getFloat(position) + selector.get()); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return buf.getFloat(position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return buf.getFloat(position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getFloat(position); + } + + @Override + public int getInt(ByteBuffer buf, int position) + { + return (int) buf.getFloat(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return (double) buf.getFloat(position); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregator.java b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregator.java index f9f50a8d494a..6f16a19bdca6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregator.java @@ -80,6 +80,18 @@ public long getLong() throw new UnsupportedOperationException("HistogramAggregator does not support getLong()"); } + @Override + public int getInt() + { + throw new UnsupportedOperationException("HistogramAggregator does not support getInt()"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("HistogramAggregator does not support getDouble()"); + } + @Override public String getName() { 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 7a7ae283d0db..41753ead1e58 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregatorFactory.java @@ -183,6 +183,12 @@ public Object getAggregatorStartValue() return new Histogram(breaks); } + @Override + public boolean isComplex() + { + return true; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java index d41244e4c66a..2b42ee5672c7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java @@ -96,6 +96,18 @@ public long getLong(ByteBuffer buf, int position) throw new UnsupportedOperationException("HistogramBufferAggregator does not support getLong()"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HistogramBufferAggregator does not support getInt()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HistogramBufferAggregator does not support getDouble()"); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/IntSumAggregator.java b/processing/src/main/java/io/druid/query/aggregation/IntSumAggregator.java new file mode 100644 index 000000000000..474965278a09 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/IntSumAggregator.java @@ -0,0 +1,118 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import com.google.common.collect.Ordering; +import com.google.common.primitives.Ints; +import io.druid.segment.IntColumnSelector; + +import java.util.Comparator; + +/** + */ +public class IntSumAggregator implements Aggregator +{ + static final Comparator COMPARATOR = new Ordering() + { + @Override + public int compare(Object o, Object o1) + { + return Ints.compare(((Number) o).intValue(), ((Number) o1).intValue()); + } + }.nullsFirst(); + + static int combineValues(Object lhs, Object rhs) + { + return ((Number) lhs).intValue() + ((Number) rhs).intValue(); + } + + private final IntColumnSelector selector; + private final String name; + + private int sum; + + public IntSumAggregator(String name, IntColumnSelector selector) + { + this.name = name; + this.selector = selector; + + this.sum = 0; + } + + @Override + public void aggregate() + { + sum += selector.get(); + } + + @Override + public void reset() + { + sum = 0; + } + + @Override + public Object get() + { + return sum; + } + + @Override + public float getFloat() + { + return (float) sum; + } + + @Override + public long getLong() + { + return (long) sum; + } + + @Override + public int getInt() + { + return sum; + } + + @Override + public double getDouble() + { + return (double) sum; + } + + @Override + public String getName() + { + return this.name; + } + + @Override + public Aggregator clone() + { + return new IntSumAggregator(name, selector); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/IntSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/IntSumAggregatorFactory.java new file mode 100644 index 000000000000..d9cf6f3f8942 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/IntSumAggregatorFactory.java @@ -0,0 +1,200 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import com.metamx.common.StringUtils; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class IntSumAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0xA; + + private final String fieldName; + private final String name; + + @JsonCreator + public IntSumAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new IntSumAggregator( + name, + metricFactory.makeIntColumnSelector(fieldName) + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new IntSumBufferAggregator(metricFactory.makeIntColumnSelector(fieldName)); + } + + @Override + public Comparator getComparator() + { + return IntSumAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return IntSumAggregator.combineValues(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new IntSumAggregatorFactory(name, name); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new IntSumAggregatorFactory(fieldName, fieldName)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Integer.parseInt((String) object); + } + return object; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "int"; + } + + @Override + public int getMaxIntermediateSize() + { + return Ints.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return 0; + } + + @Override + public boolean isComplex() + { + return false; + } + + @Override + public String toString() + { + return "IntSumAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + IntSumAggregatorFactory that = (IntSumAggregatorFactory) 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; + } + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/IntSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/IntSumBufferAggregator.java new file mode 100644 index 000000000000..1c2f7d8eca64 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/IntSumBufferAggregator.java @@ -0,0 +1,86 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import io.druid.segment.IntColumnSelector; + +import java.nio.ByteBuffer; + +/** + */ +public class IntSumBufferAggregator implements BufferAggregator +{ + private final IntColumnSelector selector; + + public IntSumBufferAggregator( + IntColumnSelector selector + ) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + buf.putInt(position, 0); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putInt(position, buf.getInt(position) + selector.get()); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return buf.getInt(position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return (float) buf.getInt(position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return (long) buf.getInt(position); + } + + @Override + public int getInt(ByteBuffer buf, int position) + { + return buf.getInt(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return (double) buf.getInt(position); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java index b9768b6dd09d..de115ff67f1a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregator.java @@ -82,6 +82,18 @@ public long getLong() return (long) current; } + @Override + public int getInt() + { + return (int) current; + } + + @Override + public double getDouble() + { + return current; + } + @Override public String getName() { 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 5a30c6c83746..7c4a4fbf052c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -243,6 +243,12 @@ public Object getAggregatorStartValue() return compiledScript.reset(); } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java index c2acca6fb349..4d01a2c9cc62 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java @@ -60,10 +60,9 @@ public Object get(ByteBuffer buf, int position) @Override public float getFloat(ByteBuffer buf, int position) { - return (float)buf.getDouble(position); + return (float) buf.getDouble(position); } - @Override public long getLong(ByteBuffer buf, int position) { @@ -71,7 +70,20 @@ public long getLong(ByteBuffer buf, int position) } @Override - public void close() { + public int getInt(ByteBuffer buf, int position) + { + return (int) buf.getDouble(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + + @Override + public void close() + { script.close(); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java index 08915de3a1e9..f2bd048f9d59 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java @@ -77,6 +77,18 @@ public long getLong() return max; } + @Override + public int getInt() + { + return (int) max; + } + + @Override + public double getDouble() + { + return max; + } + @Override public String getName() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java index 78a7c8c1c81a..dd69dc4e5fd1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java @@ -146,6 +146,12 @@ public Object getAggregatorStartValue() return Long.MIN_VALUE; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java index 291245a6faed..b3ac47675aaa 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java @@ -64,6 +64,18 @@ public long getLong(ByteBuffer buf, int position) return buf.getLong(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return buf.getInt(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java index e883fbbff886..27700fb93ebc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java @@ -77,6 +77,18 @@ public long getLong() return min; } + @Override + public int getInt() + { + return (int) min; + } + + @Override + public double getDouble() + { + return (double) min; + } + @Override public String getName() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java index 4c31189c1daf..68552d0859cb 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java @@ -146,6 +146,12 @@ public Object getAggregatorStartValue() return Long.MAX_VALUE; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java index 29c5a29e6459..06cf6ffa5189 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java @@ -65,6 +65,18 @@ public long getLong(ByteBuffer buf, int position) return buf.getLong(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return buf.getInt(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java index 63d4d14ad29d..ede3c145afd1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java @@ -84,6 +84,18 @@ public long getLong() return sum; } + @Override + public int getInt() + { + return (int) sum; + } + + @Override + public double getDouble() + { + return sum; + } + @Override public String getName() { 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 17bf61093ca9..e2168ec2720c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregatorFactory.java @@ -149,6 +149,12 @@ public Object getAggregatorStartValue() return 0; } + @Override + public boolean isComplex() + { + return false; + } + @Override public String toString() { diff --git a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java index 208155dbe8cd..07ad4e943f06 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java @@ -60,13 +60,24 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getLong(position); } - @Override public long getLong(ByteBuffer buf, int position) { return buf.getLong(position); } + @Override + public int getInt(ByteBuffer buf, int position) + { + return buf.getInt(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public void close() { 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 3ba2c15f3621..411ee4e4dbd9 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 @@ -132,6 +132,18 @@ public long getLong() throw new UnsupportedOperationException("CardinalityAggregator does not support getLong()"); } + @Override + public int getInt() + { + throw new UnsupportedOperationException("CardinalityAggregator does not support getInt()"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("CardinalityAggregator does not support getDouble()"); + } + @Override public String getName() { 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 6b177116e036..5bd696acfa3b 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 @@ -240,6 +240,12 @@ public Object getAggregatorStartValue() return HyperLogLogCollector.makeLatestCollector(); } + @Override + public boolean isComplex() + { + return true; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java index 2f8f6fe27c1e..12deede9f2a8 100644 --- a/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java @@ -82,13 +82,24 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getFloat()"); } - @Override public long getLong(ByteBuffer buf, int position) { throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getLong()"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getInt()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getDouble()"); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java index ca1f45fc4329..8cb0b8ee118c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregator.java @@ -72,6 +72,18 @@ public long getLong() throw new UnsupportedOperationException("HyperUniquesAggregator does not support getLong()"); } + @Override + public int getInt() + { + throw new UnsupportedOperationException("HyperUniquesAggregator does not support getInt()"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("HyperUniquesAggregator does not support getDouble()"); + } + @Override public String getName() { 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 7825de72e038..6cba80900cf2 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 @@ -110,10 +110,10 @@ public Comparator getComparator() @Override public int compare(HyperLogLogCollector lhs, HyperLogLogCollector rhs) { - if(lhs == null) { + if (lhs == null) { return -1; } - if(rhs == null) { + if (rhs == null) { return 1; } return lhs.compareTo(rhs); @@ -212,6 +212,12 @@ public Object getAggregatorStartValue() return HyperLogLogCollector.makeLatestCollector(); } + @Override + public boolean isComplex() + { + return true; + } + @Override public String toString() { @@ -224,13 +230,21 @@ public String toString() @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; + } HyperUniquesAggregatorFactory that = (HyperUniquesAggregatorFactory) o; - if (!fieldName.equals(that.fieldName)) return false; - if (!name.equals(that.name)) return false; + if (!fieldName.equals(that.fieldName)) { + return false; + } + if (!name.equals(that.name)) { + return false; + } return true; } diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java index 7e8902c2ec22..c995616fb399 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java @@ -82,13 +82,24 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getFloat()"); } - @Override public long getLong(ByteBuffer buf, int position) { throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getLong()"); } + @Override + public int getInt(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getInt()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getDouble()"); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java index cf59db44d288..24023394e038 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java @@ -24,6 +24,8 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; @@ -81,6 +83,12 @@ public Map analyze( case FLOAT: analysis = analyzeFloatColumn(column, analysisTypes); break; + case INT: + analysis = analyzeIntColumn(column, analysisTypes); + break; + case DOUBLE: + analysis = analyzeDoubleColumn(column, analysisTypes); + break; case STRING: analysis = analyzeStringColumn(column, analysisTypes); break; @@ -137,6 +145,20 @@ public Map analyze( numRows, NUM_BYTES_IN_TEXT_FLOAT ); break; + case INT: + analysis = lengthBasedAnalysisForAdapter( + analysisTypes, + capType.name(), capabilities, + numRows, Ints.BYTES + ); + break; + case DOUBLE: + analysis = lengthBasedAnalysisForAdapter( + analysisTypes, + capType.name(), capabilities, + numRows, Double.BYTES + ); + break; case STRING: analysis = new ColumnAnalysis( capType.name(), @@ -180,6 +202,16 @@ public ColumnAnalysis analyzeFloatColumn(Column column, EnumSet analysisTypes) + { + return lengthBasedAnalysis(column, Ints.BYTES, analysisTypes); + } + + public ColumnAnalysis analyzeDoubleColumn(Column column, EnumSet analysisTypes) + { + return lengthBasedAnalysis(column, Doubles.BYTES, analysisTypes); + } + private ColumnAnalysis lengthBasedAnalysis( Column column, final int numBytes, diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java index 2aff7f831c19..e657543eda92 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java @@ -29,5 +29,7 @@ public interface ColumnSelectorFactory public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec); public FloatColumnSelector makeFloatColumnSelector(String columnName); public LongColumnSelector makeLongColumnSelector(String columnName); + public IntColumnSelector makeIntColumnSelector(String columnName); + public DoubleColumnSelector makeDoubleColumnSelector(String columnName); public ObjectColumnSelector makeObjectColumnSelector(String columnName); } diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java new file mode 100644 index 000000000000..01657c4c9845 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java @@ -0,0 +1,27 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment; + +/** + */ +public interface DoubleColumnSelector +{ + public double get(); +} diff --git a/processing/src/main/java/io/druid/segment/DoubleMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/DoubleMetricColumnSerializer.java new file mode 100644 index 000000000000..7bf72a99df00 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DoubleMetricColumnSerializer.java @@ -0,0 +1,81 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment; + +import com.google.common.io.Files; +import io.druid.segment.data.CompressedDoublesSupplierSerializer; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.IOPeon; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class DoubleMetricColumnSerializer implements MetricColumnSerializer +{ + private final String metricName; + private final IOPeon ioPeon; + private final File outDir; + + private CompressedDoublesSupplierSerializer writer; + + public DoubleMetricColumnSerializer( + String metricName, + File outDir, + IOPeon ioPeon + ) + { + this.metricName = metricName; + this.ioPeon = ioPeon; + this.outDir = outDir; + } + + @Override + public void open() throws IOException + { + writer = CompressedDoublesSupplierSerializer.create( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ); + + writer.open(); + } + + @Override + public void serialize(Object obj) throws IOException + { + double val = (obj == null) ? 0 : ((Number) obj).doubleValue(); + writer.add(val); + } + + @Override + public void close() throws IOException + { + final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); + outFile.delete(); + MetricHolder.writeDoubleMetric( + Files.newOutputStreamSupplier(outFile, true), metricName, writer + ); + IndexIO.checkFileSize(outFile); + + writer = null; + } +} diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 2db5c23ca074..f1414bbbecb3 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -73,8 +73,10 @@ import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; import io.druid.segment.serde.DictionaryEncodedColumnSupplier; +import io.druid.segment.serde.DoubleGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnSupplier; +import io.druid.segment.serde.IntGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnSupplier; import io.druid.segment.serde.SpatialIndexColumnPartSupplier; @@ -764,6 +766,14 @@ public int size() builder.setValueType(ValueType.FLOAT); builder.addSerde(new FloatGenericColumnPartSerde(holder.floatType, BYTE_ORDER)); break; + case INT: + builder.setValueType(ValueType.INT); + builder.addSerde(new IntGenericColumnPartSerde(holder.intType, BYTE_ORDER)); + break; + case DOUBLE: + builder.setValueType(ValueType.DOUBLE); + builder.addSerde(new DoubleGenericColumnPartSerde(holder.doubleType, BYTE_ORDER)); + break; case COMPLEX: if (!(holder.complexType instanceof GenericIndexed)) { throw new ISE("Serialized complex types must be GenericIndexed objects."); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 6c61b857532a..deae537ac1f9 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -280,7 +280,7 @@ private List getMergedDimensions(List indexes) Indexed dimOrder = indexes.get(0).getDimensionNames(); for (IndexableAdapter index : indexes) { Indexed dimOrder2 = index.getDimensionNames(); - if(!Iterators.elementsEqual(dimOrder.iterator(), dimOrder2.iterator())) { + if (!Iterators.elementsEqual(dimOrder.iterator(), dimOrder2.iterator())) { return getLexicographicMergedDimensions(indexes); } } @@ -716,6 +716,12 @@ public Rowboat apply(@Nullable Rowboat input) case FLOAT: metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon)); break; + case INT: + metWriters.add(new IntMetricColumnSerializer(metric, v8OutDir, ioPeon)); + break; + case DOUBLE: + metWriters.add(new DoubleMetricColumnSerializer(metric, v8OutDir, ioPeon)); + break; case COMPLEX: final String typeName = metricTypeNames.get(metric); ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); diff --git a/processing/src/main/java/io/druid/segment/IntColumnSelector.java b/processing/src/main/java/io/druid/segment/IntColumnSelector.java new file mode 100644 index 000000000000..0029af208a00 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/IntColumnSelector.java @@ -0,0 +1,27 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment; + +/** + */ +public interface IntColumnSelector +{ + public int get(); +} diff --git a/processing/src/main/java/io/druid/segment/IntMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/IntMetricColumnSerializer.java new file mode 100644 index 000000000000..06f4bb1e326c --- /dev/null +++ b/processing/src/main/java/io/druid/segment/IntMetricColumnSerializer.java @@ -0,0 +1,81 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment; + +import com.google.common.io.Files; +import io.druid.segment.data.CompressedIntsSupplierSerializer; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.IOPeon; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class IntMetricColumnSerializer implements MetricColumnSerializer +{ + private final String metricName; + private final IOPeon ioPeon; + private final File outDir; + + private CompressedIntsSupplierSerializer writer; + + public IntMetricColumnSerializer( + String metricName, + File outDir, + IOPeon ioPeon + ) + { + this.metricName = metricName; + this.ioPeon = ioPeon; + this.outDir = outDir; + } + + @Override + public void open() throws IOException + { + writer = CompressedIntsSupplierSerializer.create( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ); + + writer.open(); + } + + @Override + public void serialize(Object obj) throws IOException + { + int val = (obj == null) ? 0 : ((Number) obj).intValue(); + writer.add(val); + } + + @Override + public void close() throws IOException + { + final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); + outFile.delete(); + MetricHolder.writeIntMetric( + Files.newOutputStreamSupplier(outFile, true), metricName, writer + ); + IndexIO.checkFileSize(outFile); + + writer = null; + } +} diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 61c242c7ca5e..799ff89e5193 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -25,14 +25,20 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import io.druid.common.utils.SerializerUtils; +import io.druid.segment.data.CompressedDoublesIndexedSupplier; +import io.druid.segment.data.CompressedDoublesSupplierSerializer; import io.druid.segment.data.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedFloatsSupplierSerializer; +import io.druid.segment.data.CompressedIntsIndexedSupplier; +import io.druid.segment.data.CompressedIntsSupplierSerializer; import io.druid.segment.data.CompressedLongsIndexedSupplier; import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedLongs; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexMetricSerde; @@ -52,20 +58,6 @@ public class MetricHolder private static final byte[] version = new byte[]{0x0}; private static final SerializerUtils serializerUtils = new SerializerUtils(); - public static MetricHolder floatMetric(String name, CompressedFloatsIndexedSupplier column) - { - MetricHolder retVal = new MetricHolder(name, "float"); - retVal.floatType = column; - return retVal; - } - - public static MetricHolder complexMetric(String name, String typeName, Indexed column) - { - MetricHolder retVal = new MetricHolder(name, typeName); - retVal.complexType = column; - return retVal; - } - public static void writeComplexMetric( OutputSupplier outSupplier, String name, String typeName, GenericIndexedWriter column ) throws IOException @@ -102,6 +94,26 @@ public static void writeLongMetric( column.closeAndConsolidate(outSupplier); } + public static void writeIntMetric( + OutputSupplier outSupplier, String name, CompressedIntsSupplierSerializer column + ) throws IOException + { + ByteStreams.write(version, outSupplier); + serializerUtils.writeString(outSupplier, name); + serializerUtils.writeString(outSupplier, "int"); + column.closeAndConsolidate(outSupplier); + } + + public static void writeDoubleMetric( + OutputSupplier outSupplier, String name, CompressedDoublesSupplierSerializer column + ) throws IOException + { + ByteStreams.write(version, outSupplier); + serializerUtils.writeString(outSupplier, name); + serializerUtils.writeString(outSupplier, "double"); + column.closeAndConsolidate(outSupplier); + } + public static void writeToChannel(MetricHolder holder, WritableByteChannel out) throws IOException { out.write(ByteBuffer.wrap(version)); @@ -112,6 +124,14 @@ public static void writeToChannel(MetricHolder holder, WritableByteChannel out) case FLOAT: holder.floatType.writeToChannel(out); break; + case LONG: + holder.longType.writeToChannel(out); + break; + case INT: + holder.intType.writeToChannel(out); + break; + case DOUBLE: + holder.doubleType.writeToChannel(out); case COMPLEX: if (holder.complexType instanceof GenericIndexed) { ((GenericIndexed) holder.complexType).writeToChannel(out); @@ -145,6 +165,12 @@ public static MetricHolder fromByteBuffer(ByteBuffer buf, ObjectStrategy strateg case FLOAT: holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); break; + case INT: + holder.intType = CompressedIntsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); + break; + case DOUBLE: + holder.doubleType = CompressedDoublesIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); + break; case COMPLEX: if (strategy != null) { holder.complexType = GenericIndexed.read(buf, strategy); @@ -171,6 +197,8 @@ public enum MetricType { LONG, FLOAT, + INT, + DOUBLE, COMPLEX; static MetricType determineType(String typeName) @@ -179,6 +207,10 @@ static MetricType determineType(String typeName) return LONG; } else if ("float".equalsIgnoreCase(typeName)) { return FLOAT; + } else if ("int".equalsIgnoreCase(typeName)) { + return INT; + } else if ("double".equalsIgnoreCase(typeName)) { + return DOUBLE; } return COMPLEX; } @@ -186,6 +218,9 @@ static MetricType determineType(String typeName) CompressedLongsIndexedSupplier longType = null; CompressedFloatsIndexedSupplier floatType = null; + CompressedIntsIndexedSupplier intType = null; + CompressedDoublesIndexedSupplier doubleType = null; + Indexed complexType = null; private MetricHolder( @@ -225,6 +260,18 @@ public IndexedFloats getFloatType() return floatType.get(); } + public IndexedInts getIntType() + { + assertType(MetricType.INT); + return intType.get(); + } + + public IndexedDoubles getDoubleType() + { + assertType(MetricType.DOUBLE); + return doubleType.get(); + } + public Indexed getComplexType() { assertType(MetricType.COMPLEX); @@ -243,6 +290,14 @@ public MetricHolder convertByteOrder(ByteOrder order) retVal = new MetricHolder(name, typeName); retVal.floatType = floatType.convertByteOrder(order); return retVal; + case INT: + retVal = new MetricHolder(name, typeName); + retVal.intType = intType.convertByteOrder(order); + return retVal; + case DOUBLE: + retVal = new MetricHolder(name, typeName); + retVal.doubleType = doubleType.convertByteOrder(order); + return retVal; case COMPLEX: return this; } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 31c555bb61d4..01b51f1792ab 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -34,7 +34,9 @@ import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.EmptyBitmapIndexSeeker; import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedDoublesGenericColumn; import io.druid.segment.column.IndexedFloatsGenericColumn; +import io.druid.segment.column.IndexedIntsGenericColumn; import io.druid.segment.column.IndexedLongsGenericColumn; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayBasedIndexedInts; @@ -270,6 +272,10 @@ public Rowboat next() metricArray[i] = ((GenericColumn) metrics[i]).getFloatSingleValueRow(currRow); } else if (metrics[i] instanceof IndexedLongsGenericColumn) { metricArray[i] = ((GenericColumn) metrics[i]).getLongSingleValueRow(currRow); + } else if (metrics[i] instanceof IndexedIntsGenericColumn) { + metricArray[i] = ((GenericColumn) metrics[i]).getIntSingleValueRow(currRow); + } else if (metrics[i] instanceof IndexedDoublesGenericColumn) { + metricArray[i] = ((GenericColumn) metrics[i]).getDoubleSingleValueRow(currRow); } else if (metrics[i] instanceof ComplexColumn) { metricArray[i] = ((ComplexColumn) metrics[i]).getRowValue(currRow); } @@ -322,6 +328,10 @@ public String getMetricType(String metric) return "float"; case LONG: return "long"; + case INT: + return "int"; + case DOUBLE: + return "double"; case COMPLEX: return column.getComplexColumn().getTypeName(); default: @@ -380,7 +390,8 @@ public IndexedInts seek(String value) if (lastVal != null) { if (GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) { throw new ISE("Value[%s] is less than the last value[%s] I have, cannot be.", - value, lastVal); + value, lastVal + ); } return new EmptyIndexedInts(); } @@ -399,7 +410,8 @@ public IndexedInts seek(String value) return ret; } else if (compareResult < 0) { throw new ISE("Skipped currValue[%s], currIndex[%,d]; incoming value[%s]", - currVal, currIndex, value); + currVal, currIndex, value + ); } else { return new EmptyIndexedInts(); } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index e67cf05df405..ac5b6665a1e1 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -347,7 +347,7 @@ public int getValueCardinality() public String lookupName(int id) { final String value = column.lookupName(id); - return extractionFn == null ? + return extractionFn == null ? Strings.nullToEmpty(value) : extractionFn.apply(Strings.nullToEmpty(value)); } @@ -438,8 +438,7 @@ public FloatColumnSelector makeFloatColumnSelector(String columnName) if (cachedMetricVals == null) { Column holder = index.getColumn(columnName); - if (holder != null && (holder.getCapabilities().getType() == ValueType.FLOAT - || holder.getCapabilities().getType() == ValueType.LONG)) { + if (holder != null && (holder.getCapabilities().isNumeric())) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(columnName, cachedMetricVals); } @@ -474,8 +473,7 @@ public LongColumnSelector makeLongColumnSelector(String columnName) if (cachedMetricVals == null) { Column holder = index.getColumn(columnName); - if (holder != null && (holder.getCapabilities().getType() == ValueType.LONG - || holder.getCapabilities().getType() == ValueType.FLOAT)) { + if (holder != null && (holder.getCapabilities().isNumeric())) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(columnName, cachedMetricVals); } @@ -503,6 +501,76 @@ public long get() }; } + @Override + public IntColumnSelector makeIntColumnSelector(String columnName) + { + GenericColumn cachedMetricVals = genericColumnCache.get(columnName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(columnName); + if (holder != null && (holder.getCapabilities().isNumeric())) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(columnName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new IntColumnSelector() + { + @Override + public int get() + { + return 0; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new IntColumnSelector() + { + @Override + public int get() + { + return metricVals.getIntSingleValueRow(cursorOffset.getOffset()); + } + }; + } + + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + GenericColumn cachedMetricVals = genericColumnCache.get(columnName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(columnName); + if (holder != null && (holder.getCapabilities().isNumeric())) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(columnName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new DoubleColumnSelector() + { + @Override + public double get() + { + return 0.0d; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new DoubleColumnSelector() + { + @Override + public double get() + { + return metricVals.getDoubleSingleValueRow(cursorOffset.getOffset()); + } + }; + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String column) { @@ -574,6 +642,38 @@ public Long get() } }; } + if (type == ValueType.INT) { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Integer.TYPE; + } + + @Override + public Integer get() + { + return columnVals.getIntSingleValueRow(cursorOffset.getOffset()); + } + }; + } + if (type == ValueType.DOUBLE) { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Double.TYPE; + } + + @Override + public Double get() + { + return columnVals.getDoubleSingleValueRow(cursorOffset.getOffset()); + } + }; + } if (type == ValueType.STRING) { return new ObjectColumnSelector() { diff --git a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java index f6b416415326..bb175f460631 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java @@ -24,6 +24,7 @@ public interface ColumnCapabilities { public ValueType getType(); + public boolean isNumeric(); public boolean isDictionaryEncoded(); public boolean isRunLengthEncoded(); diff --git a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java index f8b14e37ceed..44cd44956a9b 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java @@ -46,6 +46,12 @@ public ColumnCapabilitiesImpl setType(ValueType type) return this; } + @Override + public boolean isNumeric() + { + return (type == ValueType.FLOAT) || (type == ValueType.LONG) || (type == ValueType.INT) || (type == ValueType.DOUBLE); + } + @Override @JsonProperty public boolean isDictionaryEncoded() diff --git a/processing/src/main/java/io/druid/segment/column/GenericColumn.java b/processing/src/main/java/io/druid/segment/column/GenericColumn.java index 7bb75f15a519..46b863a81b97 100644 --- a/processing/src/main/java/io/druid/segment/column/GenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/GenericColumn.java @@ -20,7 +20,9 @@ package io.druid.segment.column; import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedLongs; import java.io.Closeable; @@ -30,13 +32,28 @@ public interface GenericColumn extends Closeable { public int length(); + public ValueType getType(); + public boolean hasMultipleValues(); public String getStringSingleValueRow(int rowNum); + public Indexed getStringMultiValueRow(int rowNum); + public float getFloatSingleValueRow(int rowNum); + public IndexedFloats getFloatMultiValueRow(int rowNum); + public long getLongSingleValueRow(int rowNum); + public IndexedLongs getLongMultiValueRow(int rowNum); + + public int getIntSingleValueRow(int rowNum); + + public IndexedInts getIntMultiValueRow(int rowNum); + + public double getDoubleSingleValueRow(int rowNum); + + public IndexedDoubles getDoubleMultiValueRow(int rowNum); } diff --git a/processing/src/main/java/io/druid/segment/column/IndexedDoublesGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedDoublesGenericColumn.java new file mode 100644 index 000000000000..96658c1d7102 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/IndexedDoublesGenericColumn.java @@ -0,0 +1,126 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.column; + +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedLongs; + +import java.io.IOException; + +/** + */ +public class IndexedDoublesGenericColumn implements GenericColumn +{ + private final IndexedDoubles column; + + public IndexedDoublesGenericColumn( + final IndexedDoubles column + ) + { + this.column = column; + } + + @Override + public int length() + { + return column.size(); + } + + @Override + public ValueType getType() + { + return ValueType.DOUBLE; + } + + @Override + public boolean hasMultipleValues() + { + return false; + } + + @Override + public String getStringSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public Indexed getStringMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + return (float) column.get(rowNum); + } + + @Override + public IndexedFloats getFloatMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getIntSingleValueRow(int rowNum) + { + return (int) column.get(rowNum); + } + + @Override + public IndexedInts getIntMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public IndexedDoubles getDoubleMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + return (long) column.get(rowNum); + } + + @Override + public IndexedLongs getLongMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + column.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java index 5335945c3e49..1a09bac203d3 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java @@ -20,20 +20,23 @@ package io.druid.segment.column; import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedLongs; import java.io.IOException; /** -*/ + */ public class IndexedFloatsGenericColumn implements GenericColumn { private final IndexedFloats column; public IndexedFloatsGenericColumn( final IndexedFloats column - ) { + ) + { this.column = column; } @@ -91,6 +94,30 @@ public IndexedLongs getLongMultiValueRow(int rowNum) throw new UnsupportedOperationException(); } + @Override + public int getIntSingleValueRow(int rowNum) + { + return (int) column.get(rowNum); + } + + @Override + public IndexedInts getIntMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + return (double) column.get(rowNum); + } + + @Override + public IndexedDoubles getDoubleMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + @Override public void close() throws IOException { diff --git a/processing/src/main/java/io/druid/segment/column/IndexedIntsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedIntsGenericColumn.java new file mode 100644 index 000000000000..c24d998581fa --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/IndexedIntsGenericColumn.java @@ -0,0 +1,126 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.column; + +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; +import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedLongs; + +import java.io.IOException; + +/** + */ +public class IndexedIntsGenericColumn implements GenericColumn +{ + private final IndexedInts column; + + public IndexedIntsGenericColumn( + final IndexedInts column + ) + { + this.column = column; + } + + @Override + public int length() + { + return column.size(); + } + + @Override + public ValueType getType() + { + return ValueType.LONG; + } + + @Override + public boolean hasMultipleValues() + { + return false; + } + + @Override + public String getStringSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public Indexed getStringMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + return (float) column.get(rowNum); + } + + @Override + public IndexedFloats getFloatMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + return (long) column.get(rowNum); + } + + @Override + public IndexedLongs getLongMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getIntSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public IndexedInts getIntMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + return (double) column.get(rowNum); + } + + @Override + public IndexedDoubles getDoubleMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + column.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java index f0a25a96b19a..34bcbf706c9b 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java @@ -20,20 +20,23 @@ package io.druid.segment.column; import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; import io.druid.segment.data.IndexedFloats; +import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedLongs; import java.io.IOException; /** -*/ + */ public class IndexedLongsGenericColumn implements GenericColumn { private final IndexedLongs column; public IndexedLongsGenericColumn( final IndexedLongs column - ) { + ) + { this.column = column; } @@ -91,6 +94,30 @@ public IndexedLongs getLongMultiValueRow(int rowNum) throw new UnsupportedOperationException(); } + @Override + public int getIntSingleValueRow(int rowNum) + { + return (int) column.get(rowNum); + } + + @Override + public IndexedInts getIntMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + return (double) column.get(rowNum); + } + + @Override + public IndexedDoubles getDoubleMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + @Override public void close() throws IOException { diff --git a/processing/src/main/java/io/druid/segment/column/ValueType.java b/processing/src/main/java/io/druid/segment/column/ValueType.java index aa9919a129b9..49008da40d25 100644 --- a/processing/src/main/java/io/druid/segment/column/ValueType.java +++ b/processing/src/main/java/io/druid/segment/column/ValueType.java @@ -26,5 +26,7 @@ public enum ValueType FLOAT, LONG, STRING, + INT, + DOUBLE, COMPLEX } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java new file mode 100644 index 000000000000..37838b8f974d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java @@ -0,0 +1,72 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.data; + +import com.google.common.collect.Ordering; +import com.google.common.primitives.Doubles; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.DoubleBuffer; + +/** +*/ +public class CompressedDoubleBufferObjectStrategy extends FixedSizeCompressedObjectStrategy +{ + public static CompressedDoubleBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) + { + return new CompressedDoubleBufferObjectStrategy(order, compression, sizePer); + } + + private CompressedDoubleBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) + { + super( + order, + new BufferConverter() + { + @Override + public DoubleBuffer convert(ByteBuffer buf) + { + return buf.asDoubleBuffer(); + } + + @Override + public int compare(DoubleBuffer lhs, DoubleBuffer rhs) + { + return Ordering.natural().nullsFirst().compare(lhs, rhs); + } + + @Override + public int sizeOf(int count) + { + return count * Doubles.BYTES; + } + + @Override + public DoubleBuffer combine(ByteBuffer into, DoubleBuffer from) + { + return into.asDoubleBuffer().put(from); + } + }, + compression, + sizePer + ); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java new file mode 100644 index 000000000000..278ffddd43ba --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java @@ -0,0 +1,317 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.data; + +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.io.Closeables; +import com.google.common.primitives.Doubles; +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; +import io.druid.segment.CompressedPools; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.DoubleBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.Iterator; + +/** + */ +public class CompressedDoublesIndexedSupplier implements Supplier +{ + public static final byte LZF_VERSION = 0x1; + public static final byte version = 0x2; + public static final int MAX_DOUBLES_IN_BUFFER = CompressedPools.BUFFER_SIZE / Doubles.BYTES; + + private final int totalSize; + private final int sizePer; + private final GenericIndexed> baseDoubleBuffers; + private final CompressedObjectStrategy.CompressionStrategy compression; + + CompressedDoublesIndexedSupplier( + int totalSize, + int sizePer, + GenericIndexed> baseDoubleBuffers, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.totalSize = totalSize; + this.sizePer = sizePer; + this.baseDoubleBuffers = baseDoubleBuffers; + this.compression = compression; + } + + public int size() + { + return totalSize; + } + + @Override + public IndexedDoubles get() + { + final int div = Integer.numberOfTrailingZeros(sizePer); + final int rem = sizePer - 1; + final boolean powerOf2 = sizePer == (1 << div); + if(powerOf2) { + return new CompressedIndexedDoubles() { + @Override + public double get(int index) + { + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + final int bufferIndex = index & rem; + return buffer.get(buffer.position() + bufferIndex); + } + }; + } else { + return new CompressedIndexedDoubles(); + } + } + + public long getSerializedSize() + { + return baseDoubleBuffers.getSerializedSize() + 1 + 4 + 4 + 1; + } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{version})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + baseDoubleBuffers.writeToChannel(channel); + } + + public CompressedDoublesIndexedSupplier convertByteOrder(ByteOrder order) + { + return new CompressedDoublesIndexedSupplier( + totalSize, + sizePer, + GenericIndexed.fromIterable(baseDoubleBuffers, CompressedDoubleBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), + compression + ); + } + + /** + * For testing. Do not depend on unless you like things breaking. + */ + GenericIndexed> getBaseDoubleBuffers() + { + return baseDoubleBuffers; + } + + public static CompressedDoublesIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) + { + byte versionFromBuffer = buffer.get(); + + if (versionFromBuffer == version) { + final int totalSize = buffer.getInt(); + final int sizePer = buffer.getInt(); + final CompressedObjectStrategy.CompressionStrategy compression = + CompressedObjectStrategy.CompressionStrategy.forId(buffer.get()); + + return new CompressedDoublesIndexedSupplier( + totalSize, + sizePer, + GenericIndexed.read( + buffer, + CompressedDoubleBufferObjectStrategy.getBufferForOrder( + order, + compression, + sizePer + ) + ), + compression + ); + } else if (versionFromBuffer == LZF_VERSION) { + final int totalSize = buffer.getInt(); + final int sizePer = buffer.getInt(); + final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; + return new CompressedDoublesIndexedSupplier( + totalSize, + sizePer, + GenericIndexed.read( + buffer, + CompressedDoubleBufferObjectStrategy.getBufferForOrder( + order, + compression, + sizePer + ) + ), + compression + ); + } + + throw new IAE("Unknown version[%s]", versionFromBuffer); + } + + public static CompressedDoublesIndexedSupplier fromDoubleBuffer(DoubleBuffer buffer, final ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression) + { + return fromDoubleBuffer(buffer, MAX_DOUBLES_IN_BUFFER, order, compression); + } + + public static CompressedDoublesIndexedSupplier fromDoubleBuffer( + final DoubleBuffer buffer, final int chunkFactor, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + ) + { + Preconditions.checkArgument( + chunkFactor <= MAX_DOUBLES_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor + ); + + return new CompressedDoublesIndexedSupplier( + buffer.remaining(), + chunkFactor, + GenericIndexed.fromIterable( + new Iterable>() + { + @Override + public Iterator> iterator() + { + return new Iterator>() + { + DoubleBuffer myBuffer = buffer.asReadOnlyBuffer(); + + @Override + public boolean hasNext() + { + return myBuffer.hasRemaining(); + } + + @Override + public ResourceHolder next() + { + final DoubleBuffer retVal = myBuffer.asReadOnlyBuffer(); + + if (chunkFactor < myBuffer.remaining()) { + retVal.limit(retVal.position() + chunkFactor); + } + myBuffer.position(myBuffer.position() + retVal.remaining()); + + return StupidResourceHolder.create(retVal); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + }; + } + }, + CompressedDoubleBufferObjectStrategy.getBufferForOrder(order, compression, chunkFactor) + ), + compression + ); + } + + private class CompressedIndexedDoubles implements IndexedDoubles + { + final Indexed> singleThreadedDoubleBuffers = baseDoubleBuffers.singleThreaded(); + + int currIndex = -1; + ResourceHolder holder; + DoubleBuffer buffer; + + @Override + public int size() + { + return totalSize; + } + + @Override + public double get(final int index) + { + // division + remainder is optimized by the compiler so keep those together + final int bufferNum = index / sizePer; + final int bufferIndex = index % sizePer; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + return buffer.get(buffer.position() + bufferIndex); + } + + @Override + public void fill(int index, double[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + + int bufferNum = index / sizePer; + int bufferIndex = index % sizePer; + + int leftToFill = toFill.length; + while (leftToFill > 0) { + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + buffer.mark(); + buffer.position(buffer.position() + bufferIndex); + final int numToGet = Math.min(buffer.remaining(), leftToFill); + buffer.get(toFill, toFill.length - leftToFill, numToGet); + buffer.reset(); + leftToFill -= numToGet; + ++bufferNum; + bufferIndex = 0; + } + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = singleThreadedDoubleBuffers.get(bufferNum); + buffer = holder.get(); + currIndex = bufferNum; + } + + @Override + public String toString() + { + return "CompressedDoublesIndexedSupplier_Anonymous{" + + "currIndex=" + currIndex + + ", sizePer=" + sizePer + + ", numChunks=" + singleThreadedDoubleBuffers.size() + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + Closeables.close(holder, false); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedDoublesSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedDoublesSupplierSerializer.java new file mode 100644 index 000000000000..6aade14cc34a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesSupplierSerializer.java @@ -0,0 +1,120 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.data; + +import com.google.common.io.ByteStreams; +import com.google.common.io.OutputSupplier; +import com.google.common.primitives.Ints; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidResourceHolder; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteOrder; +import java.nio.DoubleBuffer; + +/** + */ +public class CompressedDoublesSupplierSerializer +{ + public static CompressedDoublesSupplierSerializer create( + IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + ) throws IOException + { + return create(ioPeon, filenameBase, CompressedDoublesIndexedSupplier.MAX_DOUBLES_IN_BUFFER, order, compression); + } + + public static CompressedDoublesSupplierSerializer create( + IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + ) throws IOException + { + final CompressedDoublesSupplierSerializer retVal = new CompressedDoublesSupplierSerializer( + sizePer, + new GenericIndexedWriter>( + ioPeon, filenameBase, CompressedDoubleBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + ), + compression + ); + return retVal; + } + + private final int sizePer; + private final GenericIndexedWriter> flattener; + private final CompressedObjectStrategy.CompressionStrategy compression; + + private int numInserted = 0; + + private DoubleBuffer endBuffer; + + public CompressedDoublesSupplierSerializer( + int sizePer, + GenericIndexedWriter> flattener, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.sizePer = sizePer; + this.flattener = flattener; + this.compression = compression; + + endBuffer = DoubleBuffer.allocate(sizePer); + endBuffer.mark(); + } + + public void open() throws IOException + { + flattener.open(); + } + + public int size() + { + return numInserted; + } + + public void add(double value) throws IOException + { + if (! endBuffer.hasRemaining()) { + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = DoubleBuffer.allocate(sizePer); + endBuffer.mark(); + } + + endBuffer.put(value); + ++numInserted; + } + + public void closeAndConsolidate(OutputSupplier consolidatedOut) throws IOException + { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = null; + + flattener.close(); + + try (OutputStream out = consolidatedOut.getOutput()) { + out.write(CompressedDoublesIndexedSupplier.version); + out.write(Ints.toByteArray(numInserted)); + out.write(Ints.toByteArray(sizePer)); + out.write(new byte[]{compression.getId()}); + ByteStreams.copy(flattener.combineStreams(), out); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsSupplierSerializer.java new file mode 100644 index 000000000000..aa23568f6bf0 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsSupplierSerializer.java @@ -0,0 +1,113 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.data; + +import com.google.common.io.ByteStreams; +import com.google.common.io.OutputSupplier; +import com.google.common.primitives.Ints; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidResourceHolder; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteOrder; +import java.nio.IntBuffer; + +/** + */ +public class CompressedIntsSupplierSerializer +{ + public static CompressedIntsSupplierSerializer create( + IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + ) throws IOException + { + final CompressedIntsSupplierSerializer retVal = new CompressedIntsSupplierSerializer( + CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, + new GenericIndexedWriter>( + ioPeon, filenameBase, CompressedIntBufferObjectStrategy.getBufferForOrder(order, compression, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER) + ), + compression + ); + return retVal; + } + + private final int sizePer; + private final GenericIndexedWriter> flattener; + private final CompressedObjectStrategy.CompressionStrategy compression; + + private int numInserted = 0; + + private IntBuffer endBuffer; + + public CompressedIntsSupplierSerializer( + int sizePer, + GenericIndexedWriter> flattener, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.sizePer = sizePer; + this.flattener = flattener; + this.compression = compression; + + endBuffer = IntBuffer.allocate(sizePer); + endBuffer.mark(); + } + + public void open() throws IOException + { + flattener.open(); + } + + public int size() + { + return numInserted; + } + + public void add(int value) throws IOException + { + if (! endBuffer.hasRemaining()) { + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = IntBuffer.allocate(sizePer); + endBuffer.mark(); + } + + endBuffer.put(value); + ++numInserted; + } + + public void closeAndConsolidate(OutputSupplier consolidatedOut) throws IOException + { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = null; + + flattener.close(); + + try (OutputStream out = consolidatedOut.getOutput()) { + out.write(CompressedIntsIndexedSupplier.version); + out.write(Ints.toByteArray(numInserted)); + out.write(Ints.toByteArray(sizePer)); + out.write(new byte[]{compression.getId()}); + ByteStreams.copy(flattener.combineStreams(), out); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/IndexedDoubles.java b/processing/src/main/java/io/druid/segment/data/IndexedDoubles.java new file mode 100644 index 000000000000..d223459a6361 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/IndexedDoubles.java @@ -0,0 +1,34 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.data; + +import java.io.Closeable; + +/** + * Get a float at an index (array or list lookup abstraction without boxing). + */ +public interface IndexedDoubles extends Closeable +{ + public int size(); + + public double get(int index); + + public void fill(int index, double[] toFill); +} 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 cf7ff364c8c5..8e0c8b585f74 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -42,7 +42,9 @@ import io.druid.query.extraction.ExtractionFn; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.IntColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.Column; @@ -119,6 +121,32 @@ public float get() }; } + @Override + public IntColumnSelector makeIntColumnSelector(final String columnName) + { + return new IntColumnSelector() + { + @Override + public int get() + { + return in.get().getIntMetric(columnName); + } + }; + } + + @Override + public DoubleColumnSelector makeDoubleColumnSelector(final String columnName) + { + return new DoubleColumnSelector() + { + @Override + public double get() + { + return in.get().getDoubleMetric(columnName); + } + }; + } + @Override public ObjectColumnSelector makeObjectColumnSelector(final String column) { @@ -142,7 +170,7 @@ public Object get() if (!deserializeComplexMetrics) { return rawColumnSelector; } else { - if (typeName.equals("float")) { + if (!agg.isComplex()) { return rawColumnSelector; } @@ -334,6 +362,10 @@ public IncrementalIndex( type = ValueType.FLOAT; } else if (entry.getValue().equalsIgnoreCase("long")) { type = ValueType.LONG; + } else if (entry.getValue().equalsIgnoreCase("int")) { + type = ValueType.INT; + } else if (entry.getValue().equalsIgnoreCase("double")) { + type = ValueType.DOUBLE; } else { type = ValueType.COMPLEX; } @@ -389,6 +421,10 @@ protected abstract Integer addToFacts( protected abstract long getMetricLongValue(int rowOffset, int aggOffset); + protected abstract int getMetricIntValue(int rowOffset, int aggOffset); + + protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); + protected abstract Object getMetricObjectValue(int rowOffset, int aggOffset); @Override @@ -412,10 +448,10 @@ public InputRow formatRow(InputRow row) /** * 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. - *

- *

+ *

+ *

* Calls to add() are thread safe. - *

+ *

* * @param row the row of data to add * @@ -457,7 +493,7 @@ public int add(InputRow row) throws IndexSizeExceededException overflow = Lists.newArrayList(); } overflow.add(getDimVals(dimValues.add(dimension), dimensionValues)); - } else if (index > dims.length || dims[index] != null) { + } else if (index > dims.length || dims[index] != null) { /* * index > dims.length requires that we saw this dimension and added it to the dimensionOrder map, * otherwise index is null. Since dims is initialized based on the size of dimensionOrder on each call to add, @@ -641,8 +677,7 @@ public Row apply(final Map.Entry input) String[] dim = theDims[i]; if (dim != null && dim.length != 0) { theVals.put(dimensions.get(i), dim.length == 1 ? dim[0] : Arrays.asList(dim)); - } - else { + } else { theVals.put(dimensions.get(i), null); } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 9f3b058e2b50..0066f516d130 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -38,7 +38,9 @@ import io.druid.segment.Capabilities; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.IntColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.NullDimensionSelector; import io.druid.segment.ObjectColumnSelector; @@ -472,6 +474,58 @@ public long get() }; } + @Override + public IntColumnSelector makeIntColumnSelector(String columnName) + { + final Integer metricIndexInt = index.getMetricIndex(columnName); + if (metricIndexInt == null) { + return new IntColumnSelector() + { + @Override + public int get() + { + return 0; + } + }; + } + + final int metricIndex = metricIndexInt; + return new IntColumnSelector() + { + @Override + public int get() + { + return index.getMetricIntValue(currEntry.getValue(), metricIndex); + } + }; + } + + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + final Integer metricIndexInt = index.getMetricIndex(columnName); + if (metricIndexInt == null) { + return new DoubleColumnSelector() + { + @Override + public double get() + { + return 0.0d; + } + }; + } + + final int metricIndex = metricIndexInt; + return new DoubleColumnSelector() + { + @Override + public double get() + { + return index.getMetricDoubleValue(currEntry.getValue(), metricIndex); + } + }; + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String column) { diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java deleted file mode 100644 index 9e9a05fd0861..000000000000 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ /dev/null @@ -1,486 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment.incremental; - -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.metamx.common.ISE; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; -import io.druid.data.input.InputRow; -import io.druid.granularity.QueryGranularity; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.BufferAggregator; -import org.mapdb.BTreeKeySerializer; -import org.mapdb.DB; -import org.mapdb.DBMaker; -import org.mapdb.Serializer; -import org.mapdb.Store; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.Serializable; -import java.lang.ref.WeakReference; -import java.lang.reflect.Field; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; -import java.util.UUID; -import java.util.WeakHashMap; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.atomic.AtomicInteger; - -@Deprecated -/** - * This is not yet ready for production use and requires more work. - */ -public class OffheapIncrementalIndex extends IncrementalIndex -{ - private static final long STORE_CHUNK_SIZE; - - static - { - // MapDB allocated memory in chunks. We need to know CHUNK_SIZE - // in order to get a crude estimate of how much more direct memory - // might be used when adding an additional row. - try { - Field field = Store.class.getDeclaredField("CHUNK_SIZE"); - field.setAccessible(true); - STORE_CHUNK_SIZE = field.getLong(null); - } catch(NoSuchFieldException | IllegalAccessException e) { - throw new RuntimeException("Unable to determine MapDB store chunk size", e); - } - } - - private final ResourceHolder bufferHolder; - - private final DB db; - private final DB factsDb; - private final int[] aggPositionOffsets; - private final int totalAggSize; - private final ConcurrentNavigableMap facts; - private final int maxTotalBufferSize; - - private String outOfRowsReason = null; - - public OffheapIncrementalIndex( - IncrementalIndexSchema incrementalIndexSchema, - StupidPool bufferPool, - boolean deserializeComplexMetrics, - int maxTotalBufferSize - ) - { - super(incrementalIndexSchema, deserializeComplexMetrics); - - this.bufferHolder = bufferPool.take(); - Preconditions.checkArgument( - maxTotalBufferSize > bufferHolder.get().limit(), - "Maximum total buffer size must be greater than aggregation buffer size" - ); - - final AggregatorFactory[] metrics = incrementalIndexSchema.getMetrics(); - this.aggPositionOffsets = new int[metrics.length]; - - int currAggSize = 0; - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - aggPositionOffsets[i] = currAggSize; - currAggSize += agg.getMaxIntermediateSize(); - } - this.totalAggSize = currAggSize; - - final DBMaker dbMaker = DBMaker.newMemoryDirectDB() - .transactionDisable() - .asyncWriteEnable() - .cacheLRUEnable() - .cacheSize(16384); - - this.factsDb = dbMaker.make(); - this.db = dbMaker.make(); - final TimeAndDimsSerializer timeAndDimsSerializer = new TimeAndDimsSerializer(this); - this.facts = factsDb.createTreeMap("__facts" + UUID.randomUUID()) - .keySerializer(timeAndDimsSerializer) - .comparator(timeAndDimsSerializer.getComparator()) - .valueSerializer(Serializer.INTEGER) - .make(); - this.maxTotalBufferSize = maxTotalBufferSize; - } - - public OffheapIncrementalIndex( - long minTimestamp, - QueryGranularity gran, - final AggregatorFactory[] metrics, - StupidPool bufferPool, - boolean deserializeComplexMetrics, - int maxTotalBufferSize - ) - { - this( - new IncrementalIndexSchema.Builder().withMinTimestamp(minTimestamp) - .withQueryGranularity(gran) - .withMetrics(metrics) - .build(), - bufferPool, - deserializeComplexMetrics, - maxTotalBufferSize - ); - } - - @Override - public ConcurrentNavigableMap getFacts() - { - return facts; - } - - @Override - protected DimDim makeDimDim(String dimension) - { - return new OffheapDimDim(dimension); - } - - @Override - protected BufferAggregator[] initAggs( - AggregatorFactory[] metrics, - Supplier rowSupplier, - boolean deserializeComplexMetrics - ) - { - BufferAggregator[] aggs = new BufferAggregator[metrics.length]; - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - aggs[i] = agg.factorizeBuffered( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) - ); - } - return aggs; - } - - @Override - protected Integer addToFacts( - AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, - InputRow row, - AtomicInteger numEntries, - TimeAndDims key, - ThreadLocal rowContainer, - Supplier rowSupplier - ) throws IndexSizeExceededException - { - final BufferAggregator[] aggs = getAggs(); - Integer rowOffset; - synchronized (this) { - if (!facts.containsKey(key)) { - if (!canAppendRow(false)) { - throw new IndexSizeExceededException("%s", getOutOfRowsReason()); - } - } - rowOffset = totalAggSize * numEntries.get(); - final Integer prev = facts.putIfAbsent(key, rowOffset); - if (prev != null) { - rowOffset = prev; - } else { - numEntries.incrementAndGet(); - for (int i = 0; i < aggs.length; i++) { - aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i)); - } - } - } - rowContainer.set(row); - for (int i = 0; i < aggs.length; i++) { - synchronized (aggs[i]) { - aggs[i].aggregate(bufferHolder.get(), getMetricPosition(rowOffset, i)); - } - } - rowContainer.set(null); - return numEntries.get(); - } - - public boolean canAppendRow() { - return canAppendRow(true); - } - - private boolean canAppendRow(boolean includeFudgeFactor) - { - // there is a race condition when checking current MapDB - // when canAppendRow() is called after adding a row it may return true, but on a subsequence call - // to addToFacts that may not be the case anymore because MapDB size may have changed. - // so we add this fudge factor, hoping that will be enough. - - final int aggBufferSize = bufferHolder.get().limit(); - if ((size() + 1) * totalAggSize > aggBufferSize) { - outOfRowsReason = String.format("Maximum aggregation buffer limit reached [%d bytes].", aggBufferSize); - return false; - } - // hopefully both MapDBs will grow by at most STORE_CHUNK_SIZE each when we add the next row. - if (getCurrentSize() + totalAggSize + 2 * STORE_CHUNK_SIZE + (includeFudgeFactor ? STORE_CHUNK_SIZE : 0) > maxTotalBufferSize) { - outOfRowsReason = String.format("Maximum time and dimension buffer limit reached [%d bytes].", maxTotalBufferSize - aggBufferSize); - return false; - } - return true; - } - - public String getOutOfRowsReason() { - return outOfRowsReason; - } - - @Override - protected BufferAggregator[] getAggsForRow(int rowOffset) - { - return getAggs(); - } - - @Override - protected Object getAggVal(BufferAggregator agg, int rowOffset, int aggPosition) - { - return agg.get(bufferHolder.get(), getMetricPosition(rowOffset, aggPosition)); - } - - @Override - public float getMetricFloatValue(int rowOffset, int aggOffset) - { - return getAggs()[aggOffset].getFloat(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset)); - } - - @Override - public long getMetricLongValue(int rowOffset, int aggOffset) - { - return getAggs()[aggOffset].getLong(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset)); - } - - @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) - { - return getAggs()[aggOffset].get(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset)); - } - - @Override - public void close() - { - try { - bufferHolder.close(); - Store.forDB(db).close(); - Store.forDB(factsDb).close(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - private int getMetricPosition(int rowOffset, int metricIndex) - { - return rowOffset + aggPositionOffsets[metricIndex]; - } - - private DimDim getDimDim(int dimIndex) - { - return getDimValues().get(getDimensions().get(dimIndex)); - } - - // MapDB forces serializers to implement serializable, which sucks - private static class TimeAndDimsSerializer extends BTreeKeySerializer implements Serializable - { - private final TimeAndDimsComparator comparator; - private final transient OffheapIncrementalIndex incrementalIndex; - - TimeAndDimsSerializer(OffheapIncrementalIndex incrementalIndex) - { - this.comparator = new TimeAndDimsComparator(); - this.incrementalIndex = incrementalIndex; - } - - @Override - public void serialize(DataOutput out, int start, int end, Object[] keys) throws IOException - { - for (int i = start; i < end; i++) { - TimeAndDims timeAndDim = (TimeAndDims) keys[i]; - out.writeLong(timeAndDim.getTimestamp()); - out.writeInt(timeAndDim.getDims().length); - int index = 0; - for (String[] dims : timeAndDim.getDims()) { - if (dims == null) { - out.write(-1); - } else { - DimDim dimDim = incrementalIndex.getDimDim(index); - out.writeInt(dims.length); - for (String value : dims) { - out.writeInt(dimDim.getId(value)); - } - } - index++; - } - } - } - - @Override - public Object[] deserialize(DataInput in, int start, int end, int size) throws IOException - { - Object[] ret = new Object[size]; - for (int i = start; i < end; i++) { - final long timeStamp = in.readLong(); - final String[][] dims = new String[in.readInt()][]; - for (int k = 0; k < dims.length; k++) { - int len = in.readInt(); - if (len != -1) { - DimDim dimDim = incrementalIndex.getDimDim(k); - String[] col = new String[len]; - for (int l = 0; l < col.length; l++) { - col[l] = dimDim.get(dimDim.getValue(in.readInt())); - } - dims[k] = col; - } - } - ret[i] = new TimeAndDims(timeStamp, dims); - } - return ret; - } - - @Override - public Comparator getComparator() - { - return comparator; - } - } - - private static class TimeAndDimsComparator implements Comparator, Serializable - { - @Override - public int compare(Object o1, Object o2) - { - return ((TimeAndDims) o1).compareTo((TimeAndDims) o2); - } - } - - private class OffheapDimDim implements DimDim - { - private final Map falseIds; - private final Map falseIdsReverse; - private final WeakHashMap> cache = new WeakHashMap(); - - private volatile String[] sortedVals = null; - // size on MapDB is slow so maintain a count here - private volatile int size = 0; - - public OffheapDimDim(String dimension) - { - falseIds = db.createHashMap(dimension) - .keySerializer(Serializer.STRING) - .valueSerializer(Serializer.INTEGER) - .make(); - falseIdsReverse = db.createHashMap(dimension + "_inverse") - .keySerializer(Serializer.INTEGER) - .valueSerializer(Serializer.STRING) - .make(); - } - - /** - * Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` - * - * @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) - */ - public String get(String str) - { - final WeakReference cached = cache.get(str); - if (cached != null) { - final String value = cached.get(); - if (value != null) { - return value; - } - } - cache.put(str, new WeakReference(str)); - return str; - } - - public int getId(String value) - { - return falseIds.get(value); - } - - public String getValue(int id) - { - return falseIdsReverse.get(id); - } - - public boolean contains(String value) - { - return falseIds.containsKey(value); - } - - public int size() - { - return size; - } - - public synchronized int add(String value) - { - int id = size++; - falseIds.put(value, id); - falseIdsReverse.put(id, value); - return id; - } - - public int getSortedId(String value) - { - assertSorted(); - return Arrays.binarySearch(sortedVals, value); - } - - public String getSortedValue(int index) - { - assertSorted(); - return sortedVals[index]; - } - - public void sort() - { - if (sortedVals == null) { - sortedVals = new String[falseIds.size()]; - - int index = 0; - for (String value : falseIds.keySet()) { - sortedVals[index++] = value; - } - Arrays.sort(sortedVals); - } - } - - private void assertSorted() - { - if (sortedVals == null) { - throw new ISE("Call sort() before calling the getSorted* methods."); - } - } - - public boolean compareCannonicalValues(String s1, String s2) - { - return s1.equals(s2); - } - } - - private long getCurrentSize() - { - return Store.forDB(db).getCurrSize() + - Store.forDB(factsDb).getCurrSize() - // Size of aggregators - + size() * totalAggSize; - } -} diff --git a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java index 1930463ca2e7..245035281615 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -237,6 +237,18 @@ public long getMetricLongValue(int rowOffset, int aggOffset) return concurrentGet(rowOffset)[aggOffset].getLong(); } + @Override + public int getMetricIntValue(int rowOffset, int aggOffset) + { + return concurrentGet(rowOffset)[aggOffset].getInt(); + } + + @Override + public double getMetricDoubleValue(int rowOffset, int aggOffset) + { + return concurrentGet(rowOffset)[aggOffset].getDouble(); + } + @Override public Object getMetricObjectValue(int rowOffset, int aggOffset) { diff --git a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java index 9576a032da44..bc1e766d9db6 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java @@ -151,6 +151,28 @@ public float getFloatMetric(String metric) } } + @Override + public int getIntMetric(String metric) + { + try { + return row.getIntMetric(metric); + } + catch (ParseException e) { + throw Throwables.propagate(e); + } + } + + @Override + public double getDoubleMetric(String metric) + { + try { + return row.getDoubleMetric(metric); + } + catch (ParseException e) { + throw Throwables.propagate(e); + } + } + @Override public String toString() { @@ -223,10 +245,12 @@ private boolean isJoinedSpatialDimValValid(String dimVal) return true; } - private static Float tryParseFloat(String val) { + private static Float tryParseFloat(String val) + { try { return Float.parseFloat(val); - } catch (NullPointerException | NumberFormatException e) { + } + catch (NullPointerException | NumberFormatException e) { return null; } } diff --git a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java index 11d78ec840b0..73c8cb56bdfc 100644 --- a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -35,6 +35,8 @@ @JsonSubTypes.Type(name = "complex", value = ComplexColumnPartSerde.class), @JsonSubTypes.Type(name = "float", value = FloatGenericColumnPartSerde.class), @JsonSubTypes.Type(name = "long", value = LongGenericColumnPartSerde.class), + @JsonSubTypes.Type(name = "int", value = IntGenericColumnPartSerde.class), + @JsonSubTypes.Type(name = "double", value = DoubleGenericColumnPartSerde.class), @JsonSubTypes.Type(name = "stringDictionary", value = DictionaryEncodedColumnPartSerde.class) }) public interface ColumnPartSerde diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java new file mode 100644 index 000000000000..fa7eedd5e141 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -0,0 +1,84 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedDoublesIndexedSupplier; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** +*/ +public class DoubleGenericColumnPartSerde implements ColumnPartSerde +{ + @JsonCreator + public static DoubleGenericColumnPartSerde createDeserializer( + @JsonProperty("byteOrder") ByteOrder byteOrder + ) + { + return new DoubleGenericColumnPartSerde(null, byteOrder); + } + + private final CompressedDoublesIndexedSupplier compressedDoubles; + private final ByteOrder byteOrder; + + public DoubleGenericColumnPartSerde(CompressedDoublesIndexedSupplier compressedDoubles, ByteOrder byteOrder) + { + this.compressedDoubles = compressedDoubles; + this.byteOrder = byteOrder; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @Override + public long numBytes() + { + return compressedDoubles.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + compressedDoubles.writeToChannel(channel); + } + + @Override + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final CompressedDoublesIndexedSupplier column = CompressedDoublesIndexedSupplier.fromByteBuffer(buffer, byteOrder); + + builder.setType(ValueType.FLOAT) + .setHasMultipleValues(false) + .setGenericColumn(new DoubleGenericColumnSupplier(column, byteOrder)); + + return new DoubleGenericColumnPartSerde(column, byteOrder); + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java new file mode 100644 index 000000000000..9d95a440fe25 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java @@ -0,0 +1,49 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.serde; + +import com.google.common.base.Supplier; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedDoublesGenericColumn; +import io.druid.segment.data.CompressedDoublesIndexedSupplier; + +import java.nio.ByteOrder; + +/** +*/ +public class DoubleGenericColumnSupplier implements Supplier +{ + private final CompressedDoublesIndexedSupplier column; + private final ByteOrder byteOrder; + + public DoubleGenericColumnSupplier( + CompressedDoublesIndexedSupplier column, + ByteOrder byteOrder + ) { + this.column = column; + this.byteOrder = byteOrder; + } + + @Override + public GenericColumn get() + { + return new IndexedDoublesGenericColumn(column.get()); + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/IntGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/IntGenericColumnPartSerde.java new file mode 100644 index 000000000000..d47ec428d620 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/IntGenericColumnPartSerde.java @@ -0,0 +1,84 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.serde; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedIntsIndexedSupplier; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** +*/ +public class IntGenericColumnPartSerde implements ColumnPartSerde +{ + @JsonCreator + public static IntGenericColumnPartSerde createDeserializer( + @JsonProperty("byteOrder") ByteOrder byteOrder + ) + { + return new IntGenericColumnPartSerde(null, byteOrder); + } + + private final CompressedIntsIndexedSupplier compressedInts; + private final ByteOrder byteOrder; + + public IntGenericColumnPartSerde(CompressedIntsIndexedSupplier compressedInts, ByteOrder byteOrder) + { + this.compressedInts = compressedInts; + this.byteOrder = byteOrder; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @Override + public long numBytes() + { + return compressedInts.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + compressedInts.writeToChannel(channel); + } + + @Override + public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final CompressedIntsIndexedSupplier column = CompressedIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder); + + builder.setType(ValueType.LONG) + .setHasMultipleValues(false) + .setGenericColumn(new IntGenericColumnSupplier(column)); + + return new IntGenericColumnPartSerde(column, byteOrder); + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/IntGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/IntGenericColumnSupplier.java new file mode 100644 index 000000000000..3a04d65c421b --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/IntGenericColumnSupplier.java @@ -0,0 +1,44 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.serde; + +import com.google.common.base.Supplier; +import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedIntsGenericColumn; +import io.druid.segment.data.CompressedIntsIndexedSupplier; + +/** +*/ +public class IntGenericColumnSupplier implements Supplier +{ + private final CompressedIntsIndexedSupplier column; + + public IntGenericColumnSupplier( + CompressedIntsIndexedSupplier column + ) { + this.column = column; + } + + @Override + public GenericColumn get() + { + return new IndexedIntsGenericColumn(column.get()); + } +} diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index df66e3d88777..537f20c375db 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -157,7 +157,10 @@ public TableDataSource apply(@Nullable String input) public static ArithmeticPostAggregator hyperUniqueFinalizingPostAgg = new ArithmeticPostAggregator( hyperUniqueFinalizingPostAggMetric, "+", - Lists.newArrayList(new HyperUniqueFinalizingPostAggregator(uniqueMetric, uniqueMetric), new ConstantPostAggregator(null, 1)) + Lists.newArrayList( + new HyperUniqueFinalizingPostAggregator(uniqueMetric, uniqueMetric), + new ConstantPostAggregator(null, 1) + ) ); public static final List commonAggregators = Arrays.asList( @@ -271,9 +274,12 @@ public static Collection makeUnionQueryRunners( } ); } + /** * Iterate through the iterables in a synchronous manner and return each step as an Object[] + * * @param in The iterables to step through. (effectively columns) + * * @return An iterable of Object[] containing the "rows" of the input (effectively rows) */ public static Iterable transformToConstructionFeeder(Iterable... in) @@ -345,7 +351,7 @@ public static > QueryRunner makeQueryRunner( segmentId, adapter.getDataInterval().getStart(), factory.createRunner(adapter) ), - (QueryToolChest>)factory.getToolchest() + (QueryToolChest>) factory.getToolchest() ); } @@ -372,11 +378,16 @@ public static QueryRunner makeUnionQueryRunner( public static IntervalChunkingQueryRunnerDecorator NoopIntervalChunkingQueryRunnerDecorator() { - return new IntervalChunkingQueryRunnerDecorator(null, null, null) { + return new IntervalChunkingQueryRunnerDecorator(null, null, null) + { @Override - public QueryRunner decorate(final QueryRunner delegate, - QueryToolChest> toolChest) { - return new QueryRunner() { + public QueryRunner decorate( + final QueryRunner delegate, + QueryToolChest> toolChest + ) + { + return new QueryRunner() + { @Override public Sequence run(Query query, Map responseContext) { diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java index b1b52ed6acfc..aa0d9032891e 100644 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java @@ -35,9 +35,9 @@ public class DoubleMaxAggregationTest { private DoubleMaxAggregatorFactory doubleMaxAggFactory; private ColumnSelectorFactory colSelectorFactory; - private TestFloatColumnSelector selector; + private TestDoubleColumnSelector selector; - private float[] values = {1.1f, 2.7f, 3.5f, 1.3f}; + private double[] values = {1.1d, 2.7d, 3.5d, 1.3d}; public DoubleMaxAggregationTest() throws Exception { @@ -48,9 +48,9 @@ public DoubleMaxAggregationTest() throws Exception @Before public void setup() { - selector = new TestFloatColumnSelector(values); + selector = new TestDoubleColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); - EasyMock.expect(colSelectorFactory.makeFloatColumnSelector("nilly")).andReturn(selector); + EasyMock.expect(colSelectorFactory.makeDoubleColumnSelector("nilly")).andReturn(selector); EasyMock.replay(colSelectorFactory); } @@ -68,7 +68,7 @@ public void testDoubleMaxAggregator() Assert.assertEquals(values[2], ((Double) agg.get()).doubleValue(), 0.0001); Assert.assertEquals((long)values[2], agg.getLong()); - Assert.assertEquals(values[2], agg.getFloat(), 0.0001); + Assert.assertEquals(values[2], agg.getDouble(), 0.0001); agg.reset(); Assert.assertEquals(Double.NEGATIVE_INFINITY, (Double) agg.get(), 0.0001); @@ -89,7 +89,7 @@ public void testDoubleMaxBufferAggregator() Assert.assertEquals(values[2], ((Double) agg.get(buffer, 0)).doubleValue(), 0.0001); Assert.assertEquals((long) values[2], agg.getLong(buffer, 0)); - Assert.assertEquals(values[2], agg.getFloat(buffer, 0), 0.0001); + Assert.assertEquals(values[2], agg.getDouble(buffer, 0), 0.0001); } @Test @@ -111,13 +111,13 @@ public void testEqualsAndHashCode() throws Exception Assert.assertFalse(one.equals(two)); } - private void aggregate(TestFloatColumnSelector selector, DoubleMaxAggregator agg) + private void aggregate(TestDoubleColumnSelector selector, DoubleMaxAggregator agg) { agg.aggregate(); selector.increment(); } - private void aggregate(TestFloatColumnSelector selector, DoubleMaxBufferAggregator agg, ByteBuffer buff, int position) + private void aggregate(TestDoubleColumnSelector selector, DoubleMaxBufferAggregator agg, ByteBuffer buff, int position) { agg.aggregate(buff, position); selector.increment(); diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java index 7b89d88801ee..5ef08b53647b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java @@ -35,9 +35,9 @@ public class DoubleMinAggregationTest { private DoubleMinAggregatorFactory doubleMinAggFactory; private ColumnSelectorFactory colSelectorFactory; - private TestFloatColumnSelector selector; + private TestDoubleColumnSelector selector; - private float[] values = {3.5f, 2.7f, 1.1f, 1.3f}; + private double[] values = {3.5d, 2.7d, 1.1d, 1.3d}; public DoubleMinAggregationTest() throws Exception { @@ -48,9 +48,9 @@ public DoubleMinAggregationTest() throws Exception @Before public void setup() { - selector = new TestFloatColumnSelector(values); + selector = new TestDoubleColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); - EasyMock.expect(colSelectorFactory.makeFloatColumnSelector("nilly")).andReturn(selector); + EasyMock.expect(colSelectorFactory.makeDoubleColumnSelector("nilly")).andReturn(selector); EasyMock.replay(colSelectorFactory); } @@ -68,7 +68,7 @@ public void testDoubleMinAggregator() Assert.assertEquals(values[2], ((Double) agg.get()).doubleValue(), 0.0001); Assert.assertEquals((long)values[2], agg.getLong()); - Assert.assertEquals(values[2], agg.getFloat(), 0.0001); + Assert.assertEquals(values[2], agg.getDouble(), 0.0001); agg.reset(); Assert.assertEquals(Double.POSITIVE_INFINITY, (Double) agg.get(), 0.0001); @@ -89,7 +89,7 @@ public void testDoubleMinBufferAggregator() Assert.assertEquals(values[2], ((Double) agg.get(buffer, 0)).doubleValue(), 0.0001); Assert.assertEquals((long) values[2], agg.getLong(buffer, 0)); - Assert.assertEquals(values[2], agg.getFloat(buffer, 0), 0.0001); + Assert.assertEquals(values[2], agg.getDouble(buffer, 0), 0.0001); } @Test @@ -111,13 +111,13 @@ public void testEqualsAndHashCode() throws Exception Assert.assertFalse(one.equals(two)); } - private void aggregate(TestFloatColumnSelector selector, DoubleMinAggregator agg) + private void aggregate(TestDoubleColumnSelector selector, DoubleMinAggregator agg) { agg.aggregate(); selector.increment(); } - private void aggregate(TestFloatColumnSelector selector, DoubleMinBufferAggregator agg, ByteBuffer buff, int position) + private void aggregate(TestDoubleColumnSelector selector, DoubleMinBufferAggregator agg, ByteBuffer buff, int position) { agg.aggregate(buff, position); selector.increment(); diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleSumAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleSumAggregatorTest.java index 22f7d46ecf67..62d912791cad 100644 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleSumAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleSumAggregatorTest.java @@ -28,7 +28,7 @@ */ public class DoubleSumAggregatorTest { - private void aggregate(TestFloatColumnSelector selector, DoubleSumAggregator agg) + private void aggregate(TestDoubleColumnSelector selector, DoubleSumAggregator agg) { agg.aggregate(); selector.increment(); @@ -37,14 +37,14 @@ private void aggregate(TestFloatColumnSelector selector, DoubleSumAggregator agg @Test public void testAggregate() { - final float[] values = {0.15f, 0.27f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); + final double[] values = {0.15d, 0.27d}; + final TestDoubleColumnSelector selector = new TestDoubleColumnSelector(values); DoubleSumAggregator agg = new DoubleSumAggregator("billy", selector); Assert.assertEquals("billy", agg.getName()); - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; + double expectedFirst = values[0]; + double expectedSecond = values[1] + expectedFirst; Assert.assertEquals(0.0d, agg.get()); Assert.assertEquals(0.0d, agg.get()); @@ -62,7 +62,7 @@ public void testAggregate() @Test public void testComparator() { - final TestFloatColumnSelector selector = new TestFloatColumnSelector(new float[]{0.15f, 0.27f}); + final TestDoubleColumnSelector selector = new TestDoubleColumnSelector(new double[]{0.15d, 0.27d}); DoubleSumAggregator agg = new DoubleSumAggregator("billy", selector); Assert.assertEquals("billy", agg.getName()); diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index 7f41e2be25c4..d73575ddfe73 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.Lists; import io.druid.query.dimension.DimensionSpec; -import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.DimFilter; import io.druid.query.filter.NotDimFilter; @@ -29,7 +28,9 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; +import io.druid.segment.IntColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.data.ArrayBasedIndexedInts; @@ -52,32 +53,31 @@ public void testAggregate() final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), + new FloatSumAggregatorFactory("billy", "value"), new SelectorDimFilter("dim", "a") ); FilteredAggregator agg = (FilteredAggregator) factory.factorize( - makeColumnSelector(selector) + makeColumnSelector(selector) ); Assert.assertEquals("billy", agg.getName()); - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond; + float expectedFirst = values[0]; + float expectedSecond = values[1] + expectedFirst; + float expectedThird = expectedSecond; assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); } - private ColumnSelectorFactory makeColumnSelector(final TestFloatColumnSelector selector){ - + private ColumnSelectorFactory makeColumnSelector(final TestFloatColumnSelector selector) + { return new ColumnSelectorFactory() { @Override public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) { final String dimensionName = dimensionSpec.getDimension(); - final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); if (dimensionName.equals("dim")) { return dimensionSpec.decorate( @@ -147,6 +147,18 @@ public FloatColumnSelector makeFloatColumnSelector(String columnName) } } + @Override + public IntColumnSelector makeIntColumnSelector(String columnName) + { + throw new UnsupportedOperationException(); + } + + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + throw new UnsupportedOperationException(); + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String columnName) { @@ -155,11 +167,12 @@ public ObjectColumnSelector makeObjectColumnSelector(String columnName) }; } - private void assertValues(FilteredAggregator agg,TestFloatColumnSelector selector, double... expectedVals){ - Assert.assertEquals(0.0d, agg.get()); - Assert.assertEquals(0.0d, agg.get()); - Assert.assertEquals(0.0d, agg.get()); - for(double expectedVal : expectedVals){ + private void assertValues(FilteredAggregator agg, TestFloatColumnSelector selector, float... expectedVals) + { + Assert.assertEquals(0.0f, agg.get()); + Assert.assertEquals(0.0f, agg.get()); + Assert.assertEquals(0.0f, agg.get()); + for (float expectedVal : expectedVals) { aggregate(selector, agg); Assert.assertEquals(expectedVal, agg.get()); Assert.assertEquals(expectedVal, agg.get()); @@ -174,7 +187,7 @@ public void testAggregateWithNotFilter() final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), + new FloatSumAggregatorFactory("billy", "value"), new NotDimFilter(new SelectorDimFilter("dim", "b")) ); @@ -184,9 +197,9 @@ public void testAggregateWithNotFilter() Assert.assertEquals("billy", agg.getName()); - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond; + float expectedFirst = values[0]; + float expectedSecond = values[1] + expectedFirst; + float expectedThird = expectedSecond; assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); } @@ -197,8 +210,11 @@ public void testAggregateWithOrFilter() final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new OrDimFilter(Lists.newArrayList(new SelectorDimFilter("dim", "a"), new SelectorDimFilter("dim", "b"))) + new FloatSumAggregatorFactory("billy", "value"), + new OrDimFilter(Lists.newArrayList( + new SelectorDimFilter("dim", "a"), + new SelectorDimFilter("dim", "b") + )) ); FilteredAggregator agg = (FilteredAggregator) factory.factorize( @@ -207,9 +223,9 @@ public void testAggregateWithOrFilter() Assert.assertEquals("billy", agg.getName()); - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond + new Float(values[2]).doubleValue(); + float expectedFirst = values[0]; + float expectedSecond = values[1] + expectedFirst; + float expectedThird = expectedSecond + new Float(values[2]).floatValue(); assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); } @@ -220,8 +236,12 @@ public void testAggregateWithAndFilter() final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); FilteredAggregatorFactory factory = new FilteredAggregatorFactory( - new DoubleSumAggregatorFactory("billy", "value"), - new AndDimFilter(Lists.newArrayList(new NotDimFilter(new SelectorDimFilter("dim", "b")), new SelectorDimFilter("dim", "a")))); + new FloatSumAggregatorFactory("billy", "value"), + new AndDimFilter(Lists.newArrayList( + new NotDimFilter(new SelectorDimFilter("dim", "b")), + new SelectorDimFilter("dim", "a") + )) + ); FilteredAggregator agg = (FilteredAggregator) factory.factorize( makeColumnSelector(selector) @@ -229,9 +249,9 @@ public void testAggregateWithAndFilter() Assert.assertEquals("billy", agg.getName()); - double expectedFirst = new Float(values[0]).doubleValue(); - double expectedSecond = new Float(values[1]).doubleValue() + expectedFirst; - double expectedThird = expectedSecond; + float expectedFirst = values[0]; + float expectedSecond = values[1] + expectedFirst; + float expectedThird = expectedSecond; assertValues(agg, selector, expectedFirst, expectedSecond, expectedThird); } diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java index 86ff345d1b00..0b0d74f0c3a2 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java @@ -37,7 +37,7 @@ public class JavaScriptAggregatorBenchmark extends SimpleBenchmark scriptDoubleSum.put("fnCombine", "function combine(a,b) { return a + b }"); } - private static void aggregate(TestFloatColumnSelector selector, Aggregator agg) + private static void aggregate(TestDoubleColumnSelector selector, Aggregator agg) { agg.aggregate(); selector.increment(); @@ -45,7 +45,7 @@ private static void aggregate(TestFloatColumnSelector selector, Aggregator agg) private JavaScriptAggregator jsAggregator; private DoubleSumAggregator doubleAgg; - final LoopingFloatColumnSelector selector = new LoopingFloatColumnSelector(new float[]{42.12f, 9f}); + final LoopingDoubleColumnSelector selector = new LoopingDoubleColumnSelector(new double[]{42.12d, 9d}); @Override protected void setUp() throws Exception @@ -88,21 +88,21 @@ public static void main(String[] args) throws Exception Runner.main(JavaScriptAggregatorBenchmark.class, args); } - protected static class LoopingFloatColumnSelector extends TestFloatColumnSelector + protected static class LoopingDoubleColumnSelector extends TestDoubleColumnSelector { - private final float[] floats; + private final double[] doubles; private long index = 0; - public LoopingFloatColumnSelector(float[] floats) + public LoopingDoubleColumnSelector(double[] doubles) { - super(floats); - this.floats = floats; + super(doubles); + this.doubles = doubles; } @Override - public float get() + public double get() { - return floats[(int) (index % floats.length)]; + return doubles[(int) (index % doubles.length)]; } public void increment() diff --git a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java index d3eb9cb134ce..d4a9ff3ee2bb 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java @@ -45,15 +45,15 @@ public class JavaScriptAggregatorTest scriptDoubleSum.put("fnCombine", "function combine(a,b) { return a + b }"); } - private static void aggregate(TestFloatColumnSelector selector1, TestFloatColumnSelector selector2, Aggregator agg) + private static void aggregate(TestDoubleColumnSelector selector1, TestDoubleColumnSelector selector2, Aggregator agg) { agg.aggregate(); selector1.increment(); selector2.increment(); } - private void aggregateBuffer(TestFloatColumnSelector selector1, - TestFloatColumnSelector selector2, + private void aggregateBuffer(TestDoubleColumnSelector selector1, + TestDoubleColumnSelector selector2, BufferAggregator agg, ByteBuffer buf, int position) { @@ -62,7 +62,7 @@ private void aggregateBuffer(TestFloatColumnSelector selector1, selector2.increment(); } - private static void aggregate(TestFloatColumnSelector selector, Aggregator agg) + private static void aggregate(TestDoubleColumnSelector selector, Aggregator agg) { agg.aggregate(); selector.increment(); @@ -77,8 +77,8 @@ private static void aggregate(TestObjectColumnSelector selector, Aggregator agg) @Test public void testAggregate() { - final TestFloatColumnSelector selector1 = new TestFloatColumnSelector(new float[]{42.12f, 9f}); - final TestFloatColumnSelector selector2 = new TestFloatColumnSelector(new float[]{2f, 3f}); + final TestDoubleColumnSelector selector1 = new TestDoubleColumnSelector(new double[]{42.12d, 9d}); + final TestDoubleColumnSelector selector2 = new TestDoubleColumnSelector(new double[]{2d, 3d}); Map script = sumLogATimesBPlusTen; @@ -100,13 +100,13 @@ public void testAggregate() Assert.assertEquals(val, agg.get()); aggregate(selector1, selector2, agg); - val += Math.log(42.12f) * 2f; + val += Math.log(42.12d) * 2d; Assert.assertEquals(val, agg.get()); Assert.assertEquals(val, agg.get()); Assert.assertEquals(val, agg.get()); aggregate(selector1, selector2, agg); - val += Math.log(9f) * 3f; + val += Math.log(9d) * 3d; Assert.assertEquals(val, agg.get()); Assert.assertEquals(val, agg.get()); Assert.assertEquals(val, agg.get()); @@ -115,8 +115,8 @@ public void testAggregate() @Test public void testBufferAggregate() { - final TestFloatColumnSelector selector1 = new TestFloatColumnSelector(new float[]{42.12f, 9f}); - final TestFloatColumnSelector selector2 = new TestFloatColumnSelector(new float[]{2f, 3f}); + final TestDoubleColumnSelector selector1 = new TestDoubleColumnSelector(new double[]{42.12d, 9d}); + final TestDoubleColumnSelector selector2 = new TestDoubleColumnSelector(new double[]{2d, 3d}); Map script = sumLogATimesBPlusTen; JavaScriptBufferAggregator agg = new JavaScriptBufferAggregator( @@ -136,13 +136,13 @@ public void testBufferAggregate() Assert.assertEquals(val, agg.get(buf, position)); aggregateBuffer(selector1, selector2, agg, buf, position); - val += Math.log(42.12f) * 2f; + val += Math.log(42.12d) * 2d; Assert.assertEquals(val, agg.get(buf, position)); Assert.assertEquals(val, agg.get(buf, position)); Assert.assertEquals(val, agg.get(buf, position)); aggregateBuffer(selector1, selector2, agg, buf, position); - val += Math.log(9f) * 3f; + val += Math.log(9d) * 3d; Assert.assertEquals(val, agg.get(buf, position)); Assert.assertEquals(val, agg.get(buf, position)); Assert.assertEquals(val, agg.get(buf, position)); @@ -223,7 +223,7 @@ public void testAggregateStrings() } public static void main(String... args) throws Exception { - final JavaScriptAggregatorBenchmark.LoopingFloatColumnSelector selector = new JavaScriptAggregatorBenchmark.LoopingFloatColumnSelector(new float[]{42.12f, 9f}); + final JavaScriptAggregatorBenchmark.LoopingDoubleColumnSelector selector = new JavaScriptAggregatorBenchmark.LoopingDoubleColumnSelector(new double[]{42.12d, 9d}); /* memory usage test List aggs = Lists.newLinkedList(); diff --git a/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java b/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java index c838b61b12a8..1ee8bad17891 100644 --- a/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java +++ b/processing/src/test/java/io/druid/query/aggregation/MetricSelectorUtils.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -41,4 +42,22 @@ public Float get() } }; } + + public static ObjectColumnSelector wrap(final DoubleColumnSelector selector) + { + return new ObjectColumnSelector() + { + @Override + public Class classOfObject() + { + return Double.TYPE; + } + + @Override + public Double get() + { + return selector.get(); + } + }; + } } diff --git a/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelector.java new file mode 100644 index 000000000000..b719aac680d0 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelector.java @@ -0,0 +1,47 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation; + +import io.druid.segment.DoubleColumnSelector; + +/** + */ +public class TestDoubleColumnSelector implements DoubleColumnSelector +{ + private final double[] doubles; + + private int index = 0; + + public TestDoubleColumnSelector(double[] doubles) + { + this.doubles = doubles; + } + + @Override + public double get() + { + return doubles[index]; + } + + public void increment() + { + ++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 5d10abf97696..9dc0e50657cc 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -1457,7 +1457,7 @@ public void testGroupByWithOrderLimit3() throws Exception "rows", 6L, "idx", - 4423.6533203125D + 4423.653350830078d ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", @@ -1466,7 +1466,7 @@ public void testGroupByWithOrderLimit3() throws Exception "rows", 6L, "idx", - 4418.61865234375D + 4418.618499755859d ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", @@ -1475,7 +1475,7 @@ public void testGroupByWithOrderLimit3() throws Exception "rows", 2L, "idx", - 319.94403076171875D + 319.9440155029297d ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", @@ -1484,7 +1484,7 @@ public void testGroupByWithOrderLimit3() throws Exception "rows", 2L, "idx", - 270.3977966308594D + 270.39778900146484d ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", @@ -1502,7 +1502,7 @@ public void testGroupByWithOrderLimit3() throws Exception "rows", 2L, "idx", - 222.20980834960938D + 222.2098159790039D ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", @@ -1520,7 +1520,7 @@ public void testGroupByWithOrderLimit3() throws Exception "rows", 2L, "idx", - 216.97836303710938D + 216.97835540771484D ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-04-01", @@ -1815,7 +1815,7 @@ public void testGroupByWithOrderLimitHavingSpec() "rows", 3L, "index", - 312.38165283203125 + 312.3816375732422 ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-01-25", @@ -1824,7 +1824,7 @@ public void testGroupByWithOrderLimitHavingSpec() "rows", 3L, "index", - 312.7834167480469 + 312.78340911865234 ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-01-25", @@ -1833,7 +1833,7 @@ public void testGroupByWithOrderLimitHavingSpec() "rows", 3L, "index", - 324.6412353515625 + 324.64124298095703 ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-01-25", @@ -1842,7 +1842,7 @@ public void testGroupByWithOrderLimitHavingSpec() "rows", 3L, "index", - 393.36322021484375 + 393.3632049560547 ), GroupByQueryRunnerTestHelper.createExpectedRow( "2011-01-25", @@ -1851,7 +1851,7 @@ public void testGroupByWithOrderLimitHavingSpec() "rows", 3L, "index", - 511.2996826171875 + 511.29969787597656 ) ); @@ -3645,11 +3645,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 13219.574157714844, + 13219.574020385742, "rows", 117L, "addRowsIndexConstant", - 13337.574157714844 + 13337.574020385742 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3658,11 +3658,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 13557.738830566406, + 13557.73889541626, "rows", 117L, "addRowsIndexConstant", - 13675.738830566406 + 13675.73889541626 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3671,11 +3671,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 13493.751281738281, + 13493.751190185547, "rows", 117L, "addRowsIndexConstant", - 13611.751281738281 + 13611.751190185547 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3684,11 +3684,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 13585.541015625, + 13585.540908813477, "rows", 117L, "addRowsIndexConstant", - 13703.541015625 + 13703.540908813477 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3697,11 +3697,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 14279.127197265625, + 14279.127326965332, "rows", 126L, "addRowsIndexConstant", - 14406.127197265625 + 14406.127326965332 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3710,11 +3710,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 13199.471435546875, + 13199.471267700195, "rows", 117L, "addRowsIndexConstant", - 13317.471435546875 + 13317.471267700195 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3723,11 +3723,11 @@ public void testGroupByTimeExtraction() "market", "spot", "index", - 14271.368591308594, + 14271.368713378906, "rows", 126L, "addRowsIndexConstant", - 14398.368591308594 + 14398.368713378906 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3736,11 +3736,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 27297.8623046875, + 27297.862365722656, "rows", 26L, "addRowsIndexConstant", - 27324.8623046875 + 27324.862365722656 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3749,11 +3749,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 27619.58447265625, + 27619.58477783203, "rows", 26L, "addRowsIndexConstant", - 27646.58447265625 + 27646.58477783203 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3762,11 +3762,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 27820.83154296875, + 27820.831176757812, "rows", 26L, "addRowsIndexConstant", - 27847.83154296875 + 27847.831176757812 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3775,11 +3775,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 24791.223876953125, + 24791.22381591797, "rows", 26L, "addRowsIndexConstant", - 24818.223876953125 + 24818.22381591797 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3788,11 +3788,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 28562.748901367188, + 28562.748779296875, "rows", 28L, "addRowsIndexConstant", - 28591.748901367188 + 28591.748779296875 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3801,11 +3801,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 26968.280639648438, + 26968.28009033203, "rows", 26L, "addRowsIndexConstant", - 26995.280639648438 + 26995.28009033203 ), GroupByQueryRunnerTestHelper.createExpectedRow( "1970-01-01", @@ -3814,11 +3814,11 @@ public void testGroupByTimeExtraction() "market", "upfront", "index", - 28985.5751953125, + 28985.57501220703, "rows", 28L, "addRowsIndexConstant", - 29014.5751953125 + 29014.57501220703 ) ); diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index bffe3a755f21..d09df8e604a4 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -33,6 +33,7 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.FloatSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.incremental.IncrementalIndex; @@ -75,7 +76,7 @@ public class TestIndex private static final Logger log = new Logger(TestIndex.class); private static final Interval DATA_INTERVAL = new Interval("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"); private static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ - new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]), + new FloatSumAggregatorFactory(METRICS[0], METRICS[0]), new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; private static final IndexSpec indexSpec = new IndexSpec(); diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index 28045de3eec6..848a87450a33 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -70,26 +70,7 @@ public IncrementalIndex createIndex() ); } } - - }, - { - new IndexCreator() - { - @Override - public IncrementalIndex createIndex() - { - return new OffheapIncrementalIndex( - 0, - QueryGranularity.MINUTE, - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - TestQueryRunners.pool, - true, - 100 * 1024 * 1024 - ); - } - } } - } ); } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java index 9ddfabaf961f..0293c533967f 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/WikipediaIrcDecoder.java @@ -100,38 +100,45 @@ public WikipediaIrcDecoder( } } - private DatabaseReader openDefaultGeoIpDb() { - File geoDb = new File(System.getProperty("java.io.tmpdir"), - this.getClass().getCanonicalName() + ".GeoLite2-City.mmdb"); + private DatabaseReader openDefaultGeoIpDb() + { + File geoDb = new File( + System.getProperty("java.io.tmpdir"), + this.getClass().getCanonicalName() + ".GeoLite2-City.mmdb" + ); try { return openDefaultGeoIpDb(geoDb); } catch (RuntimeException e) { - log.warn(e.getMessage()+" Attempting to re-download.", e); + log.warn(e.getMessage() + " Attempting to re-download.", e); if (geoDb.exists() && !geoDb.delete()) { - throw new RuntimeException("Could not delete geo db file ["+ geoDb.getAbsolutePath() +"]."); + throw new RuntimeException("Could not delete geo db file [" + geoDb.getAbsolutePath() + "]."); } // local download may be corrupt, will retry once. return openDefaultGeoIpDb(geoDb); } } - private DatabaseReader openDefaultGeoIpDb(File geoDb) { + private DatabaseReader openDefaultGeoIpDb(File geoDb) + { downloadGeoLiteDbToFile(geoDb); return openGeoIpDb(geoDb); } - private DatabaseReader openGeoIpDb(File geoDb) { + private DatabaseReader openGeoIpDb(File geoDb) + { try { DatabaseReader reader = new DatabaseReader(geoDb); log.info("Using geo ip database at [%s].", geoDb); return reader; - } catch (IOException e) { - throw new RuntimeException("Could not open geo db at ["+ geoDb.getAbsolutePath() +"].", e); + } + catch (IOException e) { + throw new RuntimeException("Could not open geo db at [" + geoDb.getAbsolutePath() + "].", e); } } - private void downloadGeoLiteDbToFile(File geoDb) { + private void downloadGeoLiteDbToFile(File geoDb) + { if (geoDb.exists()) { return; } @@ -142,14 +149,14 @@ private void downloadGeoLiteDbToFile(File geoDb) { File tmpFile = File.createTempFile("druid", "geo"); FileUtils.copyInputStreamToFile( - new GZIPInputStream( - new URL("http://geolite.maxmind.com/download/geoip/database/GeoLite2-City.mmdb.gz").openStream() - ), - tmpFile + new GZIPInputStream( + new URL("http://geolite.maxmind.com/download/geoip/database/GeoLite2-City.mmdb.gz").openStream() + ), + tmpFile ); if (!tmpFile.renameTo(geoDb)) { - throw new RuntimeException("Unable to move geo file to ["+geoDb.getAbsolutePath()+"]!"); + throw new RuntimeException("Unable to move geo file to [" + geoDb.getAbsolutePath() + "]!"); } } catch (IOException e) { @@ -289,6 +296,18 @@ public long getLongMetric(String metric) return new Float(metrics.get(metric)).longValue(); } + @Override + public int getIntMetric(String metric) + { + return new Float(metrics.get(metric)).intValue(); + } + + @Override + public double getDoubleMetric(String metric) + { + return new Float(metrics.get(metric)).doubleValue(); + } + @Override public int compareTo(Row o) { diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java index 36da1c4a8388..81277cefb57d 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -101,6 +101,18 @@ public long getLongMetric(String metric) return new Float(metricValue).longValue(); } + @Override + public int getIntMetric(String s) + { + return new Float(metricValue).intValue(); + } + + @Override + public double getDoubleMetric(String s) + { + return new Float(metricValue).doubleValue(); + } + @Override public Object getRaw(String dimension) { 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 f078d5ae2c41..a8bd61051cc4 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -285,6 +285,18 @@ public long getLongMetric(String metric) return 0L; } + @Override + public int getIntMetric(String s) + { + return 0; + } + + @Override + public double getDoubleMetric(String s) + { + return 0; + } + @Override public Object getRaw(String dimension) { 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 c20451f6236a..7a522967a785 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 @@ -470,6 +470,18 @@ public long getLongMetric(String metric) return 0L; } + @Override + public int getIntMetric(String s) + { + return 0; + } + + @Override + public double getDoubleMetric(String s) + { + return 0; + } + @Override public Object getRaw(String dimension) { diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 9a5664c2b9e3..e347c50afd9d 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -109,6 +109,18 @@ public long getLongMetric(String metric) return 0L; } + @Override + public int getIntMetric(String s) + { + return 0; + } + + @Override + public double getDoubleMetric(String s) + { + return 0; + } + @Override public Object getRaw(String dimension) { @@ -168,6 +180,18 @@ public long getLongMetric(String metric) return 0L; } + @Override + public int getIntMetric(String s) + { + return 0; + } + + @Override + public double getDoubleMetric(String s) + { + return 0; + } + @Override public Object getRaw(String dimension) { diff --git a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java index 2b15b58112a9..028c41fb0601 100644 --- a/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/server/shard/HashBasedNumberedShardSpecTest.java @@ -208,6 +208,18 @@ public long getLongMetric(String s) return 0L; } + @Override + public int getIntMetric(String s) + { + return 0; + } + + @Override + public double getDoubleMetric(String s) + { + return 0d; + } + @Override public int compareTo(Row o) { From 9d1cb539b18617a757172132645b4913054b1eef Mon Sep 17 00:00:00 2001 From: fjy Date: Tue, 12 Jan 2016 17:39:35 -0800 Subject: [PATCH 2/2] remove offhepa again --- .../incremental/OffheapIncrementalIndex.java | 486 ------------------ 1 file changed, 486 deletions(-) delete mode 100644 processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java deleted file mode 100644 index ad24713e923f..000000000000 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ /dev/null @@ -1,486 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package io.druid.segment.incremental; - -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.metamx.common.ISE; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidPool; -import io.druid.data.input.InputRow; -import io.druid.granularity.QueryGranularity; -import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.BufferAggregator; -import org.mapdb.BTreeKeySerializer; -import org.mapdb.DB; -import org.mapdb.DBMaker; -import org.mapdb.Serializer; -import org.mapdb.Store; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.Serializable; -import java.lang.ref.WeakReference; -import java.lang.reflect.Field; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; -import java.util.UUID; -import java.util.WeakHashMap; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.atomic.AtomicInteger; - -@Deprecated -/** - * This is not yet ready for production use and requires more work. - */ -public class OffheapIncrementalIndex extends IncrementalIndex -{ - private static final long STORE_CHUNK_SIZE; - - static - { - // MapDB allocated memory in chunks. We need to know CHUNK_SIZE - // in order to get a crude estimate of how much more direct memory - // might be used when adding an additional row. - try { - Field field = Store.class.getDeclaredField("CHUNK_SIZE"); - field.setAccessible(true); - STORE_CHUNK_SIZE = field.getLong(null); - } catch(NoSuchFieldException | IllegalAccessException e) { - throw new RuntimeException("Unable to determine MapDB store chunk size", e); - } - } - - private final ResourceHolder bufferHolder; - - private final DB db; - private final DB factsDb; - private final int[] aggPositionOffsets; - private final int totalAggSize; - private final ConcurrentNavigableMap facts; - private final int maxTotalBufferSize; - - private String outOfRowsReason = null; - - public OffheapIncrementalIndex( - IncrementalIndexSchema incrementalIndexSchema, - StupidPool bufferPool, - boolean deserializeComplexMetrics, - int maxTotalBufferSize - ) - { - super(incrementalIndexSchema, deserializeComplexMetrics); - - this.bufferHolder = bufferPool.take(); - Preconditions.checkArgument( - maxTotalBufferSize > bufferHolder.get().limit(), - "Maximum total buffer size must be greater than aggregation buffer size" - ); - - final AggregatorFactory[] metrics = incrementalIndexSchema.getMetrics(); - this.aggPositionOffsets = new int[metrics.length]; - - int currAggSize = 0; - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - aggPositionOffsets[i] = currAggSize; - currAggSize += agg.getMaxIntermediateSize(); - } - this.totalAggSize = currAggSize; - - final DBMaker dbMaker = DBMaker.newMemoryDirectDB() - .transactionDisable() - .asyncWriteEnable() - .cacheLRUEnable() - .cacheSize(16384); - - this.factsDb = dbMaker.make(); - this.db = dbMaker.make(); - final TimeAndDimsSerializer timeAndDimsSerializer = new TimeAndDimsSerializer(this); - this.facts = factsDb.createTreeMap("__facts" + UUID.randomUUID()) - .keySerializer(timeAndDimsSerializer) - .comparator(timeAndDimsSerializer.getComparator()) - .valueSerializer(Serializer.INTEGER) - .make(); - this.maxTotalBufferSize = maxTotalBufferSize; - } - - public OffheapIncrementalIndex( - long minTimestamp, - QueryGranularity gran, - final AggregatorFactory[] metrics, - StupidPool bufferPool, - boolean deserializeComplexMetrics, - int maxTotalBufferSize - ) - { - this( - new IncrementalIndexSchema.Builder().withMinTimestamp(minTimestamp) - .withQueryGranularity(gran) - .withMetrics(metrics) - .build(), - bufferPool, - deserializeComplexMetrics, - maxTotalBufferSize - ); - } - - @Override - public ConcurrentNavigableMap getFacts() - { - return facts; - } - - @Override - protected DimDim makeDimDim(String dimension) - { - return new OffheapDimDim(dimension); - } - - @Override - protected BufferAggregator[] initAggs( - AggregatorFactory[] metrics, - Supplier rowSupplier, - boolean deserializeComplexMetrics - ) - { - BufferAggregator[] aggs = new BufferAggregator[metrics.length]; - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - aggs[i] = agg.factorizeBuffered( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) - ); - } - return aggs; - } - - @Override - protected Integer addToFacts( - AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, - InputRow row, - AtomicInteger numEntries, - TimeAndDims key, - ThreadLocal rowContainer, - Supplier rowSupplier - ) throws IndexSizeExceededException - { - final BufferAggregator[] aggs = getAggs(); - Integer rowOffset; - synchronized (this) { - if (!facts.containsKey(key)) { - if (!canAppendRow(false)) { - throw new IndexSizeExceededException("%s", getOutOfRowsReason()); - } - } - rowOffset = totalAggSize * numEntries.get(); - final Integer prev = facts.putIfAbsent(key, rowOffset); - if (prev != null) { - rowOffset = prev; - } else { - numEntries.incrementAndGet(); - for (int i = 0; i < aggs.length; i++) { - aggs[i].init(bufferHolder.get(), getMetricPosition(rowOffset, i)); - } - } - } - rowContainer.set(row); - for (int i = 0; i < aggs.length; i++) { - synchronized (aggs[i]) { - aggs[i].aggregate(bufferHolder.get(), getMetricPosition(rowOffset, i)); - } - } - rowContainer.set(null); - return numEntries.get(); - } - - public boolean canAppendRow() { - return canAppendRow(true); - } - - private boolean canAppendRow(boolean includeFudgeFactor) - { - // there is a race condition when checking current MapDB - // when canAppendRow() is called after adding a row it may return true, but on a subsequence call - // to addToFacts that may not be the case anymore because MapDB size may have changed. - // so we add this fudge factor, hoping that will be enough. - - final int aggBufferSize = bufferHolder.get().limit(); - if ((size() + 1) * totalAggSize > aggBufferSize) { - outOfRowsReason = String.format("Maximum aggregation buffer limit reached [%d bytes].", aggBufferSize); - return false; - } - // hopefully both MapDBs will grow by at most STORE_CHUNK_SIZE each when we add the next row. - if (getCurrentSize() + totalAggSize + 2 * STORE_CHUNK_SIZE + (includeFudgeFactor ? STORE_CHUNK_SIZE : 0) > maxTotalBufferSize) { - outOfRowsReason = String.format("Maximum time and dimension buffer limit reached [%d bytes].", maxTotalBufferSize - aggBufferSize); - return false; - } - return true; - } - - public String getOutOfRowsReason() { - return outOfRowsReason; - } - - @Override - protected BufferAggregator[] getAggsForRow(int rowOffset) - { - return getAggs(); - } - - @Override - protected Object getAggVal(BufferAggregator agg, int rowOffset, int aggPosition) - { - return agg.get(bufferHolder.get(), getMetricPosition(rowOffset, aggPosition)); - } - - @Override - public float getMetricFloatValue(int rowOffset, int aggOffset) - { - return getAggs()[aggOffset].getFloat(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset)); - } - - @Override - public long getMetricLongValue(int rowOffset, int aggOffset) - { - return getAggs()[aggOffset].getLong(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset)); - } - - @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) - { - return getAggs()[aggOffset].get(bufferHolder.get(), getMetricPosition(rowOffset, aggOffset)); - } - - @Override - public void close() - { - try { - bufferHolder.close(); - Store.forDB(db).close(); - Store.forDB(factsDb).close(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - private int getMetricPosition(int rowOffset, int metricIndex) - { - return rowOffset + aggPositionOffsets[metricIndex]; - } - - private DimDim getDimDim(int dimIndex) - { - return getDimensionValues(getDimensionNames().get(dimIndex)); - } - - // MapDB forces serializers to implement serializable, which sucks - private static class TimeAndDimsSerializer extends BTreeKeySerializer implements Serializable - { - private final TimeAndDimsComparator comparator; - private final transient OffheapIncrementalIndex incrementalIndex; - - TimeAndDimsSerializer(OffheapIncrementalIndex incrementalIndex) - { - this.comparator = new TimeAndDimsComparator(); - this.incrementalIndex = incrementalIndex; - } - - @Override - public void serialize(DataOutput out, int start, int end, Object[] keys) throws IOException - { - for (int i = start; i < end; i++) { - TimeAndDims timeAndDim = (TimeAndDims) keys[i]; - out.writeLong(timeAndDim.getTimestamp()); - out.writeInt(timeAndDim.getDims().length); - int index = 0; - for (String[] dims : timeAndDim.getDims()) { - if (dims == null) { - out.write(-1); - } else { - DimDim dimDim = incrementalIndex.getDimDim(index); - out.writeInt(dims.length); - for (String value : dims) { - out.writeInt(dimDim.getId(value)); - } - } - index++; - } - } - } - - @Override - public Object[] deserialize(DataInput in, int start, int end, int size) throws IOException - { - Object[] ret = new Object[size]; - for (int i = start; i < end; i++) { - final long timeStamp = in.readLong(); - final String[][] dims = new String[in.readInt()][]; - for (int k = 0; k < dims.length; k++) { - int len = in.readInt(); - if (len != -1) { - DimDim dimDim = incrementalIndex.getDimDim(k); - String[] col = new String[len]; - for (int l = 0; l < col.length; l++) { - col[l] = dimDim.get(dimDim.getValue(in.readInt())); - } - dims[k] = col; - } - } - ret[i] = new TimeAndDims(timeStamp, dims); - } - return ret; - } - - @Override - public Comparator getComparator() - { - return comparator; - } - } - - private static class TimeAndDimsComparator implements Comparator, Serializable - { - @Override - public int compare(Object o1, Object o2) - { - return ((TimeAndDims) o1).compareTo((TimeAndDims) o2); - } - } - - private class OffheapDimDim implements DimDim - { - private final Map falseIds; - private final Map falseIdsReverse; - private final WeakHashMap> cache = new WeakHashMap(); - - private volatile String[] sortedVals = null; - // size on MapDB is slow so maintain a count here - private volatile int size = 0; - - public OffheapDimDim(String dimension) - { - falseIds = db.createHashMap(dimension) - .keySerializer(Serializer.STRING) - .valueSerializer(Serializer.INTEGER) - .make(); - falseIdsReverse = db.createHashMap(dimension + "_inverse") - .keySerializer(Serializer.INTEGER) - .valueSerializer(Serializer.STRING) - .make(); - } - - /** - * Returns the interned String value to allow fast comparisons using `==` instead of `.equals()` - * - * @see io.druid.segment.incremental.IncrementalIndexStorageAdapter.EntryHolderValueMatcherFactory#makeValueMatcher(String, String) - */ - public String get(String str) - { - final WeakReference cached = cache.get(str); - if (cached != null) { - final String value = cached.get(); - if (value != null) { - return value; - } - } - cache.put(str, new WeakReference(str)); - return str; - } - - public int getId(String value) - { - return falseIds.get(value); - } - - public String getValue(int id) - { - return falseIdsReverse.get(id); - } - - public boolean contains(String value) - { - return falseIds.containsKey(value); - } - - public int size() - { - return size; - } - - public synchronized int add(String value) - { - int id = size++; - falseIds.put(value, id); - falseIdsReverse.put(id, value); - return id; - } - - public int getSortedId(String value) - { - assertSorted(); - return Arrays.binarySearch(sortedVals, value); - } - - public String getSortedValue(int index) - { - assertSorted(); - return sortedVals[index]; - } - - public void sort() - { - if (sortedVals == null) { - sortedVals = new String[falseIds.size()]; - - int index = 0; - for (String value : falseIds.keySet()) { - sortedVals[index++] = value; - } - Arrays.sort(sortedVals); - } - } - - private void assertSorted() - { - if (sortedVals == null) { - throw new ISE("Call sort() before calling the getSorted* methods."); - } - } - - public boolean compareCanonicalValues(String s1, String s2) - { - return s1.equals(s2); - } - } - - private long getCurrentSize() - { - return Store.forDB(db).getCurrSize() + - Store.forDB(factsDb).getCurrSize() - // Size of aggregators - + size() * totalAggSize; - } -}