diff --git a/api/src/main/java/io/druid/data/ValueType.java b/api/src/main/java/io/druid/data/ValueType.java new file mode 100644 index 000000000000..f2fc75bc7fa7 --- /dev/null +++ b/api/src/main/java/io/druid/data/ValueType.java @@ -0,0 +1,152 @@ +/* + * 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.data; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.collect.Ordering; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Floats; +import com.google.common.primitives.Longs; + +import java.util.Comparator; + +/** + */ +public enum ValueType +{ + FLOAT { + @Override + public Class classOfObject() + { + return Float.TYPE; + } + + @Override + public Comparator comparator() + { + return new Comparator() + { + @Override + public int compare(Float o1, Float o2) + { + return Floats.compare(o1, o2); + } + }; + } + }, + LONG { + @Override + public Class classOfObject() + { + return Long.TYPE; + } + + @Override + public Comparator comparator() + { + return new Comparator() + { + @Override + public int compare(Long o1, Long o2) + { + return Longs.compare(o1, o2); + } + }; + } + }, + DOUBLE { + @Override + public Class classOfObject() + { + return Double.TYPE; + } + + @Override + public Comparator comparator() + { + return new Comparator() + { + @Override + public int compare(Double o1, Double o2) + { + return Doubles.compare(o1, o2); + } + }; + } + }, + STRING { + @Override + public Class classOfObject() + { + return String.class; + } + + public Comparator comparator() + { + return Ordering.natural().nullsFirst(); + } + }, + COMPLEX { + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public Comparator comparator() + { + throw new UnsupportedOperationException(); + } + }; + + public abstract Class classOfObject(); + + public abstract Comparator comparator(); + + @JsonValue + @Override + public String toString() + { + return this.name().toUpperCase(); + } + + @JsonCreator + public static ValueType fromString(String name) + { + return valueOf(name.toUpperCase()); + } + + public static ValueType of(String name) + { + try { + return name == null ? COMPLEX : fromString(name); + } + catch (IllegalArgumentException e) { + return COMPLEX; + } + } + + public static boolean isNumeric(ValueType type) + { + return type == DOUBLE || type == FLOAT || type == LONG; + } +} diff --git a/api/src/main/java/io/druid/data/input/MapBasedRow.java b/api/src/main/java/io/druid/data/input/MapBasedRow.java index 8738428387cc..7c80920d00c2 100644 --- a/api/src/main/java/io/druid/data/input/MapBasedRow.java +++ b/api/src/main/java/io/druid/data/input/MapBasedRow.java @@ -23,12 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.collect.Lists; -import com.metamx.common.IAE; import com.metamx.common.logger.Logger; import com.metamx.common.parsers.ParseException; import org.joda.time.DateTime; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -134,6 +132,29 @@ public float getFloatMetric(String metric) } } + @Override + public double getDoubleMetric(String metric) + { + Object metricValue = event.get(metric); + + if (metricValue == null) { + return 0.0d; + } + + if (metricValue instanceof Number) { + return ((Number) metricValue).doubleValue(); + } else if (metricValue instanceof String) { + try { + return Double.valueOf(((String) metricValue).replace(",", "")); + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", metric, metricValue); + } + } else { + throw new ParseException("Unknown type[%s]", metricValue.getClass()); + } + } + @Override public long getLongMetric(String metric) { diff --git a/api/src/main/java/io/druid/data/input/Row.java b/api/src/main/java/io/druid/data/input/Row.java index 914d8146597c..a49d13017c4c 100644 --- a/api/src/main/java/io/druid/data/input/Row.java +++ b/api/src/main/java/io/druid/data/input/Row.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.metamx.common.parsers.ParseException; import org.joda.time.DateTime; import java.util.List; @@ -82,6 +81,8 @@ public interface Row extends Comparable */ public float getFloatMetric(String metric); + public double getDoubleMetric(String metric); + /** * Returns the long value of the given metric column. *

diff --git a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java index ac674b6282c5..ba2e04bee94c 100644 --- a/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/DimensionSchema.java @@ -19,17 +19,12 @@ package io.druid.data.input.impl; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import com.fasterxml.jackson.annotation.JsonValue; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.base.Preconditions; - -import java.util.List; +import io.druid.data.ValueType; /** */ @@ -47,30 +42,6 @@ public abstract class DimensionSchema public static final String FLOAT_TYPE_NAME = "float"; public static final String SPATIAL_TYPE_NAME = "spatial"; - - // main druid and druid-api should really use the same ValueType enum. - // merge them when druid-api is merged back into the main repo - public enum ValueType - { - FLOAT, - LONG, - STRING, - COMPLEX; - - @JsonValue - @Override - public String toString() - { - return this.name().toUpperCase(); - } - - @JsonCreator - public static ValueType fromString(String name) - { - return valueOf(name.toUpperCase()); - } - } - private final String name; protected DimensionSchema(String name) diff --git a/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java index db3b04a631bd..1068a5ad7c34 100644 --- a/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/FloatDimensionSchema.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.ValueType; public class FloatDimensionSchema extends DimensionSchema { diff --git a/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java index 4fd77d469248..46a072ee752e 100644 --- a/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/LongDimensionSchema.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.ValueType; public class LongDimensionSchema extends DimensionSchema { diff --git a/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java index ae834262bb9e..01379d4c98c3 100644 --- a/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/NewSpatialDimensionSchema.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.ValueType; import java.util.List; diff --git a/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java b/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java index 02fef40dda42..53fca60e2cec 100644 --- a/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java +++ b/api/src/main/java/io/druid/data/input/impl/StringDimensionSchema.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.ValueType; public class StringDimensionSchema extends DimensionSchema { diff --git a/benchmarks/src/main/java/io/druid/benchmark/DoubleCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/DoubleCompressionBenchmark.java new file mode 100644 index 000000000000..a8bb253112e2 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/DoubleCompressionBenchmark.java @@ -0,0 +1,104 @@ +/* + * 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.benchmark; + +// Run DoubleCompressionBenchmarkFileGenerator to generate the required files before running this benchmark + +import com.google.common.base.Supplier; +import com.google.common.io.Files; +import io.druid.segment.data.CompressedDoublesIndexedSupplier; +import io.druid.segment.data.IndexedDoubles; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class DoubleCompressionBenchmark +{ + @Param("doubleCompress/") + private static String dirPath; + + @Param({"enumerate", "zipfLow", "zipfHigh", "sequential", "uniform"}) + private static String file; + + @Param({"lz4", "none"}) + private static String strategy; + + private Random rand; + private Supplier supplier; + + @Setup + public void setup() throws Exception + { + File dir = new File(dirPath); + File compFile = new File(dir, file + "-" + strategy); + rand = new Random(); + ByteBuffer buffer = Files.map(compFile); + supplier = CompressedDoublesIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder()); + } + + @Benchmark + public void readContinuous(Blackhole bh) throws IOException + { + IndexedDoubles indexedDoubles = supplier.get(); + int count = indexedDoubles.size(); + double sum = 0; + for (int i = 0; i < count; i++) { + sum += indexedDoubles.get(i); + } + bh.consume(sum); + indexedDoubles.close(); + } + + @Benchmark + public void readSkipping(Blackhole bh) throws IOException + { + IndexedDoubles indexedDoubles = supplier.get(); + int count = indexedDoubles.size(); + double sum = 0; + for (int i = 0; i < count; i += rand.nextInt(2000)) { + sum += indexedDoubles.get(i); + } + bh.consume(sum); + indexedDoubles.close(); + } + +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/DoubleCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/DoubleCompressionBenchmarkFileGenerator.java new file mode 100644 index 000000000000..cfa016a15a3d --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/DoubleCompressionBenchmarkFileGenerator.java @@ -0,0 +1,194 @@ +/* + * 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.benchmark; + +import com.google.common.collect.ImmutableList; +import com.google.common.io.ByteSink; +import io.druid.benchmark.datagen.BenchmarkColumnSchema; +import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; +import io.druid.data.ValueType; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.DoubleSupplierSerializer; +import io.druid.segment.data.TmpFileIOPeon; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.net.URISyntaxException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class DoubleCompressionBenchmarkFileGenerator +{ + public static final int ROW_NUM = 5000000; + public static final List compressions = + ImmutableList.of( + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.NONE + ); + + private static String dirPath = "doubleCompress/"; + + public static void main(String[] args) throws IOException, URISyntaxException + { + if (args.length >= 1) { + dirPath = args[0]; + } + + BenchmarkColumnSchema enumeratedSchema = BenchmarkColumnSchema.makeEnumerated("", ValueType.DOUBLE, true, 1, 0d, + ImmutableList.of( + 0d, + 1.1d, + 2.2d, + 3.3d, + 4.4d + ), + ImmutableList.of( + 0.95, + 0.001, + 0.0189, + 0.03, + 0.0001 + ) + ); + BenchmarkColumnSchema zipfLowSchema = BenchmarkColumnSchema.makeZipf( + "", + ValueType.DOUBLE, + true, + 1, + 0d, + -1, + 1000, + 1d + ); + BenchmarkColumnSchema zipfHighSchema = BenchmarkColumnSchema.makeZipf( + "", + ValueType.DOUBLE, + true, + 1, + 0d, + -1, + 1000, + 3d + ); + BenchmarkColumnSchema sequentialSchema = BenchmarkColumnSchema.makeSequential( + "", + ValueType.DOUBLE, + true, + 1, + 0d, + 1470187671, + 2000000000 + ); + BenchmarkColumnSchema uniformSchema = BenchmarkColumnSchema.makeContinuousUniform( + "", + ValueType.DOUBLE, + true, + 1, + 0d, + 0, + 1000 + ); + + Map generators = new HashMap<>(); + generators.put("enumerate", new BenchmarkColumnValueGenerator(enumeratedSchema, 1)); + generators.put("zipfLow", new BenchmarkColumnValueGenerator(zipfLowSchema, 1)); + generators.put("zipfHigh", new BenchmarkColumnValueGenerator(zipfHighSchema, 1)); + generators.put("sequential", new BenchmarkColumnValueGenerator(sequentialSchema, 1)); + generators.put("uniform", new BenchmarkColumnValueGenerator(uniformSchema, 1)); + + File dir = new File(dirPath); + dir.mkdir(); + + // create data files using BenchmarkColunValueGenerator + for (Map.Entry entry : generators.entrySet()) { + final File dataFile = new File(dir, entry.getKey()); + dataFile.delete(); + try (Writer writer = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(dataFile)))) { + for (int i = 0; i < ROW_NUM; i++) { + writer.write((Double) entry.getValue().generateRowValue() + "\n"); + } + } + } + + // create compressed files using all combinations of CompressionStrategy and DoubleEncoding provided + for (Map.Entry entry : generators.entrySet()) { + for (CompressedObjectStrategy.CompressionStrategy compression : compressions) { + String name = entry.getKey() + "-" + compression.toString(); + System.out.print(name + ": "); + File compFile = new File(dir, name); + compFile.delete(); + File dataFile = new File(dir, entry.getKey()); + + TmpFileIOPeon iopeon = new TmpFileIOPeon(true); + DoubleSupplierSerializer writer = CompressionFactory.getDoubleSerializer( + iopeon, + "double", + ByteOrder.nativeOrder(), + compression + ); + BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(dataFile))); + + try (FileChannel output = FileChannel.open( + compFile.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + )) { + writer.open(); + String line; + while ((line = br.readLine()) != null) { + writer.add(Double.parseDouble(line)); + } + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + writer.closeAndConsolidate( + new ByteSink() + { + @Override + public OutputStream openStream() throws IOException + { + return baos; + } + } + ); + output.write(ByteBuffer.wrap(baos.toByteArray())); + } + finally { + iopeon.cleanup(); + br.close(); + } + System.out.print(compFile.length() / 1024 + "\n"); + } + } + } +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java index 6029f8716bf8..6b9f9221466b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java @@ -23,7 +23,7 @@ import com.google.common.io.ByteSink; import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; -import io.druid.segment.column.ValueType; +import io.druid.data.ValueType; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.FloatSupplierSerializer; diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java index 14a94bf7c310..ba633484d85e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java @@ -23,7 +23,7 @@ import com.google.common.io.ByteSink; import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; -import io.druid.segment.column.ValueType; +import io.druid.data.ValueType; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.LongSupplierSerializer; diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnSchema.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnSchema.java index b477f84add58..a3d17ec30312 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnSchema.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnSchema.java @@ -19,7 +19,7 @@ package io.druid.benchmark.datagen; -import io.druid.segment.column.ValueType; +import io.druid.data.ValueType; import java.util.List; diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java index 966021b483bd..ed63fe1c5282 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java @@ -19,7 +19,7 @@ package io.druid.benchmark.datagen; -import io.druid.segment.column.ValueType; +import io.druid.data.ValueType; import org.apache.commons.math3.distribution.AbstractIntegerDistribution; import org.apache.commons.math3.distribution.AbstractRealDistribution; import org.apache.commons.math3.distribution.EnumeratedDistribution; @@ -128,6 +128,13 @@ private Object convertType(Object input, ValueType type) ret = Float.parseFloat(input.toString()); } break; + case DOUBLE: + if (input instanceof Number) { + ret = ((Number) input).doubleValue(); + } else { + ret = Double.parseDouble(input.toString()); + } + break; default: throw new UnsupportedOperationException("Unknown data type: " + type); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java index 4222199a2646..7088e9893822 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/BenchmarkSchemas.java @@ -20,6 +20,7 @@ package io.druid.benchmark.datagen; import com.google.common.collect.ImmutableList; +import io.druid.data.ValueType; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleMinAggregatorFactory; @@ -27,7 +28,6 @@ import io.druid.query.aggregation.LongMaxAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; -import io.druid.segment.column.ValueType; import org.joda.time.Interval; import java.util.ArrayList; diff --git a/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java b/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java index a3733939c903..b49b6b9a66da 100644 --- a/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java +++ b/benchmarks/src/test/java/io/druid/benchmark/BenchmarkDataGeneratorTest.java @@ -21,8 +21,8 @@ import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkDataGenerator; +import io.druid.data.ValueType; import io.druid.data.input.InputRow; -import io.druid.segment.column.ValueType; import org.joda.time.Interval; import org.junit.Test; diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregator.java index f48102ca1f73..2f0d959707c7 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountAggregator.java @@ -84,4 +84,10 @@ public long getLong() { return (long) mutableBitmap.size(); } + + @Override + public double getDouble() + { + return (double) mutableBitmap.size(); + } } diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java index a90dd6f2dcca..8c5a46dfe053 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java @@ -78,6 +78,12 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getLong(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return (double) buf.getLong(position); + } + @Override public long getLong(ByteBuffer buf, int position) { diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountAggregator.java index 4828446427ac..a95501be2b93 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountAggregator.java @@ -69,4 +69,10 @@ public long getLong() { return (long) 0; } + + @Override + public double getDouble() + { + return (double) 0; + } } diff --git a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java index 0238a8e39064..8fa0b9502b3f 100644 --- a/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java +++ b/extensions-contrib/distinctcount/src/main/java/io/druid/query/aggregation/distinctcount/EmptyDistinctCountBufferAggregator.java @@ -52,6 +52,12 @@ public float getFloat(ByteBuffer buf, int position) return (float) 0; } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return (double) 0; + } + @Override public long getLong(ByteBuffer buf, int position) { diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java index 02da6e064d5e..f2238bcdd2e8 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchAggregator.java @@ -58,6 +58,12 @@ public long getLong() throw new UnsupportedOperationException("Not implemented"); } + @Override + public double getDouble() + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public String getName() { diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java index 5073c1b79b00..4f93ea691f13 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/EmptySketchBufferAggregator.java @@ -51,6 +51,12 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("Not implemented"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public long getLong(ByteBuffer buf, int position) { diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java index b9a369c33351..68d2cea80d87 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -89,6 +89,12 @@ public long getLong() throw new UnsupportedOperationException("Not implemented"); } + @Override + public double getDouble() + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public String getName() { diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java index 2958b1846371..4073ec38d4dd 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchBufferAggregator.java @@ -104,6 +104,12 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("Not implemented"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + @Override public long getLong(ByteBuffer buf, int position) { diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java index 88be52f31eaf..e2fc77aefd75 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregator.java @@ -68,7 +68,7 @@ public ApproximateHistogramAggregator( @Override public void aggregate() { - histogram.offer(selector.get()); + histogram.offer((float)selector.get()); } @Override @@ -95,6 +95,12 @@ 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 String getName() { diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java index 297fde021d09..cb3dbf44efd1 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregator.java @@ -87,6 +87,12 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getFloat()"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramBufferAggregator does not support getDouble()"); + } + @Override public long getLong(ByteBuffer buf, int position) diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java index ee04f45b8adf..d7826ddbdec3 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregator.java @@ -93,6 +93,12 @@ public long getLong() throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getLong()"); } + @Override + public double getDouble() + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingAggregator does not support getDouble()"); + } + @Override public String getName() { diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java index bd52b603badb..712ea564c0dd 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingBufferAggregator.java @@ -91,6 +91,12 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getFloat()"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("ApproximateHistogramFoldingBufferAggregator does not support getDouble()"); + } + @Override public long getLong(ByteBuffer buf, int position) { diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregator.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregator.java index 2553322d2fda..d4c382d88046 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregator.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregator.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation.variance; import io.druid.query.aggregation.Aggregator; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -66,6 +67,12 @@ public float getFloat() throw new UnsupportedOperationException("VarianceAggregator does not support getFloat()"); } + @Override + public double getDouble() + { + throw new UnsupportedOperationException("VarianceAggregator does not support getDouble()"); + } + @Override public long getLong() { @@ -89,6 +96,23 @@ public void aggregate() } } + public static final class DoubleVarianceAggregator extends VarianceAggregator + { + private final DoubleColumnSelector selector; + + public DoubleVarianceAggregator(String name, DoubleColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + holder.add(selector.get()); + } + } + public static final class LongVarianceAggregator extends VarianceAggregator { private final LongColumnSelector selector; diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java index 4ab6bc25a970..64d4e3fdcd80 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorCollector.java @@ -137,6 +137,17 @@ public VarianceAggregatorCollector add(float v) return this; } + public VarianceAggregatorCollector add(double v) + { + count++; + sum += v; + if (count > 1) { + double t = count * v - sum; + nvariance += (t * t) / ((double) count * (count - 1)); + } + return this; + } + public VarianceAggregatorCollector add(long v) { count++; diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java index d5b85718f348..e332ec944896 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -28,10 +28,8 @@ import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactoryNotMergeableException; -import io.druid.query.aggregation.Aggregators; import io.druid.query.aggregation.BufferAggregator; import io.druid.segment.ColumnSelectorFactory; -import io.druid.segment.ObjectColumnSelector; import org.apache.commons.codec.binary.Base64; import java.nio.ByteBuffer; @@ -83,6 +81,12 @@ public String getTypeName() return "variance"; } + @Override + public String getInputTypeName() + { + return inputType; + } + @Override public int getMaxIntermediateSize() { @@ -92,51 +96,54 @@ public int getMaxIntermediateSize() @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); - if (selector == null) { - return Aggregators.noopAggregator(); - } - if ("float".equalsIgnoreCase(inputType)) { return new VarianceAggregator.FloatVarianceAggregator( name, metricFactory.makeFloatColumnSelector(fieldName) ); + } else if ("double".equalsIgnoreCase(inputType)) { + return new VarianceAggregator.DoubleVarianceAggregator( + name, + metricFactory.makeDoubleColumnSelector(fieldName) + ); } else if ("long".equalsIgnoreCase(inputType)) { return new VarianceAggregator.LongVarianceAggregator( name, metricFactory.makeLongColumnSelector(fieldName) ); } else if ("variance".equalsIgnoreCase(inputType)) { - return new VarianceAggregator.ObjectVarianceAggregator(name, selector); + return new VarianceAggregator.ObjectVarianceAggregator( + name, + metricFactory.makeObjectColumnSelector(fieldName) + ); } throw new IAE( - "Incompatible type for metric[%s], expected a float, long or variance, got a %s", fieldName, inputType + "Incompatible type for metric[%s], expected a float, double, long or variance, got a %s", fieldName, inputType ); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - ObjectColumnSelector selector = metricFactory.makeObjectColumnSelector(fieldName); - if (selector == null) { - return Aggregators.noopBufferAggregator(); - } if ("float".equalsIgnoreCase(inputType)) { return new VarianceBufferAggregator.FloatVarianceAggregator( - name, metricFactory.makeFloatColumnSelector(fieldName) ); + } else if ("double".equalsIgnoreCase(inputType)) { + return new VarianceBufferAggregator.DoubleVarianceAggregator( + metricFactory.makeDoubleColumnSelector(fieldName) + ); } else if ("long".equalsIgnoreCase(inputType)) { return new VarianceBufferAggregator.LongVarianceAggregator( - name, metricFactory.makeLongColumnSelector(fieldName) ); } else if ("variance".equalsIgnoreCase(inputType)) { - return new VarianceBufferAggregator.ObjectVarianceAggregator(name, selector); + return new VarianceBufferAggregator.ObjectVarianceAggregator( + metricFactory.makeObjectColumnSelector(fieldName) + ); } throw new IAE( - "Incompatible type for metric[%s], expected a float, long or variance, got a %s", fieldName, inputType + "Incompatible type for metric[%s], expected a float, double, long or variance, got a %s", fieldName, inputType ); } diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java index 77017de52312..56966e0e3c0c 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceBufferAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Doubles; import com.google.common.primitives.Longs; import io.druid.query.aggregation.BufferAggregator; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -36,13 +37,6 @@ public abstract class VarianceBufferAggregator implements BufferAggregator private static final int SUM_OFFSET = Longs.BYTES; private static final int NVARIANCE_OFFSET = SUM_OFFSET + Doubles.BYTES; - protected final String name; - - public VarianceBufferAggregator(String name) - { - this.name = name; - } - @Override public void init(final ByteBuffer buf, final int position) { @@ -67,10 +61,16 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("VarianceBufferAggregator does not support getFloat()"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("VarianceBufferAggregator does not support getDouble()"); + } + @Override public long getLong(ByteBuffer buf, int position) { - throw new UnsupportedOperationException("VarianceBufferAggregator does not support getFloat()"); + throw new UnsupportedOperationException("VarianceBufferAggregator does not support getLong()"); } @Override @@ -82,9 +82,8 @@ public static final class FloatVarianceAggregator extends VarianceBufferAggregat { private final FloatColumnSelector selector; - public FloatVarianceAggregator(String name, FloatColumnSelector selector) + public FloatVarianceAggregator(FloatColumnSelector selector) { - super(name); this.selector = selector; } @@ -104,13 +103,37 @@ public void aggregate(ByteBuffer buf, int position) } } + public static final class DoubleVarianceAggregator extends VarianceBufferAggregator + { + private final DoubleColumnSelector selector; + + public DoubleVarianceAggregator(DoubleColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + double v = selector.get(); + long count = buf.getLong(position + COUNT_OFFSET) + 1; + double sum = buf.getDouble(position + SUM_OFFSET) + v; + buf.putLong(position, count); + buf.putDouble(position + SUM_OFFSET, sum); + if (count > 1) { + double t = count * v - sum; + double variance = buf.getDouble(position + NVARIANCE_OFFSET) + (t * t) / ((double) count * (count - 1)); + buf.putDouble(position + NVARIANCE_OFFSET, variance); + } + } + } + public static final class LongVarianceAggregator extends VarianceBufferAggregator { private final LongColumnSelector selector; - public LongVarianceAggregator(String name, LongColumnSelector selector) + public LongVarianceAggregator(LongColumnSelector selector) { - super(name); this.selector = selector; } @@ -134,9 +157,8 @@ public static final class ObjectVarianceAggregator extends VarianceBufferAggrega { private final ObjectColumnSelector selector; - public ObjectVarianceAggregator(String name, ObjectColumnSelector selector) + public ObjectVarianceAggregator(ObjectColumnSelector selector) { - super(name); this.selector = selector; } diff --git a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java index 89d92179770b..fdf443311d75 100644 --- a/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java +++ b/extensions-core/stats/src/test/java/io/druid/query/aggregation/variance/VarianceAggregatorCollectorTest.java @@ -113,7 +113,7 @@ public void testVariance() for (int i = 0; i < mergeOn; i++) { holders1.add(new VarianceAggregatorCollector()); holders2.add(Pair.of( - new VarianceBufferAggregator.FloatVarianceAggregator("XX", valueHandOver), + new VarianceBufferAggregator.FloatVarianceAggregator(valueHandOver), ByteBuffer.allocate(VarianceAggregatorCollector.getMaxIntermediateSize()) )); } @@ -129,7 +129,7 @@ public void testVariance() } ObjectHandOver collectHandOver = new ObjectHandOver(); ByteBuffer buffer = ByteBuffer.allocate(VarianceAggregatorCollector.getMaxIntermediateSize()); - VarianceBufferAggregator.ObjectVarianceAggregator merger = new VarianceBufferAggregator.ObjectVarianceAggregator("xxx", collectHandOver); + VarianceBufferAggregator.ObjectVarianceAggregator merger = new VarianceBufferAggregator.ObjectVarianceAggregator(collectHandOver); for (int i = 0; i < mergeOn; i++) { collectHandOver.v = holders2.get(i).lhs.get(holders2.get(i).rhs, 0); merger.aggregate(buffer, 0); 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 2872de2a3745..92589ea9b218 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -426,6 +426,12 @@ public float getFloatMetric(String metric) return row.getFloatMetric(metric); } + @Override + public double getDoubleMetric(String metric) + { + return row.getDoubleMetric(metric); + } + @Override public long getLongMetric(String metric) { 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 46bf466c8422..dc8aac21e996 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/InputRowSerde.java @@ -108,6 +108,8 @@ public InputRow get() if (t.equals("float")) { out.writeFloat(agg.getFloat()); + } else if (t.equals("double")) { + out.writeDouble(agg.getDouble()); } else if (t.equals("long")) { WritableUtils.writeVLong(out, agg.getLong()); } else { @@ -208,6 +210,8 @@ public static final InputRow fromBytes(byte[] data, AggregatorFactory[] aggs) String type = getType(metric, aggs, i); if (type.equals("float")) { event.put(metric, in.readFloat()); + } else if (type.equals("double")) { + event.put(metric, in.readDouble()); } else if (type.equals("long")) { event.put(metric, WritableUtils.readVLong(in)); } else { 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..0256eb50f8ba 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 @@ -75,6 +75,12 @@ public float getFloatMetric(String metric) return delegate.getFloatMetric(metric); } + @Override + public double getDoubleMetric(String metric) + { + return delegate.getDoubleMetric(metric); + } + @Override public long getLongMetric(String metric) { diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 2e9582da89c6..761c577b0c0f 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -29,6 +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.GenericMaxAggregatorFactory; +import io.druid.query.aggregation.GenericMinAggregatorFactory; +import io.druid.query.aggregation.GenericSumAggregatorFactory; import io.druid.query.aggregation.HistogramAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongMaxAggregatorFactory; @@ -63,6 +66,9 @@ public AggregatorsModule() @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { + @JsonSubTypes.Type(name = "max", value = GenericMaxAggregatorFactory.class), + @JsonSubTypes.Type(name = "min", value = GenericMinAggregatorFactory.class), + @JsonSubTypes.Type(name = "sum", value = GenericSumAggregatorFactory.class), @JsonSubTypes.Type(name = "count", value = CountAggregatorFactory.class), @JsonSubTypes.Type(name = "longSum", value = LongSumAggregatorFactory.class), @JsonSubTypes.Type(name = "doubleSum", value = DoubleSumAggregatorFactory.class), diff --git a/processing/src/main/java/io/druid/query/DruidMetrics.java b/processing/src/main/java/io/druid/query/DruidMetrics.java index a62f5c2203cc..3b6a5a6c58d6 100644 --- a/processing/src/main/java/io/druid/query/DruidMetrics.java +++ b/processing/src/main/java/io/druid/query/DruidMetrics.java @@ -55,7 +55,8 @@ 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.getTypeName().equals("float") && !agg.getTypeName().equals("long") && + !agg.getTypeName().equals("double")) { 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..16676eecf219 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java @@ -41,4 +41,5 @@ public interface Aggregator { void close(); long getLong(); + double getDouble(); } 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 f438570cada1..955be30362c5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorFactory.java @@ -116,6 +116,12 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre public abstract String getTypeName(); + // input type for ingestion + public String getInputTypeName() + { + return getTypeName(); + } + /** * Returns the maximum size that this aggregator will require in bytes for intermediate storage of results. * 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..a6f6b48310b7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregators.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregators.java @@ -70,6 +70,12 @@ public long getLong() { return 0; } + + @Override + public double getDouble() + { + return 0D; + } }; } @@ -101,6 +107,12 @@ public float getFloat(ByteBuffer buf, int position) return 0; } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return 0D; + } + @Override public long getLong(ByteBuffer buf, int position) 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..793dc7c3d8b7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -89,6 +89,8 @@ public interface BufferAggregator */ float getFloat(ByteBuffer buf, int position); + double getDouble(ByteBuffer buf, int position); + /** * Returns the long representation of the given aggregate byte array * 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..e744b51c85df 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,12 @@ public long getLong() return count; } + @Override + public double getDouble() + { + return (double) count; + } + @Override public String getName() { 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..abeb5d802654 100644 --- a/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/CountBufferAggregator.java @@ -50,6 +50,11 @@ public float getFloat(ByteBuffer buf, int position) return buf.getLong(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getLong(position); + } @Override public long getLong(ByteBuffer buf, int position) 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..c9bda688c51e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java @@ -19,13 +19,14 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.util.Comparator; /** */ -public class DoubleMaxAggregator implements Aggregator +public abstract class DoubleMaxAggregator implements Aggregator { static final Comparator COMPARATOR = DoubleSumAggregator.COMPARATOR; @@ -34,25 +35,16 @@ static double combineValues(Object lhs, Object rhs) return Math.max(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue()); } - private final FloatColumnSelector selector; - private final String name; + final String name; - private double max; + double max; - public DoubleMaxAggregator(String name, FloatColumnSelector selector) + public DoubleMaxAggregator(String name) { this.name = name; - this.selector = selector; - reset(); } - @Override - public void aggregate() - { - max = Math.max(max, selector.get()); - } - @Override public void reset() { @@ -78,15 +70,15 @@ public long getLong() } @Override - public String getName() + public double getDouble() { - return this.name; + return max; } @Override - public Aggregator clone() + public String getName() { - return new DoubleMaxAggregator(name, selector); + return this.name; } @Override @@ -94,4 +86,50 @@ public void close() { // no resources to cleanup } + + public static class FloatInput extends DoubleMaxAggregator + { + private final FloatColumnSelector selector; + + public FloatInput(String name, FloatColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + max = Math.max(max, selector.get()); + } + + @Override + public Aggregator clone() + { + return new FloatInput(name, selector); + } + } + + public static class DoubleInput extends DoubleMaxAggregator + { + private final DoubleColumnSelector selector; + + public DoubleInput(String name, DoubleColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + max = Math.max(max, selector.get()); + } + + @Override + public Aggregator clone() + { + return new DoubleInput(name, selector); + } + } } 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 029bc86afafb..add5b339e8df 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.FloatInput(name, metricFactory.makeFloatColumnSelector(fieldName)); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new DoubleMaxBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleMaxBufferAggregator.FloatInput(metricFactory.makeFloatColumnSelector(fieldName)); } @Override 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..376c7b89b0c1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java @@ -19,33 +19,21 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; /** */ -public class DoubleMaxBufferAggregator implements BufferAggregator +public abstract class DoubleMaxBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; - - public DoubleMaxBufferAggregator(FloatColumnSelector selector) - { - this.selector = selector; - } - @Override public void init(ByteBuffer buf, int position) { buf.putDouble(position, Double.NEGATIVE_INFINITY); } - @Override - public void aggregate(ByteBuffer buf, int position) - { - buf.putDouble(position, Math.max(buf.getDouble(position), (double) selector.get())); - } - @Override public Object get(ByteBuffer buf, int position) { @@ -58,6 +46,12 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getDouble(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public long getLong(ByteBuffer buf, int position) { @@ -69,4 +63,36 @@ public void close() { // no resources to cleanup } + + public static class FloatInput extends DoubleMaxBufferAggregator + { + private final FloatColumnSelector selector; + + public FloatInput(FloatColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putDouble(position, Math.max(buf.getDouble(position), (double) selector.get())); + } + } + + public static class DoubleInput extends DoubleMaxBufferAggregator + { + private final DoubleColumnSelector selector; + + public DoubleInput(DoubleColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putDouble(position, Math.max(buf.getDouble(position), selector.get())); + } + } } 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..cb95e830aadb 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java @@ -19,13 +19,14 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.util.Comparator; /** */ -public class DoubleMinAggregator implements Aggregator +public abstract class DoubleMinAggregator implements Aggregator { static final Comparator COMPARATOR = DoubleSumAggregator.COMPARATOR; @@ -34,25 +35,16 @@ static double combineValues(Object lhs, Object rhs) return Math.min(((Number) lhs).doubleValue(), ((Number) rhs).doubleValue()); } - private final FloatColumnSelector selector; - private final String name; + final String name; - private double min; + double min; - public DoubleMinAggregator(String name, FloatColumnSelector selector) + public DoubleMinAggregator(String name) { this.name = name; - this.selector = selector; - reset(); } - @Override - public void aggregate() - { - min = Math.min(min, (double) selector.get()); - } - @Override public void reset() { @@ -78,15 +70,15 @@ public long getLong() } @Override - public String getName() + public double getDouble() { - return this.name; + return min; } @Override - public Aggregator clone() + public String getName() { - return new DoubleMinAggregator(name, selector); + return this.name; } @Override @@ -94,4 +86,50 @@ public void close() { // no resources to cleanup } + + public static class FloatInput extends DoubleMinAggregator + { + private final FloatColumnSelector selector; + + public FloatInput(String name, FloatColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + min = Math.min(min, selector.get()); + } + + @Override + public Aggregator clone() + { + return new FloatInput(name, selector); + } + } + + public static class DoubleInput extends DoubleMinAggregator + { + private final DoubleColumnSelector selector; + + public DoubleInput(String name, DoubleColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + min = Math.min(min, selector.get()); + } + + @Override + public Aggregator clone() + { + return new DoubleInput(name, selector); + } + } } 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 04f1de0ede39..9b02552ec471 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.FloatInput(name, metricFactory.makeFloatColumnSelector(fieldName)); } @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new DoubleMinBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleMinBufferAggregator.FloatInput(metricFactory.makeFloatColumnSelector(fieldName)); } @Override 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..fb9f1023f1d1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java @@ -19,33 +19,21 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; /** */ -public class DoubleMinBufferAggregator implements BufferAggregator +public abstract class DoubleMinBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; - - public DoubleMinBufferAggregator(FloatColumnSelector selector) - { - this.selector = selector; - } - @Override public void init(ByteBuffer buf, int position) { buf.putDouble(position, Double.POSITIVE_INFINITY); } - @Override - public void aggregate(ByteBuffer buf, int position) - { - buf.putDouble(position, Math.min(buf.getDouble(position), (double) selector.get())); - } - @Override public Object get(ByteBuffer buf, int position) { @@ -58,11 +46,16 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getDouble(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } @Override public long getLong(ByteBuffer buf, int position) { - return (long) buf.getDouble(position); + return (long) buf.getDouble(position); } @Override @@ -70,4 +63,36 @@ public void close() { // no resources to cleanup } + + public static class FloatInput extends DoubleMinBufferAggregator + { + private final FloatColumnSelector selector; + + public FloatInput(FloatColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putDouble(position, Math.min(buf.getDouble(position), (double) selector.get())); + } + } + + public static class DoubleInput extends DoubleMinBufferAggregator + { + private final DoubleColumnSelector selector; + + public DoubleInput(DoubleColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putDouble(position, Math.min(buf.getDouble(position), selector.get())); + } + } } 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..cae63b80c316 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregator.java @@ -21,13 +21,14 @@ 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; /** */ -public class DoubleSumAggregator implements Aggregator +public abstract class DoubleSumAggregator implements Aggregator { static final Comparator COMPARATOR = new Ordering() { @@ -43,25 +44,15 @@ static double combineValues(Object lhs, Object rhs) return ((Number) lhs).doubleValue() + ((Number) rhs).doubleValue(); } - private final FloatColumnSelector selector; - private final String name; + final String name; + double sum; - private double sum; - - public DoubleSumAggregator(String name, FloatColumnSelector selector) + public DoubleSumAggregator(String name) { this.name = name; - this.selector = selector; - this.sum = 0; } - @Override - public void aggregate() - { - sum += selector.get(); - } - @Override public void reset() { @@ -87,15 +78,15 @@ public long getLong() } @Override - public String getName() + public double getDouble() { - return this.name; + return sum; } @Override - public Aggregator clone() + public String getName() { - return new DoubleSumAggregator(name, selector); + return this.name; } @Override @@ -103,4 +94,51 @@ public void close() { // no resources to cleanup } + + + public static class FloatInput extends DoubleSumAggregator + { + private final FloatColumnSelector selector; + + public FloatInput(String name, FloatColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + sum += selector.get(); + } + + @Override + public Aggregator clone() + { + return new FloatInput(name, selector); + } + } + + public static class DoubleInput extends DoubleSumAggregator + { + private final DoubleColumnSelector selector; + + public DoubleInput(String name, DoubleColumnSelector selector) + { + super(name); + this.selector = selector; + } + + @Override + public void aggregate() + { + sum += selector.get(); + } + + @Override + public Aggregator clone() + { + return new DoubleInput(name, selector); + } + } } 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 a05d76a10948..337521017fca 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -56,7 +56,7 @@ public DoubleSumAggregatorFactory( @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - return new DoubleSumAggregator( + return new DoubleSumAggregator.FloatInput( name, metricFactory.makeFloatColumnSelector(fieldName) ); @@ -65,7 +65,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return new DoubleSumBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + return new DoubleSumBufferAggregator.FloatInput(metricFactory.makeFloatColumnSelector(fieldName)); } @Override 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..4a8c2392bac3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleSumBufferAggregator.java @@ -19,35 +19,21 @@ package io.druid.query.aggregation; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import java.nio.ByteBuffer; /** */ -public class DoubleSumBufferAggregator implements BufferAggregator +public abstract class DoubleSumBufferAggregator implements BufferAggregator { - private final FloatColumnSelector selector; - - public DoubleSumBufferAggregator( - FloatColumnSelector selector - ) - { - this.selector = selector; - } - @Override public void init(ByteBuffer buf, int position) { buf.putDouble(position, 0.0d); } - @Override - public void aggregate(ByteBuffer buf, int position) - { - buf.putDouble(position, buf.getDouble(position) + (double) selector.get()); - } - @Override public Object get(ByteBuffer buf, int position) { @@ -60,6 +46,12 @@ public float getFloat(ByteBuffer buf, int position) return (float) buf.getDouble(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public long getLong(ByteBuffer buf, int position) @@ -72,4 +64,36 @@ public void close() { // no resources to cleanup } + + public static class FloatInput extends DoubleSumBufferAggregator + { + private final FloatColumnSelector selector; + + public FloatInput(FloatColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putDouble(position, buf.getDouble(position) + selector.get()); + } + } + + public static class DoubleInput extends DoubleSumBufferAggregator + { + private final DoubleColumnSelector selector; + + public DoubleInput(DoubleColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putDouble(position, buf.getDouble(position) + selector.get()); + } + } } 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..d2a629b4bf3d 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,12 @@ public long getLong() return delegate.getLong(); } + @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 77f54a296408..75f3d457d32a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/FilteredAggregatorFactory.java @@ -23,6 +23,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Strings; +import io.druid.data.ValueType; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.filter.DimFilter; import io.druid.query.filter.DruidLongPredicate; @@ -33,7 +34,6 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.BooleanValueMatcher; import io.druid.segment.filter.Filters; @@ -146,6 +146,12 @@ public String getTypeName() return delegate.getTypeName(); } + @Override + public String getInputTypeName() + { + return delegate.getInputTypeName(); + } + @Override public int getMaxIntermediateSize() { 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..0813c6547a9f 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,12 @@ public float getFloat(ByteBuffer buf, int position) return delegate.getFloat(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/GenericMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/GenericMaxAggregatorFactory.java new file mode 100644 index 000000000000..c6be3cee0f60 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/GenericMaxAggregatorFactory.java @@ -0,0 +1,242 @@ +/* + * 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.Doubles; +import com.metamx.common.StringUtils; +import io.druid.data.ValueType; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + */ +public class GenericMaxAggregatorFactory extends AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x18; + + private final String fieldName; + private final String name; + private final ValueType inputType; + + private final Comparator comparator; + + @JsonCreator + public GenericMaxAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("inputType") final ValueType inputType + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + Preconditions.checkArgument(inputType == null || ValueType.isNumeric(inputType)); + + this.name = name; + this.fieldName = fieldName; + this.inputType = inputType == null ? ValueType.DOUBLE : inputType; + this.comparator = inputType.comparator(); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + switch (inputType) { + case FLOAT: + return new DoubleMaxAggregator.FloatInput(name, metricFactory.makeFloatColumnSelector(fieldName)); + case DOUBLE: + return new DoubleMaxAggregator.DoubleInput(name, metricFactory.makeDoubleColumnSelector(fieldName)); + case LONG: + return new LongMaxAggregator(name, metricFactory.makeLongColumnSelector(fieldName)); + } + throw new IllegalStateException(); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + switch (inputType) { + case FLOAT: + return new DoubleMaxBufferAggregator.FloatInput(metricFactory.makeFloatColumnSelector(fieldName)); + case DOUBLE: + return new DoubleMaxBufferAggregator.DoubleInput(metricFactory.makeDoubleColumnSelector(fieldName)); + case LONG: + return new LongMaxBufferAggregator(metricFactory.makeLongColumnSelector(fieldName)); + } + throw new IllegalStateException(); + } + + @Override + public Comparator getComparator() + { + return comparator; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return comparator.compare(lhs, rhs) >= 0 ? lhs : rhs; + } + + @Override + public AggregatorFactory getCombiningFactory() + { + switch (inputType) { + case FLOAT: + case DOUBLE: + return new GenericMaxAggregatorFactory(name, name, ValueType.DOUBLE); + case LONG: + return new GenericMaxAggregatorFactory(name, name, ValueType.LONG); + } + throw new IllegalStateException(); + } + + @Override + public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException + { + if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { + return getCombiningFactory(); + } else { + throw new AggregatorFactoryNotMergeableException(this, other); + } + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new GenericMaxAggregatorFactory(fieldName, fieldName, inputType)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((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; + } + + @JsonProperty + public String getInputType() + { + return inputType.toString(); + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + byte[] inputTypeBytes = StringUtils.toUtf8(inputType.name()); + + return ByteBuffer.allocate(1 + fieldNameBytes.length + inputTypeBytes.length) + .put(CACHE_TYPE_ID) + .put(fieldNameBytes) + .put(inputTypeBytes) + .array(); + } + + @Override + public String getTypeName() + { + return inputType.name().toLowerCase(); + } + + @Override + public int getMaxIntermediateSize() + { + return Doubles.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return Double.NEGATIVE_INFINITY; + } + + @Override + public String toString() + { + return "GenericMaxAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + "inputType='" + inputType + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GenericMaxAggregatorFactory that = (GenericMaxAggregatorFactory) o; + + if (!Objects.equals(fieldName, that.fieldName)) { + return false; + } + if (!Objects.equals(inputType, that.inputType)) { + return false; + } + if (!Objects.equals(name, that.name)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return Objects.hash(fieldName, inputType, name); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/GenericMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/GenericMinAggregatorFactory.java new file mode 100644 index 000000000000..4a7951a4dd74 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/GenericMinAggregatorFactory.java @@ -0,0 +1,241 @@ +/* + * 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.Doubles; +import com.metamx.common.StringUtils; +import io.druid.data.ValueType; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + */ +public class GenericMinAggregatorFactory extends AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x19; + + private final String fieldName; + private final String name; + private final ValueType inputType; + private final Comparator comparator; + + @JsonCreator + public GenericMinAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("inputType") final ValueType inputType + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + Preconditions.checkArgument(inputType == null || ValueType.isNumeric(inputType)); + + this.name = name; + this.fieldName = fieldName; + this.inputType = inputType == null ? ValueType.DOUBLE : inputType; + this.comparator = inputType.comparator(); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + switch (inputType) { + case FLOAT: + return new DoubleMinAggregator.FloatInput(name, metricFactory.makeFloatColumnSelector(fieldName)); + case DOUBLE: + return new DoubleMinAggregator.DoubleInput(name, metricFactory.makeDoubleColumnSelector(fieldName)); + case LONG: + return new LongMinAggregator(name, metricFactory.makeLongColumnSelector(fieldName)); + } + throw new IllegalStateException(); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + switch (inputType) { + case FLOAT: + return new DoubleMinBufferAggregator.FloatInput(metricFactory.makeFloatColumnSelector(fieldName)); + case DOUBLE: + return new DoubleMinBufferAggregator.DoubleInput(metricFactory.makeDoubleColumnSelector(fieldName)); + case LONG: + return new LongMinBufferAggregator(metricFactory.makeLongColumnSelector(fieldName)); + } + throw new IllegalStateException(); + } + + @Override + public Comparator getComparator() + { + return comparator; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return comparator.compare(lhs, rhs) < 0 ? lhs : rhs; + } + + @Override + public AggregatorFactory getCombiningFactory() + { + switch (inputType) { + case FLOAT: + case DOUBLE: + return new GenericMinAggregatorFactory(name, name, ValueType.DOUBLE); + case LONG: + return new GenericMinAggregatorFactory(name, name, ValueType.LONG); + } + throw new IllegalStateException(); + } + + @Override + public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException + { + if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { + return getCombiningFactory(); + } else { + throw new AggregatorFactoryNotMergeableException(this, other); + } + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new GenericMinAggregatorFactory(fieldName, fieldName, inputType)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((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; + } + + @JsonProperty + public String getInputType() + { + return inputType.toString(); + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + byte[] inputTypeBytes = StringUtils.toUtf8(inputType.name()); + + return ByteBuffer.allocate(1 + fieldNameBytes.length + inputTypeBytes.length) + .put(CACHE_TYPE_ID) + .put(fieldNameBytes) + .put(inputTypeBytes) + .array(); + } + + @Override + public String getTypeName() + { + return inputType.name().toLowerCase(); + } + + @Override + public int getMaxIntermediateSize() + { + return Doubles.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return Double.POSITIVE_INFINITY; + } + + @Override + public String toString() + { + return "GenericMinAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + "inputType='" + inputType + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GenericMinAggregatorFactory that = (GenericMinAggregatorFactory) o; + + if (!Objects.equals(fieldName, that.fieldName)) { + return false; + } + if (!Objects.equals(inputType, that.inputType)) { + return false; + } + if (!Objects.equals(name, that.name)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return Objects.hash(fieldName, inputType, name); + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/GenericSumAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/GenericSumAggregatorFactory.java new file mode 100644 index 000000000000..ef35d582bcf5 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/GenericSumAggregatorFactory.java @@ -0,0 +1,250 @@ +/* + * 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.Doubles; +import com.metamx.common.StringUtils; +import io.druid.data.ValueType; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + */ +public class GenericSumAggregatorFactory extends AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x17; + + private final String fieldName; + private final String name; + private final ValueType inputType; + + @JsonCreator + public GenericSumAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("inputType") final ValueType inputType + ) + { + 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; + this.inputType = inputType == null ? ValueType.DOUBLE : inputType; + Preconditions.checkArgument(ValueType.isNumeric(inputType)); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + switch (inputType) { + case FLOAT: + return new DoubleSumAggregator.FloatInput(name, metricFactory.makeFloatColumnSelector(fieldName)); + case DOUBLE: + return new DoubleSumAggregator.DoubleInput(name, metricFactory.makeDoubleColumnSelector(fieldName)); + case LONG: + return new LongSumAggregator(name, metricFactory.makeLongColumnSelector(fieldName)); + } + throw new IllegalStateException(); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + switch (inputType) { + case FLOAT: + return new DoubleSumBufferAggregator.FloatInput(metricFactory.makeFloatColumnSelector(fieldName)); + case DOUBLE: + return new DoubleSumBufferAggregator.DoubleInput(metricFactory.makeDoubleColumnSelector(fieldName)); + case LONG: + return new LongSumBufferAggregator(metricFactory.makeLongColumnSelector(fieldName)); + } + throw new IllegalStateException(); + } + + @Override + public Comparator getComparator() + { + return inputType.comparator(); + } + + @Override + public Object combine(Object lhs, Object rhs) + { + switch (inputType) { + case FLOAT: + case DOUBLE: + return ((Number) lhs).doubleValue() + ((Number) rhs).doubleValue(); + case LONG: + return ((Number) lhs).longValue() + ((Number) rhs).longValue(); + } + throw new IllegalStateException(); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + switch (inputType) { + case FLOAT: + case DOUBLE: + return new GenericSumAggregatorFactory(name, name, ValueType.DOUBLE); + case LONG: + return new GenericSumAggregatorFactory(name, name, ValueType.LONG); + } + throw new IllegalStateException(); + } + + @Override + public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException + { + if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) { + return getCombiningFactory(); + } else { + throw new AggregatorFactoryNotMergeableException(this, other); + } + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new GenericSumAggregatorFactory(fieldName, fieldName, inputType)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((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; + } + + @JsonProperty + public String getInputType() + { + return inputType.toString(); + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + byte[] inputTypeBytes = StringUtils.toUtf8(inputType.name()); + + return ByteBuffer.allocate(1 + fieldNameBytes.length + inputTypeBytes.length) + .put(CACHE_TYPE_ID) + .put(fieldNameBytes) + .put(inputTypeBytes) + .array(); + } + + @Override + public String getTypeName() + { + return inputType.name().toLowerCase(); + } + + @Override + public int getMaxIntermediateSize() + { + return Doubles.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return 0D; + } + + @Override + public String toString() + { + return "GenericSumAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + "inputType='" + inputType + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + GenericSumAggregatorFactory that = (GenericSumAggregatorFactory) o; + + if (!Objects.equals(fieldName, that.fieldName)) { + return false; + } + if (!Objects.equals(inputType, that.inputType)) { + return false; + } + if (!Objects.equals(name, that.name)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + return Objects.hash(fieldName, inputType, name); + } +} 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..d5043af518e1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramAggregator.java @@ -53,7 +53,7 @@ public HistogramAggregator(String name, FloatColumnSelector selector, float[] br @Override public void aggregate() { - histogram.offer(selector.get()); + histogram.offer((float)selector.get()); } @Override @@ -80,6 +80,12 @@ public long getLong() throw new UnsupportedOperationException("HistogramAggregator does not support getLong()"); } + @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/HistogramBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java index d41244e4c66a..cf668ab39d9a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/HistogramBufferAggregator.java @@ -56,7 +56,7 @@ public void init(ByteBuffer buf, int position) @Override public void aggregate(ByteBuffer buf, int position) { - final float value = selector.get(); + final float value = (float)selector.get(); final int minPos = position + minOffset; final int maxPos = position + maxOffset; @@ -90,10 +90,16 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("HistogramBufferAggregator does not support getFloat()"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HistogramBufferAggregator does not support getFloat()"); + } + @Override public long getLong(ByteBuffer buf, int position) { - throw new UnsupportedOperationException("HistogramBufferAggregator does not support getLong()"); + throw new UnsupportedOperationException("HistogramBufferAggregator does not support getDouble()"); } @Override 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..710aec7a12f1 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,12 @@ public long getLong() return (long) current; } + @Override + public double getDouble() + { + return current; + } + @Override public String getName() { 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..770c87466fce 100644 --- a/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/JavaScriptBufferAggregator.java @@ -63,6 +63,12 @@ public float getFloat(ByteBuffer buf, int position) return (float)buf.getDouble(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getDouble(position); + } + @Override public long getLong(ByteBuffer buf, int position) 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..d7b867c85b16 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java @@ -68,7 +68,7 @@ public Object get() @Override public float getFloat() { - return (float) max; + return max; } @Override @@ -77,6 +77,12 @@ public long getLong() return max; } + @Override + public double getDouble() + { + return max; + } + @Override public String getName() { 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..c9ad71a2ec86 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java @@ -55,7 +55,13 @@ public Object get(ByteBuffer buf, int position) @Override public float getFloat(ByteBuffer buf, int position) { - return (float) buf.getLong(position); + return buf.getLong(position); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getLong(position); } @Override 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..0d828fa7596f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java @@ -68,7 +68,7 @@ public Object get() @Override public float getFloat() { - return (float) min; + return min; } @Override @@ -77,6 +77,12 @@ public long getLong() return min; } + @Override + public double getDouble() + { + return min; + } + @Override public String getName() { 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..361b0bddb75b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java @@ -55,9 +55,14 @@ public Object get(ByteBuffer buf, int position) @Override public float getFloat(ByteBuffer buf, int position) { - return (float) buf.getLong(position); + return buf.getLong(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getLong(position); + } @Override public long getLong(ByteBuffer buf, int position) 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..454289cc3e6b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumAggregator.java @@ -75,7 +75,7 @@ public Object get() @Override public float getFloat() { - return (float) sum; + return sum; } @Override @@ -84,6 +84,12 @@ public long getLong() return sum; } + @Override + public double getDouble() + { + return sum; + } + @Override public String getName() { 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..ea8bc00f84b3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/LongSumBufferAggregator.java @@ -57,9 +57,14 @@ public Object get(ByteBuffer buf, int position) @Override public float getFloat(ByteBuffer buf, int position) { - return (float) buf.getLong(position); + return buf.getLong(position); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + return buf.getLong(position); + } @Override public long getLong(ByteBuffer buf, int position) 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..5f48ba363371 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,12 @@ public long getLong() throw new UnsupportedOperationException("CardinalityAggregator does not support getLong()"); } + @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/CardinalityBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/cardinality/CardinalityBufferAggregator.java index c791dc650f43..d6fe12c6d471 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 @@ -89,6 +89,12 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getFloat()"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("CardinalityBufferAggregator does not support getDouble()"); + } + @Override public long getLong(ByteBuffer buf, int position) 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..7f71fc653d7a 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,12 @@ public long getLong() throw new UnsupportedOperationException("HyperUniquesAggregator does not support getLong()"); } + @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/HyperUniquesBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesBufferAggregator.java index 98cba7dce450..d9a9e22558da 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 @@ -89,6 +89,11 @@ public float getFloat(ByteBuffer buf, int position) throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getFloat()"); } + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("HyperUniquesBufferAggregator does not support getDouble()"); + } @Override public long getLong(ByteBuffer buf, int position) diff --git a/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java b/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java index 06c82af8d5f5..7cc73267ee1c 100644 --- a/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java +++ b/processing/src/main/java/io/druid/query/filter/ValueMatcherFactory.java @@ -19,9 +19,6 @@ package io.druid.query.filter; -import com.google.common.base.Predicate; -import io.druid.segment.column.ValueType; - /** * A ValueMatcherFactory is an object associated with a collection of rows (e.g., an IncrementalIndexStorageAdapter) * that generates ValueMatchers for filtering on the associated collection of rows. diff --git a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index a3c7f6e3889f..96ca90c96e73 100644 --- a/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/io/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -44,6 +44,7 @@ import io.druid.query.groupby.strategy.GroupByStrategyV2; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -608,6 +609,19 @@ public float get() }; } + @Override + public DoubleColumnSelector makeDoubleColumnSelector(final String columnName) + { + return new DoubleColumnSelector() + { + @Override + public double get() + { + return row.get().getDoubleMetric(columnName); + } + }; + } + @Override public LongColumnSelector makeLongColumnSelector(final String columnName) { diff --git a/processing/src/main/java/io/druid/query/groupby/having/HavingSpecMetricComparator.java b/processing/src/main/java/io/druid/query/groupby/having/HavingSpecMetricComparator.java index 324962478447..4802c943eff8 100644 --- a/processing/src/main/java/io/druid/query/groupby/having/HavingSpecMetricComparator.java +++ b/processing/src/main/java/io/druid/query/groupby/having/HavingSpecMetricComparator.java @@ -19,6 +19,7 @@ package io.druid.query.groupby.having; +import com.google.common.primitives.Doubles; import com.google.common.primitives.Floats; import com.google.common.primitives.Longs; import io.druid.data.input.Row; @@ -34,20 +35,31 @@ class HavingSpecMetricComparator static int compare(Row row, String aggregationName, Number value) { Object metricValueObj = row.getRaw(aggregationName); - if (metricValueObj != null) { + if (metricValueObj instanceof Number) { if (metricValueObj instanceof Long) { - long l = ((Long) metricValueObj).longValue(); + long l = (Long) metricValueObj; return Longs.compare(l, value.longValue()); - } else if (metricValueObj instanceof String) { + } + if (metricValueObj instanceof Float) { + float f = (Float) metricValueObj; + return Floats.compare(f, value.floatValue()); + } + if (metricValueObj instanceof Double) { + double d = (Double) metricValueObj; + return Doubles.compare(d, value.doubleValue()); + } + } else if (metricValueObj instanceof String) { + if (metricValueObj instanceof String) { String metricValueStr = (String) metricValueObj; - if (LONG_PAT.matcher(metricValueStr).matches()) { - long l = row.getLongMetric(aggregationName); + Long l = Longs.tryParse(metricValueStr); + if (l != null) { return Longs.compare(l, value.longValue()); } } } + // default float f = row.getFloatMetric(aggregationName); - return Floats.compare(f, value.floatValue()); + return Float.compare(f, value.floatValue()); } } 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 406740636376..c7e270ef75fd 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java @@ -30,6 +30,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.logger.Logger; import io.druid.common.utils.StringUtils; +import io.druid.data.ValueType; import io.druid.granularity.QueryGranularities; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.metadata.metadata.ColumnAnalysis; @@ -44,7 +45,6 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ComplexColumn; -import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; @@ -69,6 +69,8 @@ public class SegmentAnalyzer */ private static final int NUM_BYTES_IN_TEXT_FLOAT = 8; + private static final int NUM_BYTES_IN_TEXT_DOUBLE = 12; + private final EnumSet analysisTypes; public SegmentAnalyzer(EnumSet analysisTypes) @@ -112,6 +114,9 @@ public Map analyze(Segment segment) case FLOAT: analysis = analyzeNumericColumn(capabilities, length, NUM_BYTES_IN_TEXT_FLOAT); break; + case DOUBLE: + analysis = analyzeNumericColumn(capabilities, length, NUM_BYTES_IN_TEXT_DOUBLE); + break; case STRING: if (index != null) { analysis = analyzeStringColumn(capabilities, column); diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index ccfbfee55996..c5535ec63b73 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -24,13 +24,12 @@ import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.guava.CloseQuietly; +import io.druid.data.ValueType; import io.druid.query.filter.BitmapIndexSelector; -import io.druid.query.filter.Filter; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedIterable; import io.druid.segment.filter.Filters; diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java index f550fef14e5c..bdd94ce0a175 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorFactory.java @@ -29,6 +29,7 @@ public interface ColumnSelectorFactory { public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec); public FloatColumnSelector makeFloatColumnSelector(String columnName); + public DoubleColumnSelector makeDoubleColumnSelector(String columnName); public LongColumnSelector makeLongColumnSelector(String columnName); public ObjectColumnSelector makeObjectColumnSelector(String columnName); public ColumnCapabilities getColumnCapabilities(String columnName); diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index c4472df2a46a..411d76aaaaa5 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -19,24 +19,13 @@ package io.druid.segment; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.metamx.common.io.smoosh.FileSmoosher; -import io.druid.query.dimension.DimensionSpec; -import io.druid.query.groupby.GroupByQueryEngine; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ValueType; import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedInts; import java.io.Closeable; import java.io.File; -import java.nio.ByteBuffer; -import java.util.Comparator; -import java.util.List; -import java.util.Map; /** * Processing related interface diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java index d265d111fdfc..c3dc5551e512 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtil.java @@ -19,10 +19,9 @@ package io.druid.segment; -import com.google.common.base.Function; import com.metamx.common.IAE; +import io.druid.data.ValueType; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ValueType; public final class DimensionHandlerUtil { 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..d6c5d59f6a8f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java @@ -0,0 +1,8 @@ +package io.druid.segment; + +/** + */ +public interface DoubleColumnSelector +{ + double get(); +} diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java new file mode 100644 index 000000000000..48bcbba74484 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java @@ -0,0 +1,98 @@ +/* + * 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 io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.DoubleSupplierSerializer; +import io.druid.segment.data.IOPeon; + +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class DoubleColumnSerializer implements GenericColumnSerializer +{ + public static DoubleColumnSerializer create( + IOPeon ioPeon, + String filenameBase, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + return new DoubleColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + } + + private final IOPeon ioPeon; + private final String filenameBase; + private final ByteOrder byteOrder; + private final CompressedObjectStrategy.CompressionStrategy compression; + private DoubleSupplierSerializer writer; + + public DoubleColumnSerializer( + IOPeon ioPeon, + String filenameBase, + ByteOrder byteOrder, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.ioPeon = ioPeon; + this.filenameBase = filenameBase; + this.byteOrder = byteOrder; + this.compression = compression; + } + + @Override + public void open() throws IOException + { + writer = CompressionFactory.getDoubleSerializer( + ioPeon, + String.format("%s.float_column", filenameBase), + byteOrder, + compression + ); + writer.open(); + } + + @Override + public void serialize(Object obj) throws IOException + { + float val = (obj == null) ? 0 : ((Number) obj).floatValue(); + writer.add(val); + } + + @Override + public void close() throws IOException + { + writer.close(); + } + + @Override + public long getSerializedSize() + { + return writer.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + writer.writeToChannel(channel); + } + +} 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..cb119fd30ff5 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DoubleMetricColumnSerializer.java @@ -0,0 +1,85 @@ +/* + * 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.FileWriteMode; +import com.google.common.io.Files; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.DoubleSupplierSerializer; +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 final CompressedObjectStrategy.CompressionStrategy compression; + + private DoubleSupplierSerializer writer; + + public DoubleMetricColumnSerializer( + String metricName, + File outDir, + IOPeon ioPeon, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.metricName = metricName; + this.ioPeon = ioPeon; + this.outDir = outDir; + this.compression = compression; + } + + @Override + public void open() throws IOException + { + writer = CompressionFactory.getDoubleSerializer( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, compression + ); + + 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.asByteSink(outFile, FileWriteMode.APPEND), 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 d88984799de9..2a51ddfe3627 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -50,12 +50,12 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import io.druid.common.utils.SerializerUtils; +import io.druid.data.ValueType; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ColumnDescriptor; -import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayIndexed; import io.druid.segment.data.BitmapSerde; import io.druid.segment.data.BitmapSerdeFactory; @@ -77,6 +77,8 @@ 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.DoubleGenericColumnSupplier; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnSupplier; import io.druid.segment.serde.LongGenericColumnPartSerde; @@ -728,6 +730,15 @@ public int size() .build() ); break; + case DOUBLE: + builder.setValueType(ValueType.DOUBLE); + builder.addSerde( + DoubleGenericColumnPartSerde.legacySerializerBuilder() + .withByteOrder(BYTE_ORDER) + .withDelegate(holder.doubleType) + .build() + ); + break; case COMPLEX: if (!(holder.complexType instanceof GenericIndexed)) { throw new ISE("Serialized complex types must be GenericIndexed objects."); @@ -916,6 +927,14 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException .setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType, BYTE_ORDER)) .build() ); + } else if (metricHolder.getType() == MetricHolder.MetricType.DOUBLE) { + columns.put( + metric, + new ColumnBuilder() + .setType(ValueType.DOUBLE) + .setGenericColumn(new DoubleGenericColumnSupplier(metricHolder.doubleType, BYTE_ORDER)) + .build() + ); } else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) { columns.put( metric, diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 8fd2ecb87207..8083498b47d6 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -53,20 +53,19 @@ import io.druid.common.guava.GuavaUtils; import io.druid.common.utils.JodaUtils; import io.druid.common.utils.SerializerUtils; +import io.druid.data.ValueType; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; -import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.ListIndexed; +import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.TmpFileIOPeon; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -731,6 +730,9 @@ public void close() throws IOException case FLOAT: metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon, metCompression)); break; + case DOUBLE: + metWriters.add(new DoubleMetricColumnSerializer(metric, v8OutDir, ioPeon, metCompression)); + break; case COMPLEX: final String typeName = metricTypeNames.get(metric); ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index b504a6db1993..fb78d1999e08 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -30,24 +30,17 @@ import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.inject.Inject; -import com.metamx.collections.bitmap.BitmapFactory; -import com.metamx.collections.bitmap.ImmutableBitmap; -import com.metamx.collections.bitmap.MutableBitmap; -import com.metamx.collections.spatial.ImmutableRTree; -import com.metamx.collections.spatial.RTree; -import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; -import com.metamx.common.ByteBufferUtils; import com.metamx.common.ISE; import com.metamx.common.io.smoosh.FileSmoosher; import com.metamx.common.io.smoosh.SmooshedWriter; import com.metamx.common.logger.Logger; import io.druid.common.utils.JodaUtils; +import io.druid.data.ValueType; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnDescriptor; -import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.GenericIndexed; @@ -57,6 +50,7 @@ import io.druid.segment.serde.ComplexColumnSerializer; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; +import io.druid.segment.serde.DoubleGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde; import org.apache.commons.io.FileUtils; @@ -360,6 +354,15 @@ private void makeMetricsColumns( .build() ); break; + case DOUBLE: + builder.setValueType(ValueType.DOUBLE); + builder.addSerde( + DoubleGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate((DoubleColumnSerializer) writer) + .build() + ); + break; case COMPLEX: final String typeName = metricTypeNames.get(metric); builder.setValueType(ValueType.COMPLEX); @@ -522,6 +525,9 @@ private ArrayList setupMetricsWriters( case FLOAT: writer = FloatColumnSerializer.create(ioPeon, metric, metCompression); break; + case DOUBLE: + writer = DoubleColumnSerializer.create(ioPeon, metric, metCompression); + break; case COMPLEX: final String typeName = metricTypeNames.get(metric); ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 02c23759d6fb..9a75778562ff 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -26,13 +26,16 @@ 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.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedLongsIndexedSupplier; +import io.druid.segment.data.DoubleSupplierSerializer; import io.druid.segment.data.FloatSupplierSerializer; import io.druid.segment.data.LongSupplierSerializer; 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.IndexedLongs; import io.druid.segment.data.ObjectStrategy; @@ -60,6 +63,13 @@ public static MetricHolder floatMetric(String name, CompressedFloatsIndexedSuppl return retVal; } + public static MetricHolder doubleMetric(String name, CompressedDoublesIndexedSupplier column) + { + MetricHolder retVal = new MetricHolder(name, "double"); + retVal.doubleType = column; + return retVal; + } + public static MetricHolder complexMetric(String name, String typeName, Indexed column) { MetricHolder retVal = new MetricHolder(name, typeName); @@ -93,6 +103,16 @@ public static void writeFloatMetric( column.closeAndConsolidate(outSupplier); } + public static void writeDoubleMetric( + final ByteSink outSupplier, String name, DoubleSupplierSerializer column + ) throws IOException + { + outSupplier.write(version); + serializerUtils.writeString(toOutputSupplier(outSupplier), name); + serializerUtils.writeString(toOutputSupplier(outSupplier), "double"); + column.closeAndConsolidate(outSupplier); + } + public static void writeLongMetric( ByteSink outSupplier, String name, LongSupplierSerializer column ) throws IOException @@ -113,6 +133,12 @@ public static void writeToChannel(MetricHolder holder, WritableByteChannel out) case FLOAT: holder.floatType.writeToChannel(out); break; + case DOUBLE: + holder.doubleType.writeToChannel(out); + break; + case LONG: + holder.longType.writeToChannel(out); + break; case COMPLEX: if (holder.complexType instanceof GenericIndexed) { ((GenericIndexed) holder.complexType).writeToChannel(out); @@ -146,6 +172,9 @@ public static MetricHolder fromByteBuffer(ByteBuffer buf, ObjectStrategy strateg case FLOAT: holder.floatType = CompressedFloatsIndexedSupplier.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); @@ -184,6 +213,7 @@ public enum MetricType { LONG, FLOAT, + DOUBLE, COMPLEX; static MetricType determineType(String typeName) @@ -192,6 +222,8 @@ static MetricType determineType(String typeName) return LONG; } else if ("float".equalsIgnoreCase(typeName)) { return FLOAT; + } else if ("double".equalsIgnoreCase(typeName)) { + return DOUBLE; } return COMPLEX; } @@ -199,6 +231,7 @@ static MetricType determineType(String typeName) CompressedLongsIndexedSupplier longType = null; CompressedFloatsIndexedSupplier floatType = null; + CompressedDoublesIndexedSupplier doubleType = null; Indexed complexType = null; private MetricHolder( @@ -238,6 +271,12 @@ public IndexedFloats getFloatType() return floatType.get(); } + public IndexedDoubles getDoubleType() + { + assertType(MetricType.DOUBLE); + return doubleType.get(); + } + public Indexed getComplexType() { assertType(MetricType.COMPLEX); diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index a33e64f5f673..1637b2d878b0 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -27,15 +27,16 @@ import com.metamx.common.ISE; import com.metamx.common.guava.CloseQuietly; import com.metamx.common.logger.Logger; +import io.druid.data.ValueType; import io.druid.segment.column.BitmapIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; +import io.druid.segment.column.IndexedDoublesGenericColumn; import io.druid.segment.column.IndexedFloatsGenericColumn; import io.druid.segment.column.IndexedLongsGenericColumn; -import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapCompressedIndexedInts; import io.druid.segment.data.EmptyIndexedInts; import io.druid.segment.data.Indexed; @@ -212,6 +213,7 @@ public Closeable apply(DimensionHandler handler) final ValueType type = column.getCapabilities().getType(); switch (type) { case FLOAT: + case DOUBLE: case LONG: metrics[i] = column.getGenericColumn(); break; @@ -261,6 +263,8 @@ public Rowboat next() for (int i = 0; i < metricArray.length; ++i) { if (metrics[i] instanceof IndexedFloatsGenericColumn) { metricArray[i] = ((GenericColumn) metrics[i]).getFloatSingleValueRow(currRow); + } else if (metrics[i] instanceof IndexedDoublesGenericColumn) { + metricArray[i] = ((GenericColumn) metrics[i]).getDoubleSingleValueRow(currRow); } else if (metrics[i] instanceof IndexedLongsGenericColumn) { metricArray[i] = ((GenericColumn) metrics[i]).getLongSingleValueRow(currRow); } else if (metrics[i] instanceof ComplexColumn) { @@ -313,6 +317,8 @@ public String getMetricType(String metric) switch (type) { case FLOAT: return "float"; + case DOUBLE: + return "double"; case LONG: return "long"; case COMPLEX: diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java index db724373a51a..91b0606c4491 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexStorageAdapter.java @@ -34,6 +34,7 @@ import com.metamx.common.guava.CloseQuietly; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import io.druid.data.ValueType; import io.druid.granularity.QueryGranularity; import io.druid.query.QueryInterruptedException; import io.druid.query.dimension.DefaultDimensionSpec; @@ -52,7 +53,6 @@ import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.GenericColumn; -import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.Offset; @@ -589,8 +589,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 && ValueType.isNumeric(holder.getCapabilities().getType())) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(columnName, cachedMetricVals); } @@ -618,6 +617,41 @@ public float get() }; } + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + GenericColumn cachedMetricVals = genericColumnCache.get(columnName); + + if (cachedMetricVals == null) { + Column holder = index.getColumn(columnName); + if (holder != null && ValueType.isNumeric(holder.getCapabilities().getType())) { + cachedMetricVals = holder.getGenericColumn(); + genericColumnCache.put(columnName, cachedMetricVals); + } + } + + if (cachedMetricVals == null) { + return new DoubleColumnSelector() + { + @Override + public double get() + { + return 0.0f; + } + }; + } + + final GenericColumn metricVals = cachedMetricVals; + return new DoubleColumnSelector() + { + @Override + public double get() + { + return metricVals.getDoubleSingleValueRow(cursorOffset.getOffset()); + } + }; + } + @Override public LongColumnSelector makeLongColumnSelector(String columnName) { @@ -625,8 +659,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 && ValueType.isNumeric(holder.getCapabilities().getType())) { cachedMetricVals = holder.getGenericColumn(); genericColumnCache.put(columnName, cachedMetricVals); } @@ -710,6 +743,22 @@ public Float get() } }; } + 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.LONG) { return new ObjectColumnSelector() { diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 8d9e5c117941..85b52c8c46bd 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -38,9 +38,9 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import io.druid.collections.CombiningIterable; +import io.druid.data.ValueType; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; -import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedObjectStrategy; diff --git a/processing/src/main/java/io/druid/segment/column/AbstractGenericColumn.java b/processing/src/main/java/io/druid/segment/column/AbstractGenericColumn.java new file mode 100644 index 000000000000..40c4d2d982ff --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/AbstractGenericColumn.java @@ -0,0 +1,67 @@ +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.IndexedLongs; + +import java.io.IOException; + +/** + */ +public abstract class AbstractGenericColumn implements GenericColumn +{ + @Override + public String getStringSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public Indexed getStringMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexedFloats getFloatMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public IndexedLongs getLongMultiValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + throw new UnsupportedOperationException(); + } + + @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/ColumnBuilder.java b/processing/src/main/java/io/druid/segment/column/ColumnBuilder.java index ebbd8efbf09f..80082531f0a4 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnBuilder.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnBuilder.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import io.druid.data.ValueType; /** */ 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..9ef8bc8b1fe9 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilities.java @@ -19,6 +19,8 @@ package io.druid.segment.column; +import io.druid.data.ValueType; + /** */ public interface ColumnCapabilities 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..d991e1ef070c 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnCapabilitiesImpl.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.common.ISE; +import io.druid.data.ValueType; /** */ diff --git a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java index b2c0fb9a0d58..607de0d739d3 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.metamx.common.IAE; +import io.druid.data.ValueType; import io.druid.segment.serde.ColumnPartSerde; import java.io.IOException; diff --git a/processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java b/processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java index 1008559975fc..6272817ba4d2 100644 --- a/processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java +++ b/processing/src/main/java/io/druid/segment/column/ComplexColumnImpl.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.data.ValueType; import io.druid.segment.data.Indexed; /** diff --git a/processing/src/main/java/io/druid/segment/column/DoubleColumn.java b/processing/src/main/java/io/druid/segment/column/DoubleColumn.java new file mode 100644 index 000000000000..4d9959d0cb68 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/DoubleColumn.java @@ -0,0 +1,56 @@ +/* + * 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.data.ValueType; +import io.druid.segment.data.CompressedDoublesIndexedSupplier; + +/** + */ +public class DoubleColumn extends AbstractColumn +{ + private static final ColumnCapabilitiesImpl CAPABILITIES = new ColumnCapabilitiesImpl() + .setType(ValueType.DOUBLE); + + private final CompressedDoublesIndexedSupplier column; + + public DoubleColumn(CompressedDoublesIndexedSupplier column) + { + this.column = column; + } + + @Override + public ColumnCapabilities getCapabilities() + { + return CAPABILITIES; + } + + @Override + public int getLength() + { + return column.size(); + } + + @Override + public GenericColumn getGenericColumn() + { + return new IndexedDoublesGenericColumn(column.get()); + } +} diff --git a/processing/src/main/java/io/druid/segment/column/FloatColumn.java b/processing/src/main/java/io/druid/segment/column/FloatColumn.java index b97c3b33d98a..7b11e6cd4eed 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.data.ValueType; import io.druid.segment.data.CompressedFloatsIndexedSupplier; /** 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..293e95e23930 100644 --- a/processing/src/main/java/io/druid/segment/column/GenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/GenericColumn.java @@ -19,7 +19,9 @@ package io.druid.segment.column; +import io.druid.data.ValueType; import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedDoubles; import io.druid.segment.data.IndexedFloats; import io.druid.segment.data.IndexedLongs; @@ -39,4 +41,6 @@ public interface GenericColumn extends Closeable public IndexedFloats getFloatMultiValueRow(int rowNum); public long getLongSingleValueRow(int rowNum); public IndexedLongs getLongMultiValueRow(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..93a8935fefc8 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/IndexedDoublesGenericColumn.java @@ -0,0 +1,80 @@ +/* + * 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.data.ValueType; +import io.druid.segment.data.IndexedDoubles; + +import java.io.IOException; + +/** +*/ +public class IndexedDoublesGenericColumn extends AbstractGenericColumn +{ + 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 double getDoubleSingleValueRow(int rowNum) + { + return column.get(rowNum); + } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + return (float)column.get(rowNum); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + return (long) column.get(rowNum); + } + + @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..fbf3192006e9 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedFloatsGenericColumn.java @@ -19,15 +19,14 @@ package io.druid.segment.column; -import io.druid.segment.data.Indexed; +import io.druid.data.ValueType; import io.druid.segment.data.IndexedFloats; -import io.druid.segment.data.IndexedLongs; import java.io.IOException; /** */ -public class IndexedFloatsGenericColumn implements GenericColumn +public class IndexedFloatsGenericColumn extends AbstractGenericColumn { private final IndexedFloats column; @@ -55,30 +54,12 @@ 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 column.get(rowNum); } - @Override - public IndexedFloats getFloatMultiValueRow(int rowNum) - { - throw new UnsupportedOperationException(); - } - @Override public long getLongSingleValueRow(int rowNum) { @@ -86,9 +67,9 @@ public long getLongSingleValueRow(int rowNum) } @Override - public IndexedLongs getLongMultiValueRow(int rowNum) + public double getDoubleSingleValueRow(int rowNum) { - throw new UnsupportedOperationException(); + return (double) column.get(rowNum); } @Override 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..3c3929856951 100644 --- a/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/IndexedLongsGenericColumn.java @@ -19,15 +19,14 @@ package io.druid.segment.column; -import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedFloats; +import io.druid.data.ValueType; import io.druid.segment.data.IndexedLongs; import java.io.IOException; /** */ -public class IndexedLongsGenericColumn implements GenericColumn +public class IndexedLongsGenericColumn extends AbstractGenericColumn { private final IndexedLongs column; @@ -55,30 +54,12 @@ 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) { @@ -86,9 +67,9 @@ public long getLongSingleValueRow(int rowNum) } @Override - public IndexedLongs getLongMultiValueRow(int rowNum) + public double getDoubleSingleValueRow(int rowNum) { - throw new UnsupportedOperationException(); + return (double) column.get(rowNum); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/LongColumn.java b/processing/src/main/java/io/druid/segment/column/LongColumn.java index 3e06ea03970e..9f103e6d4883 100644 --- a/processing/src/main/java/io/druid/segment/column/LongColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.data.ValueType; import io.druid.segment.data.CompressedLongsIndexedSupplier; /** diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java new file mode 100644 index 000000000000..caf96fabe4ad --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java @@ -0,0 +1,143 @@ +/* + * 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.ByteSink; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Ints; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidResourceHolder; +import io.druid.segment.CompressedPools; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteOrder; +import java.nio.DoubleBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; + +public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer +{ + private final IOPeon ioPeon; + private final int sizePer; + private final GenericIndexedWriter> flattener; + private final CompressedObjectStrategy.CompressionStrategy compression; + private final String metaFile; + + private long metaCount = 0; + private int numInserted = 0; + private DoubleBuffer endBuffer; + + public BlockLayoutDoubleSupplierSerializer( + IOPeon ioPeon, + String filenameBase, + ByteOrder order, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.ioPeon = ioPeon; + this.sizePer = CompressedPools.BUFFER_SIZE / Doubles.BYTES; + this.flattener = new GenericIndexedWriter<>( + ioPeon, filenameBase, CompressedDoubleBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + ); + this.metaFile = filenameBase + ".format"; + this.compression = compression; + + endBuffer = DoubleBuffer.allocate(sizePer); + endBuffer.mark(); + } + + @Override + public void open() throws IOException + { + flattener.open(); + } + + @Override + public int size() + { + return numInserted; + } + + @Override + 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; + } + + @Override + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + { + close(); + try (OutputStream out = consolidatedOut.openStream(); + InputStream meta = ioPeon.makeInputStream(metaFile)) { + ByteStreams.copy(meta, out); + ByteStreams.copy(flattener.combineStreams(), out); + } + } + + @Override + public void close() throws IOException + { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = null; + flattener.close(); + + try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { + metaOut.write(CompressedDoublesIndexedSupplier.version); + metaOut.write(Ints.toByteArray(numInserted)); + metaOut.write(Ints.toByteArray(sizePer)); + metaOut.write(compression.getId()); + metaOut.close(); + metaCount = metaOut.getCount(); + } + } + + @Override + public long getSerializedSize() + { + return metaCount + flattener.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + try (InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream input = flattener.combineStreams().getInput()) { + ByteStreams.copy(Channels.newChannel(meta), channel); + final ReadableByteChannel from = Channels.newChannel(input); + ByteStreams.copy(from, channel); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java new file mode 100644 index 000000000000..c97be355feaf --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java @@ -0,0 +1,148 @@ +/* + * 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.Supplier; +import com.google.common.io.Closeables; +import com.google.common.primitives.Doubles; +import com.metamx.common.guava.CloseQuietly; +import io.druid.collections.ResourceHolder; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.DoubleBuffer; + +public class BlockLayoutIndexedDoubleSupplier implements Supplier +{ + private final GenericIndexed> baseDoubleBuffers; + private final int totalSize; + private final int sizePer; + + public BlockLayoutIndexedDoubleSupplier( + int totalSize, int sizePer, ByteBuffer fromBuffer, ByteOrder order, + CompressedObjectStrategy.CompressionStrategy strategy + ) + { + baseDoubleBuffers = GenericIndexed.read(fromBuffer, VSizeCompressedObjectStrategy.getBufferForOrder( + order, strategy, sizePer * Doubles.BYTES + )); + this.totalSize = totalSize; + this.sizePer = sizePer; + } + + @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 BlockLayoutIndexedDoubles() + { + @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 doubleBuffer.get(doubleBuffer.position() + bufferIndex); + } + }; + } else { + return new BlockLayoutIndexedDoubles(); + } + } + + private class BlockLayoutIndexedDoubles implements IndexedDoubles + { + final Indexed> singleThreadedDoubleBuffers = baseDoubleBuffers.singleThreaded(); + int currIndex = -1; + ResourceHolder holder; + ByteBuffer buffer; + DoubleBuffer doubleBuffer; + + @Override + public int size() + { + return totalSize; + } + + @Override + public double get(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 doubleBuffer.get(doubleBuffer.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 + ) + ); + } + for (int i = 0; i < toFill.length; i++) { + toFill[i] = get(index + i); + } + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = singleThreadedDoubleBuffers.get(bufferNum); + buffer = holder.get(); + doubleBuffer = buffer.asDoubleBuffer(); + currIndex = bufferNum; + } + + @Override + public String toString() + { + return "BlockCompressedIndexedDoubles_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/CompressedDoubleBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java new file mode 100644 index 000000000000..4c4cf10d80f3 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java @@ -0,0 +1,76 @@ +/* + * 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..0c35dc611294 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java @@ -0,0 +1,307 @@ +/* + * 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); + } + + /** + * 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/CompressionFactory.java b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java index 1f0958f7cf52..353824d3f2c0 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -330,4 +330,31 @@ public static FloatSupplierSerializer getFloatSerializer( } } + public static Supplier getDoubleSupplier( + int totalSize, int sizePer, ByteBuffer fromBuffer, ByteOrder order, + CompressedObjectStrategy.CompressionStrategy strategy + ) + { + if (strategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + return new EntireLayoutIndexedDoubleSupplier(totalSize, fromBuffer, order); + } else { + return new BlockLayoutIndexedDoubleSupplier(totalSize, sizePer, fromBuffer, order, strategy); + } + } + + public static DoubleSupplierSerializer getDoubleSerializer( + IOPeon ioPeon, String filenameBase, ByteOrder order, + CompressedObjectStrategy.CompressionStrategy compressionStrategy + ) + { + if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + return new EntireLayoutDoubleSupplierSerializer( + ioPeon, filenameBase, order + ); + } else{ + return new BlockLayoutDoubleSupplierSerializer( + ioPeon, filenameBase, order, compressionStrategy + ); + } + } } diff --git a/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java new file mode 100644 index 000000000000..9026fcf0ffd6 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java @@ -0,0 +1,36 @@ +/* + * 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.ByteSink; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public interface DoubleSupplierSerializer extends Closeable +{ + void open() throws IOException; + int size(); + void add(double value) throws IOException; + void closeAndConsolidate(ByteSink consolidatedOut) throws IOException; + long getSerializedSize(); + void writeToChannel(WritableByteChannel channel) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java new file mode 100644 index 000000000000..9448d5220142 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java @@ -0,0 +1,122 @@ +/* + * 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.ByteSink; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import com.google.common.primitives.Doubles; +import com.google.common.primitives.Ints; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; + +public class EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer +{ + private final IOPeon ioPeon; + private final String valueFile; + private final String metaFile; + private CountingOutputStream valuesOut; + private long metaCount = 0; + + private final ByteBuffer orderBuffer; + + private int numInserted = 0; + + public EntireLayoutDoubleSupplierSerializer( + IOPeon ioPeon, String filenameBase, ByteOrder order + ) + { + this.ioPeon = ioPeon; + this.valueFile = filenameBase + ".value"; + this.metaFile = filenameBase + ".format"; + + orderBuffer = ByteBuffer.allocate(Doubles.BYTES); + orderBuffer.order(order); + } + + @Override + public void open() throws IOException + { + valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + } + + @Override + public int size() + { + return numInserted; + } + + @Override + public void add(double value) throws IOException + { + orderBuffer.rewind(); + orderBuffer.putDouble(value); + valuesOut.write(orderBuffer.array()); + ++numInserted; + } + + @Override + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + { + close(); + try (OutputStream out = consolidatedOut.openStream(); + InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream value = ioPeon.makeInputStream(valueFile)) { + ByteStreams.copy(meta, out); + ByteStreams.copy(value, out); + } + } + + @Override + public void close() throws IOException + { + valuesOut.close(); + try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { + metaOut.write(CompressedDoublesIndexedSupplier.version); + metaOut.write(Ints.toByteArray(numInserted)); + metaOut.write(Ints.toByteArray(0)); + metaOut.write(CompressedObjectStrategy.CompressionStrategy.NONE.getId()); + metaOut.close(); + metaCount = metaOut.getCount(); + } + } + + @Override + public long getSerializedSize() + { + return metaCount + valuesOut.getCount(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + try (InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream value = ioPeon.makeInputStream(valueFile)) { + ByteStreams.copy(Channels.newChannel(meta), channel); + ByteStreams.copy(Channels.newChannel(value), channel); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedDoubleSupplier.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedDoubleSupplier.java new file mode 100644 index 000000000000..f6b62c8fa7d1 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedDoubleSupplier.java @@ -0,0 +1,89 @@ +/* + * 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.Supplier; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.DoubleBuffer; + +public class EntireLayoutIndexedDoubleSupplier implements Supplier +{ + private final int totalSize; + private DoubleBuffer buffer; + + public EntireLayoutIndexedDoubleSupplier(int totalSize, ByteBuffer fromBuffer, ByteOrder order) + { + this.totalSize = totalSize; + this.buffer = fromBuffer.asReadOnlyBuffer().order(order).asDoubleBuffer(); + } + + @Override + public IndexedDoubles get() + { + return new EntireLayoutIndexedDoubles(); + } + + private class EntireLayoutIndexedDoubles implements IndexedDoubles + { + + @Override + public int size() + { + return totalSize; + } + + @Override + public double get(int index) + { + return buffer.get(buffer.position() + index); + } + + @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 + ) + ); + } + for (int i = 0; i < toFill.length; i++) { + toFill[i] = get(index + i); + } + } + + @Override + public String toString() + { + return "EntireCompressedIndexedDoubles_Anonymous{" + + ", totalSize=" + totalSize + + '}'; + } + + @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/data/IndexedDoubles.java similarity index 79% rename from processing/src/main/java/io/druid/segment/column/ValueType.java rename to processing/src/main/java/io/druid/segment/data/IndexedDoubles.java index aa9919a129b9..3d578ce2a42c 100644 --- a/processing/src/main/java/io/druid/segment/column/ValueType.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedDoubles.java @@ -17,14 +17,15 @@ * under the License. */ -package io.druid.segment.column; +package io.druid.segment.data; + +import java.io.Closeable; /** -*/ -public enum ValueType + */ +public interface IndexedDoubles extends Closeable { - FLOAT, - LONG, - STRING, - COMPLEX + 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/filter/Filters.java b/processing/src/main/java/io/druid/segment/filter/Filters.java index c995055e39d6..a51e3ac1097a 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -26,9 +26,8 @@ import com.google.common.collect.Lists; import com.google.common.primitives.Longs; import com.metamx.collections.bitmap.ImmutableBitmap; -import com.metamx.common.IAE; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.common.parsers.ParseException; +import io.druid.data.ValueType; import io.druid.query.Query; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.filter.BooleanFilter; @@ -36,16 +35,11 @@ import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.Filter; import io.druid.query.filter.ValueMatcher; -import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.LongColumnSelector; import io.druid.segment.column.BitmapIndex; -import io.druid.segment.column.Column; -import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; -import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import java.util.ArrayList; -import java.util.Arrays; import java.util.Iterator; import java.util.List; diff --git a/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java index bd5fdaaf54e9..760d3a98b2ce 100644 --- a/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java +++ b/processing/src/main/java/io/druid/segment/filter/JavaScriptFilter.java @@ -20,17 +20,12 @@ package io.druid.segment.filter; import com.google.common.base.Predicate; -import com.google.common.base.Strings; import com.metamx.collections.bitmap.ImmutableBitmap; import io.druid.query.filter.BitmapIndexSelector; -import io.druid.query.filter.DruidLongPredicate; import io.druid.query.filter.Filter; import io.druid.query.filter.JavaScriptDimFilter; -import io.druid.query.filter.RowOffsetMatcherFactory; import io.druid.query.filter.ValueMatcher; import io.druid.query.filter.ValueMatcherFactory; -import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ValueType; import org.mozilla.javascript.Context; public class JavaScriptFilter implements Filter 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 c8a2fe9b3654..3a73a4d1f8e3 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -32,6 +32,7 @@ import com.google.common.primitives.Longs; import com.metamx.common.IAE; import com.metamx.common.ISE; +import io.druid.data.ValueType; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; @@ -48,6 +49,7 @@ import io.druid.segment.DimensionHandlerUtil; import io.druid.segment.DimensionIndexer; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.Metadata; @@ -55,7 +57,6 @@ import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; -import io.druid.segment.column.ValueType; import io.druid.segment.data.IndexedInts; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; @@ -93,12 +94,13 @@ public abstract class IncrementalIndex implements Iterable, // Also used to convert between the duplicate ValueType enums in DimensionSchema (druid-api) and main druid. private static final Map TYPE_MAP = ImmutableMap.builder() .put(Long.class, ValueType.LONG) - .put(Double.class, ValueType.FLOAT) .put(Float.class, ValueType.FLOAT) + .put(Double.class, ValueType.DOUBLE) .put(String.class, ValueType.STRING) - .put(DimensionSchema.ValueType.LONG, ValueType.LONG) - .put(DimensionSchema.ValueType.FLOAT, ValueType.FLOAT) - .put(DimensionSchema.ValueType.STRING, ValueType.STRING) + .put(ValueType.LONG, ValueType.LONG) + .put(ValueType.FLOAT, ValueType.FLOAT) + .put(ValueType.DOUBLE, ValueType.DOUBLE) + .put(ValueType.STRING, ValueType.STRING) .build(); public static ColumnSelectorFactory makeColumnSelectorFactory( @@ -147,32 +149,40 @@ public float get() } @Override - public ObjectColumnSelector makeObjectColumnSelector(final String column) + public DoubleColumnSelector makeDoubleColumnSelector(final String columnName) { - final String typeName = agg.getTypeName(); - - final ObjectColumnSelector rawColumnSelector = new ObjectColumnSelector() + return new DoubleColumnSelector() { @Override - public Class classOfObject() + public double get() { - return Object.class; + return in.get().getDoubleMetric(columnName); } + }; + } - @Override - public Object get() + @Override + public ObjectColumnSelector makeObjectColumnSelector(final String column) + { + final String typeName = agg.getInputTypeName(); + final ValueType type = ValueType.of(typeName); + + if (type != ValueType.COMPLEX || !deserializeComplexMetrics) { + return new ObjectColumnSelector() { - return in.get().getRaw(column); - } - }; + @Override + public Class classOfObject() + { + return type.classOfObject(); + } - if (!deserializeComplexMetrics) { - return rawColumnSelector; + @Override + public Object get() + { + return in.get().getRaw(column); + } + }; } else { - if (typeName.equals("float")) { - return rawColumnSelector; - } - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); if (serde == null) { throw new ISE("Don't know how to handle type[%s]", typeName); @@ -426,6 +436,8 @@ protected abstract Integer addToFacts( protected abstract float getMetricFloatValue(int rowOffset, int aggOffset); + protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); + protected abstract long getMetricLongValue(int rowOffset, int aggOffset); protected abstract Object getMetricObjectValue(int rowOffset, int aggOffset); @@ -904,6 +916,9 @@ public MetricDesc(int index, AggregatorFactory factory) if (typeInfo.equalsIgnoreCase("float")) { capabilities.setType(ValueType.FLOAT); this.type = typeInfo; + } else if (typeInfo.equalsIgnoreCase("double")) { + capabilities.setType(ValueType.DOUBLE); + this.type = typeInfo; } else if (typeInfo.equalsIgnoreCase("long")) { capabilities.setType(ValueType.LONG); this.type = typeInfo; 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 0af3152c4ca1..6e8c14a5eff1 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; +import io.druid.data.ValueType; import io.druid.granularity.QueryGranularity; import io.druid.query.QueryInterruptedException; import io.druid.query.dimension.DefaultDimensionSpec; @@ -41,6 +42,7 @@ import io.druid.segment.DimensionHandler; import io.druid.segment.DimensionIndexer; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.Metadata; @@ -50,7 +52,6 @@ import io.druid.segment.StorageAdapter; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.ValueType; import io.druid.segment.data.Indexed; import io.druid.segment.data.ListIndexed; import io.druid.segment.filter.BooleanValueMatcher; @@ -399,6 +400,32 @@ public float get() }; } + @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 LongColumnSelector makeLongColumnSelector(String columnName) { diff --git a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java index 9649a6206d35..e424ec04581b 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -341,6 +341,15 @@ public float getMetricFloatValue(int rowOffset, int aggOffset) return agg.getFloat(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } + @Override + protected double getMetricDoubleValue(int rowOffset, int aggOffset) + { + BufferAggregator agg = getAggs()[aggOffset]; + int[] indexAndOffset = indexAndOffsets.get(rowOffset); + ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); + return agg.getDouble(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + } + @Override public long getMetricLongValue(int rowOffset, int aggOffset) { 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 3080aa713e26..f9e756f8235e 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -31,6 +31,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -302,6 +303,12 @@ public float getMetricFloatValue(int rowOffset, int aggOffset) return concurrentGet(rowOffset)[aggOffset].getFloat(); } + @Override + protected double getMetricDoubleValue(int rowOffset, int aggOffset) + { + return concurrentGet(rowOffset)[aggOffset].getDouble(); + } + @Override public long getMetricLongValue(int rowOffset, int aggOffset) { @@ -336,6 +343,7 @@ static class ObjectCachingColumnSelectorFactory implements ColumnSelectorFactory { private final ConcurrentMap longColumnSelectorMap = Maps.newConcurrentMap(); private final ConcurrentMap floatColumnSelectorMap = Maps.newConcurrentMap(); + private final ConcurrentMap doubleColumnSelectorMap = Maps.newConcurrentMap(); private final ConcurrentMap objectColumnSelectorMap = Maps.newConcurrentMap(); private final ColumnSelectorFactory delegate; @@ -366,6 +374,22 @@ public FloatColumnSelector makeFloatColumnSelector(String columnName) } } + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + DoubleColumnSelector existing = doubleColumnSelectorMap.get(columnName); + if (existing != null) { + return existing; + } else { + DoubleColumnSelector newSelector = delegate.makeDoubleColumnSelector(columnName); + DoubleColumnSelector prev = doubleColumnSelectorMap.putIfAbsent( + columnName, + newSelector + ); + return prev != null ? prev : newSelector; + } + } + @Override public LongColumnSelector makeLongColumnSelector(String columnName) { 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 c42823ca61de..afcfbc70653f 100644 --- a/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java +++ b/processing/src/main/java/io/druid/segment/incremental/SpatialDimensionRowTransformer.java @@ -153,6 +153,17 @@ public float getFloatMetric(String metric) } } + @Override + public double getDoubleMetric(String metric) + { + try { + return row.getDoubleMetric(metric); + } + catch (ParseException e) { + throw Throwables.propagate(e); + } + } + @Override public String toString() { 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 09b8ffc54afd..1ab4a45fe76f 100644 --- a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -34,6 +34,7 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "complex", value = ComplexColumnPartSerde.class), @JsonSubTypes.Type(name = "float", value = FloatGenericColumnPartSerde.class), + @JsonSubTypes.Type(name = "double", value = DoubleGenericColumnPartSerde.class), @JsonSubTypes.Type(name = "long", value = LongGenericColumnPartSerde.class), @JsonSubTypes.Type(name = "stringDictionary", value = DictionaryEncodedColumnPartSerde.class) }) diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 650443da2458..b000c9c64008 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -26,11 +26,11 @@ import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; +import io.druid.data.ValueType; import io.druid.segment.CompressedVSizeIndexedSupplier; import io.druid.segment.CompressedVSizeIndexedV3Supplier; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; -import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapSerde; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferSerializer; 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..1bebfb072f46 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -0,0 +1,172 @@ +/* + * 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.data.ValueType; +import io.druid.segment.DoubleColumnSerializer; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; +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(byteOrder, null); + } + + private final ByteOrder byteOrder; + private Serializer serializer; + + private DoubleGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) + { + this.byteOrder = byteOrder; + this.serializer = serializer; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + public static SerializerBuilder serializerBuilder() + { + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private ByteOrder byteOrder = null; + private DoubleColumnSerializer delegate = null; + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withDelegate(final DoubleColumnSerializer delegate) + { + this.delegate = delegate; + return this; + } + + public DoubleGenericColumnPartSerde build() + { + return new DoubleGenericColumnPartSerde( + byteOrder, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } + } + + public static LegacySerializerBuilder legacySerializerBuilder() + { + return new LegacySerializerBuilder(); + } + + public static class LegacySerializerBuilder + { + private ByteOrder byteOrder = null; + private CompressedDoublesIndexedSupplier delegate = null; + + public LegacySerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public LegacySerializerBuilder withDelegate(final CompressedDoublesIndexedSupplier delegate) + { + this.delegate = delegate; + return this; + } + + public DoubleGenericColumnPartSerde build() + { + return new DoubleGenericColumnPartSerde( + byteOrder, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } + } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + public Deserializer getDeserializer() + { + return new Deserializer() + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final CompressedDoublesIndexedSupplier column = CompressedDoublesIndexedSupplier.fromByteBuffer( + buffer, + byteOrder + ); + builder.setType(ValueType.DOUBLE) + .setHasMultipleValues(false) + .setGenericColumn(new DoubleGenericColumnSupplier(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/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index 503b460027e8..8ac8e5522278 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -21,10 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.ValueType; import io.druid.segment.FloatColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; -import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedFloatsIndexedSupplier; import java.io.IOException; diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index 707050855a16..a8497571cced 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -21,10 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.data.ValueType; import io.druid.segment.LongColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; -import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedLongsIndexedSupplier; import java.io.IOException; 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..1d7e1c8077ee 100644 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleSumAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleSumAggregatorTest.java @@ -39,7 +39,7 @@ public void testAggregate() { final float[] values = {0.15f, 0.27f}; final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - DoubleSumAggregator agg = new DoubleSumAggregator("billy", selector); + DoubleSumAggregator agg = new DoubleSumAggregator.FloatInput("billy", selector); Assert.assertEquals("billy", agg.getName()); @@ -63,7 +63,7 @@ public void testAggregate() public void testComparator() { final TestFloatColumnSelector selector = new TestFloatColumnSelector(new float[]{0.15f, 0.27f}); - DoubleSumAggregator agg = new DoubleSumAggregator("billy", selector); + DoubleSumAggregator agg = new DoubleSumAggregator.FloatInput("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 73523879b22e..d4add8341146 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -20,6 +20,7 @@ package io.druid.query.aggregation; import com.google.common.collect.Lists; +import io.druid.data.ValueType; import io.druid.js.JavaScriptConfig; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; @@ -38,12 +39,12 @@ import io.druid.query.search.search.ContainsSearchQuerySpec; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; -import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.IndexedInts; import org.junit.Assert; @@ -161,6 +162,12 @@ public FloatColumnSelector makeFloatColumnSelector(String columnName) } } + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + throw new UnsupportedOperationException(); + } + @Override public ObjectColumnSelector makeObjectColumnSelector(String columnName) { 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 87c5bc3e318d..a810511afa58 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorBenchmark.java @@ -63,7 +63,7 @@ protected void setUp() throws Exception ) ); - doubleAgg = new DoubleSumAggregator("billy", selector); + doubleAgg = new DoubleSumAggregator.FloatInput("billy", selector); } public double timeJavaScriptDoubleSum(int reps) 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 fb8004e89ed8..90a919d97b12 100644 --- a/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/JavaScriptAggregatorTest.java @@ -26,6 +26,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -59,6 +60,12 @@ public FloatColumnSelector makeFloatColumnSelector(String columnName) return null; } + @Override + public DoubleColumnSelector makeDoubleColumnSelector(String columnName) + { + return null; + } + @Override public LongColumnSelector makeLongColumnSelector(String columnName) { @@ -345,7 +352,7 @@ public static void main(String... args) throws Exception ) ); - DoubleSumAggregator doubleAgg = new DoubleSumAggregator("billy", selector); + DoubleSumAggregator doubleAgg = new DoubleSumAggregator.FloatInput("billy", selector); // warmup int i = 0; 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/groupby/epinephelinae/TestColumnSelectorFactory.java b/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java index 77c57c0bf3b7..0707ed83f782 100644 --- a/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java +++ b/processing/src/test/java/io/druid/query/groupby/epinephelinae/TestColumnSelectorFactory.java @@ -23,6 +23,7 @@ import io.druid.query.dimension.DimensionSpec; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.DimensionSelector; +import io.druid.segment.DoubleColumnSelector; import io.druid.segment.FloatColumnSelector; import io.druid.segment.LongColumnSelector; import io.druid.segment.ObjectColumnSelector; @@ -56,6 +57,19 @@ public float get() }; } + @Override + public DoubleColumnSelector makeDoubleColumnSelector(final String columnName) + { + return new DoubleColumnSelector() + { + @Override + public double get() + { + return row.get().getDoubleMetric(columnName); + } + }; + } + @Override public LongColumnSelector makeLongColumnSelector(final String columnName) { diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java index e1e5c32d7f89..bfb8cc7f9ac1 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentAnalyzerTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; +import io.druid.data.ValueType; import io.druid.query.LegacyDataSource; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -33,7 +34,6 @@ import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; import io.druid.segment.TestIndex; -import io.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java index 24da3e4b9b14..da3176a790a3 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import io.druid.data.ValueType; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CacheStrategy; import io.druid.query.TableDataSource; @@ -36,7 +37,6 @@ import io.druid.query.metadata.metadata.SegmentAnalysis; import io.druid.query.metadata.metadata.SegmentMetadataQuery; import io.druid.query.spec.QuerySegmentSpecs; -import io.druid.segment.column.ValueType; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index 63dff115fbe9..315b011931b6 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -28,6 +28,7 @@ import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.guava.Sequences; import io.druid.common.utils.JodaUtils; +import io.druid.data.ValueType; import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; @@ -51,7 +52,6 @@ import io.druid.segment.QueryableIndexSegment; import io.druid.segment.TestHelper; import io.druid.segment.TestIndex; -import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndex; import io.druid.timeline.LogicalSegment; import org.joda.time.Interval; diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java index 929fdfcb506e..81a36ca87fe4 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataUnionQueryTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequences; +import io.druid.data.ValueType; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; @@ -36,7 +37,6 @@ import io.druid.segment.QueryableIndexSegment; import io.druid.segment.TestHelper; import io.druid.segment.TestIndex; -import io.druid.segment.column.ValueType; import org.joda.time.Interval; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 0b3055bcd022..30a099cdd883 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -26,13 +26,14 @@ import com.google.common.io.LineProcessor; import com.google.common.io.Resources; import com.metamx.common.logger.Logger; +import io.druid.data.ValueType; import io.druid.data.input.impl.DelimitedParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.aggregation.GenericSumAggregatorFactory; 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"); public static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ - new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]), + new GenericSumAggregatorFactory(METRICS[0], METRICS[0], ValueType.FLOAT), new HyperUniquesAggregatorFactory("quality_uniques", "quality") }; private static final IndexSpec indexSpec = new IndexSpec(); 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..e8008ecbd315 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 @@ -283,6 +283,12 @@ public float getFloatMetric(String metric) return metrics.get(metric); } + @Override + public double getDoubleMetric(String metric) + { + return metrics.get(metric); + } + @Override public long getLongMetric(String metric) { 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..d85dae4d865f 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -95,6 +95,12 @@ public float getFloatMetric(String metric) return metricValue; } + @Override + public double getDoubleMetric(String metric) + { + return metricValue; + } + @Override public long getLongMetric(String metric) { 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 23fc6a71b2ee..7f7616ea1ffd 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -747,6 +747,12 @@ public float getFloatMetric(String metric) return 0; } + @Override + public double getDoubleMetric(String metric) + { + return 0; + } + @Override public long getLongMetric(String metric) { 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 b604ae713e86..89b4784a138b 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 @@ -606,6 +606,12 @@ public float getFloatMetric(String metric) return 0; } + @Override + public double getDoubleMetric(String metric) + { + return 0; + } + @Override public long getLongMetric(String metric) { @@ -660,6 +666,12 @@ public float getFloatMetric(String metric) return 0; } + @Override + public double getDoubleMetric(String metric) + { + return 0; + } + @Override public long getLongMetric(String metric) { 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 a086c4ed849b..4ea77a414459 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 @@ -115,6 +115,12 @@ public float getFloatMetric(String metric) return 0; } + @Override + public double getDoubleMetric(String metric) + { + return 0; + } + @Override public long getLongMetric(String metric) { @@ -174,6 +180,12 @@ public float getFloatMetric(String metric) return 0; } + @Override + public double getDoubleMetric(String metric) + { + return 0; + } + @Override public long getLongMetric(String metric) { diff --git a/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java b/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java index 4466f7ef737e..62193c46d50a 100644 --- a/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/io/druid/timeline/partition/HashBasedNumberedShardSpecTest.java @@ -239,6 +239,12 @@ public float getFloatMetric(String s) return 0; } + @Override + public double getDoubleMetric(String metric) + { + return 0; + } + @Override public long getLongMetric(String s) {