From 226794ffc7fbff200b8cb25d89ef73d11bcc8e05 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 23 Jan 2018 00:08:45 +0530 Subject: [PATCH 01/19] SQL compatible Null Handling Part - Expressions, Storage and Dimension Selector Changes fix travis strict compilation --- .../antlr4/io/druid/math/expr/antlr/Expr.g4 | 3 +- .../io/druid/common/config/NullHandling.java | 106 ++++++++++ .../config/NullValueHandlingConfig.java | 43 ++++ .../main/java/io/druid/math/expr/Expr.java | 14 +- .../java/io/druid/math/expr/ExprEval.java | 11 +- .../io/druid/math/expr/ExprListenerImpl.java | 6 + .../java/io/druid/math/expr/Function.java | 87 +++++++- .../java/io/druid/math/expr/EvalTest.java | 8 +- .../java/io/druid/math/expr/FunctionTest.java | 22 +- .../io/druid/segment/MapVirtualColumn.java | 6 + .../VarianceAggregatorCollectorTest.java | 7 + .../druid/java/util/common/StringUtils.java | 38 +++- .../query/aggregation/AggregateCombiner.java | 6 + .../druid/query/aggregation/Aggregator.java | 13 ++ .../query/aggregation/AggregatorUtil.java | 21 ++ .../query/aggregation/BufferAggregator.java | 19 ++ .../aggregation/ObjectAggregateCombiner.java | 5 + .../RowBasedColumnSelectorFactory.java | 32 ++- .../BaseDoubleColumnValueSelector.java | 2 +- .../segment/BaseFloatColumnValueSelector.java | 2 +- .../segment/BaseLongColumnValueSelector.java | 2 +- .../BaseNullableColumnValueSelector.java | 28 +++ .../ColumnSelectorBitmapIndexSelector.java | 8 +- .../segment/ConstantColumnValueSelector.java | 6 + .../segment/ConstantDimensionSelector.java | 4 +- .../druid/segment/DimensionHandlerUtils.java | 28 ++- .../io/druid/segment/DimensionSelector.java | 12 ++ .../druid/segment/DoubleColumnSelector.java | 3 + .../segment/DoubleColumnSerializerV2.java | 134 ++++++++++++ .../druid/segment/DoubleDimensionIndexer.java | 24 ++- .../segment/DoubleDimensionMergerV9.java | 22 +- .../segment/FloatColumnSerializerV2.java | 128 +++++++++++ .../druid/segment/FloatDimensionIndexer.java | 20 +- .../druid/segment/FloatDimensionMergerV9.java | 19 +- .../main/java/io/druid/segment/IndexIO.java | 25 ++- .../java/io/druid/segment/IndexMerger.java | 26 +-- .../java/io/druid/segment/IndexMergerV9.java | 198 ++++++++++++++---- .../druid/segment/LongColumnSerializerV2.java | 142 +++++++++++++ .../druid/segment/LongDimensionIndexer.java | 21 +- .../druid/segment/LongDimensionMergerV9.java | 22 +- .../druid/segment/NilColumnValueSelector.java | 6 + .../druid/segment/NullDimensionSelector.java | 4 +- .../druid/segment/ObjectColumnSelector.java | 13 ++ .../QueryableIndexIndexableAdapter.java | 11 +- .../druid/segment/StringDimensionHandler.java | 16 +- .../druid/segment/StringDimensionIndexer.java | 79 ++++--- .../segment/StringDimensionMergerV9.java | 15 +- .../druid/segment/column/DoublesColumn.java | 13 +- .../io/druid/segment/column/FloatsColumn.java | 16 +- .../druid/segment/column/GenericColumn.java | 3 + .../io/druid/segment/column/LongsColumn.java | 16 +- .../column/SimpleDictionaryEncodedColumn.java | 18 +- .../druid/segment/data/ColumnarDoubles.java | 9 +- .../io/druid/segment/data/ColumnarFloats.java | 9 +- .../io/druid/segment/data/ColumnarLongs.java | 9 +- .../io/druid/segment/data/GenericIndexed.java | 38 ++-- .../segment/data/GenericIndexedWriter.java | 5 +- .../io/druid/segment/data/ObjectStrategy.java | 11 +- .../data/RoaringBitmapSerdeFactory.java | 3 + .../java/io/druid/segment/filter/Filters.java | 3 +- .../segment/incremental/IncrementalIndex.java | 33 ++- ...IncrementalIndexColumnSelectorFactory.java | 6 + .../incremental/OffheapIncrementalIndex.java | 9 + .../incremental/OnheapIncrementalIndex.java | 6 + .../serde/BitmapIndexColumnPartSupplier.java | 3 +- .../druid/segment/serde/ColumnPartSerde.java | 5 +- .../serde/DoubleGenericColumnPartSerde.java | 9 +- .../serde/DoubleGenericColumnPartSerdeV2.java | 162 ++++++++++++++ .../serde/DoubleGenericColumnSupplier.java | 10 +- .../serde/FloatGenericColumnPartSerde.java | 12 +- .../serde/FloatGenericColumnPartSerdeV2.java | 161 ++++++++++++++ .../serde/FloatGenericColumnSupplier.java | 10 +- .../serde/LongGenericColumnPartSerde.java | 12 +- .../serde/LongGenericColumnPartSerdeV2.java | 159 ++++++++++++++ .../serde/LongGenericColumnSupplier.java | 7 +- .../ExpressionColumnValueSelector.java | 6 + .../segment/virtual/ExpressionSelectors.java | 12 +- ...tCachingExpressionColumnValueSelector.java | 6 + ...tCachingExpressionColumnValueSelector.java | 6 + .../aggregation/MetricManipulatorFnsTest.java | 12 ++ .../TestDoubleColumnSelectorImpl.java | 6 + .../aggregation/TestFloatColumnSelector.java | 6 + .../aggregation/TestLongColumnSelector.java | 6 + .../TestColumnSelectorFactory.java | 8 + .../segment/IndexMergerNullHandlingTest.java | 25 ++- .../IncrementalIndexStorageAdapterTest.java | 11 +- .../incremental/IncrementalIndexTest.java | 1 + .../virtual/ExpressionVirtualColumnTest.java | 66 ++++-- .../segment/virtual/VirtualColumnsTest.java | 12 ++ .../main/java/io/druid/cli/DumpSegment.java | 32 +-- 90 files changed, 2139 insertions(+), 300 deletions(-) create mode 100644 common/src/main/java/io/druid/common/config/NullHandling.java create mode 100644 common/src/main/java/io/druid/common/config/NullValueHandlingConfig.java create mode 100644 processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java create mode 100644 processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java create mode 100644 processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java create mode 100644 processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java create mode 100644 processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java create mode 100644 processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java create mode 100644 processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java diff --git a/common/src/main/antlr4/io/druid/math/expr/antlr/Expr.g4 b/common/src/main/antlr4/io/druid/math/expr/antlr/Expr.g4 index c3f88029b031..c3547e70d2ed 100644 --- a/common/src/main/antlr4/io/druid/math/expr/antlr/Expr.g4 +++ b/common/src/main/antlr4/io/druid/math/expr/antlr/Expr.g4 @@ -1,6 +1,7 @@ grammar Expr; -expr : ('-'|'!') expr # unaryOpExpr +expr : 'null' # null + | ('-'|'!') expr # unaryOpExpr | expr '^' expr # powOpExpr | expr ('*'|'/'|'%') expr # mulDivModuloExpr | expr ('+'|'-') expr # addSubExpr diff --git a/common/src/main/java/io/druid/common/config/NullHandling.java b/common/src/main/java/io/druid/common/config/NullHandling.java new file mode 100644 index 000000000000..75df6e980e73 --- /dev/null +++ b/common/src/main/java/io/druid/common/config/NullHandling.java @@ -0,0 +1,106 @@ +/* + * 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.common.config; + +import com.google.common.base.Strings; +import com.google.inject.Inject; + +import javax.annotation.Nullable; + +/** + * Helper class for NullHandling. This class is used to switch between SQL compatible Null Handling behavior + * introduced as part of {@link https://github.com/druid-io/druid/issues/4349} and the old druid behavior + * where null values are replaced with default values e.g Null Strings are replaced with empty values. + */ +public class NullHandling +{ + private static String NULL_HANDLING_CONFIG_STRING = "druid.generic.useDefaultValueForNull"; + + /** + * use these values to ensure that {@link NullHandling#defaultDoubleValue()}, + * {@link NullHandling#defaultFloatValue()} , {@link NullHandling#defaultFloatValue()} + * return the same boxed object when returning a constant zero + */ + public static final Double ZERO_DOUBLE = 0.0d; + public static final Float ZERO_FLOAT = 0.0f; + public static final Long ZERO_LONG = 0L; + + /** + * INSTANCE is injected using static injection to avoid adding JacksonInject annotations all over the code. + * See {@link io.druid.guice.NullHandlingModule} for details. + * It does not take effect in all unit tests since we don't use Guice Injection. + * For tests default system property is supposed to be used only in tests + */ + @Inject + private static NullValueHandlingConfig INSTANCE = new NullValueHandlingConfig( + Boolean.valueOf(System.getProperty(NULL_HANDLING_CONFIG_STRING, "true")) + ); + + public static boolean useDefaultValuesForNull() + { + return INSTANCE.isUseDefaultValuesForNull(); + } + + @Nullable + public static String nullToEmptyIfNeeded(@Nullable String value) + { + //CHECKSTYLE.OFF: Regexp + return useDefaultValuesForNull() ? Strings.nullToEmpty(value) : value; + //CHECKSTYLE.ON: Regexp + } + + @Nullable + public static String emptyToNullIfNeeded(@Nullable String value) + { + //CHECKSTYLE.OFF: Regexp + return useDefaultValuesForNull() ? Strings.emptyToNull(value) : value; + //CHECKSTYLE.ON: Regexp + } + + @Nullable + public static String defaultStringValue() + { + return useDefaultValuesForNull() ? "" : null; + } + + @Nullable + public static Long defaultLongValue() + { + return useDefaultValuesForNull() ? ZERO_LONG : null; + } + + @Nullable + public static Float defaultFloatValue() + { + return useDefaultValuesForNull() ? ZERO_FLOAT : null; + } + + @Nullable + public static Double defaultDoubleValue() + { + return useDefaultValuesForNull() ? ZERO_DOUBLE : null; + } + + public static boolean isNullOrEquivalent(@Nullable String value) + { + return INSTANCE.isUseDefaultValuesForNull() ? Strings.isNullOrEmpty(value) : value == null; + } + +} diff --git a/common/src/main/java/io/druid/common/config/NullValueHandlingConfig.java b/common/src/main/java/io/druid/common/config/NullValueHandlingConfig.java new file mode 100644 index 000000000000..fdd1638b510e --- /dev/null +++ b/common/src/main/java/io/druid/common/config/NullValueHandlingConfig.java @@ -0,0 +1,43 @@ +/* + * 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.common.config; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class NullValueHandlingConfig +{ + + @JsonProperty("useDefaultValueForNull") + private final boolean useDefaultValuesForNull; + + @JsonCreator + public NullValueHandlingConfig(@JsonProperty("useDefaultValueForNull") Boolean useDefaultValuesForNull) + { + this.useDefaultValuesForNull = useDefaultValuesForNull == null + ? true + : useDefaultValuesForNull; + } + + public boolean isUseDefaultValuesForNull() + { + return useDefaultValuesForNull; + } +} diff --git a/common/src/main/java/io/druid/math/expr/Expr.java b/common/src/main/java/io/druid/math/expr/Expr.java index 913f40244bc8..0ab1f9f82bf7 100644 --- a/common/src/main/java/io/druid/math/expr/Expr.java +++ b/common/src/main/java/io/druid/math/expr/Expr.java @@ -20,9 +20,9 @@ package io.druid.math.expr; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import com.google.common.math.LongMath; import com.google.common.primitives.Ints; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Comparators; @@ -124,7 +124,7 @@ class StringExpr extends ConstantExpr public StringExpr(String value) { - this.value = Strings.emptyToNull(value); + this.value = NullHandling.emptyToNullIfNeeded(value); } @Nullable @@ -362,6 +362,13 @@ public ExprEval eval(ObjectBinding bindings) { ExprEval leftVal = left.eval(bindings); ExprEval rightVal = right.eval(bindings); + + // Result of any Binary expressions is null if any of the argument is null. + // e.g "select null * 2 as c;" or "select null + 1 as c;" will return null as per Standard SQL spec. + if (!NullHandling.useDefaultValuesForNull() && (leftVal.isNull() || rightVal.isNull())) { + return ExprEval.of(null); + } + if (leftVal.type() == ExprType.STRING && rightVal.type() == ExprType.STRING) { return evalString(leftVal.asString(), rightVal.asString()); } else if (leftVal.type() == ExprType.LONG && rightVal.type() == ExprType.LONG) { @@ -491,7 +498,8 @@ class BinPlusExpr extends BinaryEvalOpExprBase @Override protected ExprEval evalString(@Nullable String left, @Nullable String right) { - return ExprEval.of(Strings.nullToEmpty(left) + Strings.nullToEmpty(right)); + return ExprEval.of(NullHandling.nullToEmptyIfNeeded(left) + + NullHandling.nullToEmptyIfNeeded(right)); } @Override diff --git a/common/src/main/java/io/druid/math/expr/ExprEval.java b/common/src/main/java/io/druid/math/expr/ExprEval.java index 972ce342fe54..2aa724c6081b 100644 --- a/common/src/main/java/io/druid/math/expr/ExprEval.java +++ b/common/src/main/java/io/druid/math/expr/ExprEval.java @@ -20,12 +20,14 @@ package io.druid.math.expr; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import com.google.common.primitives.Doubles; import com.google.common.primitives.Ints; +import io.druid.common.config.NullHandling; import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.IAE; +import javax.annotation.Nullable; + /** */ public abstract class ExprEval @@ -50,7 +52,7 @@ public static ExprEval of(double doubleValue) return new DoubleExprEval(doubleValue); } - public static ExprEval of(String stringValue) + public static ExprEval of(@Nullable String stringValue) { return new StringExprEval(stringValue); } @@ -108,6 +110,7 @@ public boolean isNull() public abstract double asDouble(); + @Nullable public String asString() { return value == null ? null : String.valueOf(value); @@ -228,9 +231,9 @@ public Expr toExpr() private static class StringExprEval extends ExprEval { - private StringExprEval(String value) + private StringExprEval(@Nullable String value) { - super(Strings.emptyToNull(value)); + super(NullHandling.emptyToNullIfNeeded(value)); } @Override diff --git a/common/src/main/java/io/druid/math/expr/ExprListenerImpl.java b/common/src/main/java/io/druid/math/expr/ExprListenerImpl.java index 270cd870c552..cb4eab179a3b 100644 --- a/common/src/main/java/io/druid/math/expr/ExprListenerImpl.java +++ b/common/src/main/java/io/druid/math/expr/ExprListenerImpl.java @@ -335,4 +335,10 @@ public void exitFunctionArgs(ExprParser.FunctionArgsContext ctx) nodes.put(ctx, args); } + + @Override + public void exitNull(ExprParser.NullContext ctx) + { + nodes.put(ctx, new StringExpr(null)); + } } diff --git a/common/src/main/java/io/druid/math/expr/Function.java b/common/src/main/java/io/druid/math/expr/Function.java index 0e067583cfcf..9a589ebc9fd3 100644 --- a/common/src/main/java/io/druid/math/expr/Function.java +++ b/common/src/main/java/io/druid/math/expr/Function.java @@ -19,7 +19,7 @@ package io.druid.math.expr; -import com.google.common.base.Strings; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.IAE; import io.druid.java.util.common.StringUtils; @@ -74,6 +74,9 @@ abstract class SingleParamMath extends SingleParam @Override protected final ExprEval eval(ExprEval param) { + if (!NullHandling.useDefaultValuesForNull() && param.isNull()) { + return ExprEval.of(null); + } if (param.type() == ExprType.LONG) { return eval(param.asLong()); } else if (param.type() == ExprType.DOUBLE) { @@ -896,10 +899,20 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) return ExprEval.of(null); } else { // Pass first argument in to the constructor to provide StringBuilder a little extra sizing hint. - final StringBuilder builder = new StringBuilder(Strings.nullToEmpty(args.get(0).eval(bindings).asString())); + String first = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString()); + if (first == null) { + // Result of concatenation is null if any of the Values is null. + // e.g. 'select CONCAT(null, "abc") as c;' will return null as per Standard SQL spec. + return ExprEval.of(null); + } + final StringBuilder builder = new StringBuilder(first); for (int i = 1; i < args.size(); i++) { - final String s = args.get(i).eval(bindings).asString(); - if (s != null) { + final String s = NullHandling.nullToEmptyIfNeeded(args.get(i).eval(bindings).asString()); + if (s == null) { + // Result of concatenation is null if any of the Values is null. + // e.g. 'select CONCAT(null, "abc") as c;' will return null as per Standard SQL spec. + return ExprEval.of(null); + } else { builder.append(s); } } @@ -943,9 +956,12 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) throw new IAE("Function[%s] needs 2 arguments", name()); } - final String haystack = Strings.nullToEmpty(args.get(0).eval(bindings).asString()); - final String needle = Strings.nullToEmpty(args.get(1).eval(bindings).asString()); + final String haystack = NullHandling.nullToEmptyIfNeeded(args.get(0).eval(bindings).asString()); + final String needle = NullHandling.nullToEmptyIfNeeded(args.get(1).eval(bindings).asString()); + if (haystack == null || needle == null) { + return ExprEval.of(null); + } return ExprEval.of(haystack.indexOf(needle)); } } @@ -982,7 +998,9 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) return ExprEval.of(arg.substring(index)); } } else { - return ExprEval.of(null); + // If starting index of substring is greater then the length of string, the result will be a zero length string. + // e.g. 'select substring("abc", 4,5) as c;' will return an empty string + return ExprEval.of(NullHandling.defaultStringValue()); } } } @@ -1005,8 +1023,11 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) final String arg = args.get(0).eval(bindings).asString(); final String pattern = args.get(1).eval(bindings).asString(); final String replacement = args.get(2).eval(bindings).asString(); + if (arg == null) { + return ExprEval.of(NullHandling.defaultStringValue()); + } return ExprEval.of( - Strings.nullToEmpty(arg).replace(Strings.nullToEmpty(pattern), Strings.nullToEmpty(replacement)) + arg.replace(NullHandling.nullToEmptyIfNeeded(pattern), NullHandling.nullToEmptyIfNeeded(replacement)) ); } } @@ -1027,7 +1048,10 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) } final String arg = args.get(0).eval(bindings).asString(); - return ExprEval.of(StringUtils.toLowerCase(Strings.nullToEmpty(arg))); + if (arg == null) { + return ExprEval.of(NullHandling.defaultStringValue()); + } + return ExprEval.of(StringUtils.toLowerCase(arg)); } } @@ -1047,7 +1071,50 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) } final String arg = args.get(0).eval(bindings).asString(); - return ExprEval.of(StringUtils.toUpperCase(Strings.nullToEmpty(arg))); + if (arg == null) { + return ExprEval.of(NullHandling.defaultStringValue()); + } + return ExprEval.of(StringUtils.toUpperCase(arg)); + } + } + + class IsNullFunc implements Function + { + @Override + public String name() + { + return "isnull"; + } + + @Override + public ExprEval apply(List args, Expr.ObjectBinding bindings) + { + if (args.size() != 1) { + throw new IAE("Function[%s] needs 1 argument", name()); + } + + final ExprEval expr = args.get(0).eval(bindings); + return ExprEval.of(expr.isNull(), ExprType.LONG); + } + } + + class IsNotNullFunc implements Function + { + @Override + public String name() + { + return "notnull"; + } + + @Override + public ExprEval apply(List args, Expr.ObjectBinding bindings) + { + if (args.size() != 1) { + throw new IAE("Function[%s] needs 1 argument", name()); + } + + final ExprEval expr = args.get(0).eval(bindings); + return ExprEval.of(!expr.isNull(), ExprType.LONG); } } } diff --git a/common/src/test/java/io/druid/math/expr/EvalTest.java b/common/src/test/java/io/druid/math/expr/EvalTest.java index 75a06a03d6eb..68ef37f309cf 100644 --- a/common/src/test/java/io/druid/math/expr/EvalTest.java +++ b/common/src/test/java/io/druid/math/expr/EvalTest.java @@ -20,6 +20,7 @@ package io.druid.math.expr; import com.google.common.collect.ImmutableMap; +import io.druid.common.config.NullHandling; import org.junit.Assert; import org.junit.Test; @@ -139,8 +140,11 @@ public void testLongEval() Assert.assertEquals(1271055781L, evalLong("unix_timestamp('2010-04-12T07:03:01')", bindings)); Assert.assertEquals(1271023381L, evalLong("unix_timestamp('2010-04-12T07:03:01+09:00')", bindings)); Assert.assertEquals(1271023381L, evalLong("unix_timestamp('2010-04-12T07:03:01.419+09:00')", bindings)); - - Assert.assertEquals("NULL", eval("nvl(if(x == 9223372036854775807, '', 'x'), 'NULL')", bindings).asString()); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals("NULL", eval("nvl(if(x == 9223372036854775807, '', 'x'), 'NULL')", bindings).asString()); + } else { + Assert.assertEquals("", eval("nvl(if(x == 9223372036854775807, '', 'x'), 'NULL')", bindings).asString()); + } Assert.assertEquals("x", eval("nvl(if(x == 9223372036854775806, '', 'x'), 'NULL')", bindings).asString()); } diff --git a/common/src/test/java/io/druid/math/expr/FunctionTest.java b/common/src/test/java/io/druid/math/expr/FunctionTest.java index 2351dcf16b4a..99c153da550c 100644 --- a/common/src/test/java/io/druid/math/expr/FunctionTest.java +++ b/common/src/test/java/io/druid/math/expr/FunctionTest.java @@ -20,6 +20,7 @@ package io.druid.math.expr; import com.google.common.collect.ImmutableMap; +import io.druid.common.config.NullHandling; import org.junit.Assert; import org.junit.Test; @@ -55,7 +56,12 @@ public void testCaseSearched() public void testConcat() { assertExpr("concat(x,' ',y)", "foo 2"); - assertExpr("concat(x,' ',nonexistent,' ',y)", "foo 2"); + if (NullHandling.useDefaultValuesForNull()) { + assertExpr("concat(x,' ',nonexistent,' ',y)", "foo 2"); + } else { + assertExpr("concat(x,' ',nonexistent,' ',y)", null); + } + assertExpr("concat(z)", "3.1"); assertExpr("concat()", null); } @@ -110,4 +116,18 @@ private void assertExpr(final String expression, final Object expectedResult) final Expr expr = Parser.parse(expression, ExprMacroTable.nil()); Assert.assertEquals(expression, expectedResult, expr.eval(bindings).value()); } + + @Test + public void testIsNull() + { + assertExpr("isnull(null)", 1L); + assertExpr("isnull('abc')", 0L); + } + + @Test + public void testIsNotNull() + { + assertExpr("notnull(null)", 0L); + assertExpr("notnull('abc')", 1L); + } } diff --git a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java index 76cd4d12cca4..64889315792c 100644 --- a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java +++ b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java @@ -192,6 +192,12 @@ public long getLong() return 0L; } + @Override + public boolean isNull() + { + return false; + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { 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 ec20b5e37fe3..3448dd4334ea 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 @@ -151,6 +151,13 @@ public float getFloat() { return v; } + + @Override + public boolean isNull() + { + return false; + } + } private static class ObjectHandOver extends TestObjectColumnSelector diff --git a/java-util/src/main/java/io/druid/java/util/common/StringUtils.java b/java-util/src/main/java/io/druid/java/util/common/StringUtils.java index dbf84aad7315..0d7d84b1c544 100644 --- a/java-util/src/main/java/io/druid/java/util/common/StringUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/StringUtils.java @@ -22,6 +22,7 @@ import com.google.common.base.Charsets; import com.google.common.base.Throwables; +import javax.annotation.Nullable; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.nio.charset.Charset; @@ -77,11 +78,37 @@ public static String fromUtf8(final byte[] bytes) } } + @Nullable + public static String fromUtf8Nullable(@Nullable final byte[] bytes) + { + if (bytes == null) { + return null; + } + return fromUtf8(bytes); + } + public static String fromUtf8(final ByteBuffer buffer, final int numBytes) { final byte[] bytes = new byte[numBytes]; buffer.get(bytes); - return StringUtils.fromUtf8(bytes); + return fromUtf8(bytes); + } + + /** + * Reads numBytes bytes from buffer and converts that to a utf-8 string + * @param buffer buffer to read bytes from + * @param numBytes number of bytes to read + * @return returns null if numBytes is -1 otherwise utf-8 string represetation of bytes read + */ + @Nullable + public static String fromUtf8Nullable(final ByteBuffer buffer, final int numBytes) + { + if (numBytes < 0) { + return null; + } + final byte[] bytes = new byte[numBytes]; + buffer.get(bytes); + return fromUtf8Nullable(bytes); } public static String fromUtf8(final ByteBuffer buffer) @@ -100,6 +127,15 @@ public static byte[] toUtf8(final String string) } } + @Nullable + public static byte[] toUtf8Nullable(@Nullable final String string) + { + if (string == null) { + return null; + } + return toUtf8(string); + } + /** * Equivalent of String.format(Locale.ENGLISH, message, formatArgs). */ diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java index 2287307eb749..797f44843837 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java @@ -70,6 +70,12 @@ public interface AggregateCombiner extends ColumnValueSelector @SuppressWarnings("unused") // Going to be used when https://github.com/druid-io/druid/projects/2 is complete void fold(ColumnValueSelector selector); + @Override + default boolean isNull() + { + return false; + } + @Override default void inspectRuntimeShape(RuntimeShapeInspector inspector) { 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 a13eebf1b371..6f3f0afd4787 100644 --- a/processing/src/main/java/io/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/Aggregator.java @@ -53,6 +53,19 @@ default double getDouble() return (double) getFloat(); } + /** + * returns true if aggregator's output type is primitive long/double/float and aggregated value is null, + * but when aggregated output type is Object, this method always returns false, + * and users are advised to check nullability for the object returned by {@link #get()} + * method. + * The default implementation always return false to enable smooth backward compatibility, + * re-implement if your aggregator is nullable. + */ + default boolean isNull() + { + return false; + } + @Override void close(); } diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java index 7f3591182605..8a7477a73588 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java @@ -164,6 +164,13 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("baseSelector", baseSelector); } + + @Override + public boolean isNull() + { + final ExprEval exprEval = baseSelector.getObject(); + return exprEval.isNull(); + } } return new ExpressionFloatColumnSelector(); } @@ -198,6 +205,13 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("baseSelector", baseSelector); } + + @Override + public boolean isNull() + { + final ExprEval exprEval = baseSelector.getObject(); + return exprEval.isNull(); + } } return new ExpressionLongColumnSelector(); } @@ -232,6 +246,13 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("baseSelector", baseSelector); } + + @Override + public boolean isNull() + { + final ExprEval exprEval = baseSelector.getObject(); + return exprEval.isNull(); + } } return new ExpressionDoubleColumnSelector(); } 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 cdddd7629f2d..b32ea821fa26 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -178,4 +178,23 @@ default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, By { } + /** + * returns true if aggregator's output type is primitive long/double/float and aggregated value is null, + * but when aggregated output type is Object, this method always returns false, + * and users are advised to check nullability for the object returned by {@link #get()} + * method. + * The default implementation always return false to enable smooth backward compatibility, + * re-implement if your aggregator is nullable. + * + * @param buf byte buffer storing the byte array representation of the aggregate + * @param position offset within the byte buffer at which the aggregate value is stored + * + * @return true if the aggrgeated value is null otherwise false. + * For backwards compatibility, isNull() may return false even if {@link BufferAggregator#get(ByteBuffer, int)} returns null. Users of this method should account for this case. + */ + default boolean isNull(ByteBuffer buf, int position) + { + return false; + } + } diff --git a/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java index bc19ba476e7a..d486ecad82bb 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java @@ -26,4 +26,9 @@ */ public abstract class ObjectAggregateCombiner implements AggregateCombiner, ObjectColumnSelector { + @Override + public boolean isNull() + { + return false; + } } diff --git a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java index 70e53fa37d32..856491080071 100644 --- a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java @@ -20,9 +20,9 @@ package io.druid.query.groupby; import com.google.common.base.Predicate; -import com.google.common.base.Strings; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; +import io.druid.common.config.NullHandling; import io.druid.data.input.Row; import io.druid.query.dimension.DimensionSpec; import io.druid.query.extraction.ExtractionFn; @@ -30,6 +30,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.ColumnValueSelector; +import io.druid.segment.DimensionHandlerUtils; import io.druid.segment.DimensionSelector; import io.druid.segment.IdLookup; import io.druid.segment.LongColumnSelector; @@ -228,7 +229,7 @@ public boolean matches() } for (String dimensionValue : dimensionValues) { - if (Objects.equals(Strings.emptyToNull(dimensionValue), value)) { + if (Objects.equals(NullHandling.emptyToNullIfNeeded(dimensionValue), value)) { return true; } } @@ -253,7 +254,7 @@ public boolean matches() } for (String dimensionValue : dimensionValues) { - if (Objects.equals(extractionFn.apply(Strings.emptyToNull(dimensionValue)), value)) { + if (Objects.equals(extractionFn.apply(NullHandling.emptyToNullIfNeeded(dimensionValue)), value)) { return true; } } @@ -286,7 +287,7 @@ public boolean matches() } for (String dimensionValue : dimensionValues) { - if (predicate.apply(Strings.emptyToNull(dimensionValue))) { + if (predicate.apply(NullHandling.emptyToNullIfNeeded(dimensionValue))) { return true; } } @@ -312,7 +313,7 @@ public boolean matches() } for (String dimensionValue : dimensionValues) { - if (predicate.apply(extractionFn.apply(Strings.emptyToNull(dimensionValue)))) { + if (predicate.apply(extractionFn.apply(NullHandling.emptyToNullIfNeeded(dimensionValue)))) { return true; } } @@ -338,7 +339,7 @@ public int getValueCardinality() @Override public String lookupName(int id) { - final String value = Strings.emptyToNull(row.get().getDimension(dimension).get(id)); + final String value = NullHandling.emptyToNullIfNeeded(row.get().getDimension(dimension).get(id)); return extractionFn == null ? value : extractionFn.apply(value); } @@ -397,6 +398,13 @@ public long getLong() return row.get().getTimestampFromEpoch(); } + @Override + public boolean isNull() + { + // Time column never has null values + return false; + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { @@ -410,19 +418,25 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public double getDouble() { - return row.get().getMetric(columnName).doubleValue(); + return DimensionHandlerUtils.nullToZeroDouble(row.get().getMetric(columnName)).doubleValue(); + } + + @Override + public boolean isNull() + { + return row.get().getRaw(columnName) == null; } @Override public float getFloat() { - return row.get().getMetric(columnName).floatValue(); + return DimensionHandlerUtils.nullToZeroDouble(row.get().getMetric(columnName)).floatValue(); } @Override public long getLong() { - return row.get().getMetric(columnName).longValue(); + return DimensionHandlerUtils.nullToZeroDouble(row.get().getMetric(columnName)).longValue(); } @Nullable diff --git a/processing/src/main/java/io/druid/segment/BaseDoubleColumnValueSelector.java b/processing/src/main/java/io/druid/segment/BaseDoubleColumnValueSelector.java index e1d0d40d23f7..829f37711cab 100644 --- a/processing/src/main/java/io/druid/segment/BaseDoubleColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/BaseDoubleColumnValueSelector.java @@ -31,7 +31,7 @@ * All implementations of this interface MUST also implement {@link ColumnValueSelector}. */ @PublicApi -public interface BaseDoubleColumnValueSelector extends HotLoopCallee +public interface BaseDoubleColumnValueSelector extends HotLoopCallee, BaseNullableColumnValueSelector { @CalledFromHotLoop double getDouble(); diff --git a/processing/src/main/java/io/druid/segment/BaseFloatColumnValueSelector.java b/processing/src/main/java/io/druid/segment/BaseFloatColumnValueSelector.java index 9223ff339422..42f8d35b5bde 100644 --- a/processing/src/main/java/io/druid/segment/BaseFloatColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/BaseFloatColumnValueSelector.java @@ -31,7 +31,7 @@ * All implementations of this interface MUST also implement {@link ColumnValueSelector}. */ @PublicApi -public interface BaseFloatColumnValueSelector extends HotLoopCallee +public interface BaseFloatColumnValueSelector extends HotLoopCallee, BaseNullableColumnValueSelector { @CalledFromHotLoop float getFloat(); diff --git a/processing/src/main/java/io/druid/segment/BaseLongColumnValueSelector.java b/processing/src/main/java/io/druid/segment/BaseLongColumnValueSelector.java index ac06ab7d3273..70d9fb33ff2b 100644 --- a/processing/src/main/java/io/druid/segment/BaseLongColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/BaseLongColumnValueSelector.java @@ -31,7 +31,7 @@ * All implementations of this interface MUST also implement {@link ColumnValueSelector}. */ @PublicApi -public interface BaseLongColumnValueSelector extends HotLoopCallee +public interface BaseLongColumnValueSelector extends HotLoopCallee, BaseNullableColumnValueSelector { @CalledFromHotLoop long getLong(); diff --git a/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java b/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java new file mode 100644 index 000000000000..3d2c6a5b990f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java @@ -0,0 +1,28 @@ +/* + * 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.guice.annotations.PublicApi; + +@PublicApi +public interface BaseNullableColumnValueSelector +{ + boolean isNull(); +} diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index 3ad6574a08dc..7026b0cd7042 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -19,10 +19,10 @@ package io.druid.segment; -import com.google.common.base.Strings; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.spatial.ImmutableRTree; +import io.druid.common.config.NullHandling; import io.druid.query.filter.BitmapIndexSelector; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.column.BitmapIndex; @@ -180,7 +180,7 @@ public int getIndex(String value) // Return -2 for non-null values to match what the BitmapIndex implementation in BitmapIndexColumnPartSupplier // would return for getIndex() when there is only a single index, for the null value. // i.e., return an 'insertion point' of 1 for non-null values (see BitmapIndex interface) - return Strings.isNullOrEmpty(value) ? 0 : -2; + return NullHandling.isNullOrEquivalent(value) ? 0 : -2; } @Override @@ -210,7 +210,7 @@ public ImmutableBitmap getBitmapIndex(String dimension, String value) final Column column = index.getColumn(dimension); if (column == null || !columnSupportsFiltering(column)) { - if (Strings.isNullOrEmpty(value)) { + if (NullHandling.isNullOrEquivalent(value)) { return bitmapFactory.complement(bitmapFactory.makeEmptyImmutableBitmap(), getNumRows()); } else { return bitmapFactory.makeEmptyImmutableBitmap(); @@ -222,7 +222,7 @@ public ImmutableBitmap getBitmapIndex(String dimension, String value) } final BitmapIndex bitmapIndex = column.getBitmapIndex(); - return bitmapIndex.getBitmap(bitmapIndex.getIndex(value)); + return bitmapIndex.getBitmap(bitmapIndex.getIndex(NullHandling.emptyToNullIfNeeded(value))); } @Override diff --git a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java index 010dd841dd21..087ce74de2ae 100644 --- a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java @@ -86,4 +86,10 @@ public void inspectRuntimeShape(final RuntimeShapeInspector inspector) { // Nothing here: objectValue is nullable but getObject is not @CalledFromHotLoop } + + @Override + public boolean isNull() + { + return objectValue == null; + } } diff --git a/processing/src/main/java/io/druid/segment/ConstantDimensionSelector.java b/processing/src/main/java/io/druid/segment/ConstantDimensionSelector.java index 8210d7d5c948..5f05f8f9d6d5 100644 --- a/processing/src/main/java/io/druid/segment/ConstantDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/ConstantDimensionSelector.java @@ -20,7 +20,7 @@ package io.druid.segment; import com.google.common.base.Predicate; -import com.google.common.base.Strings; +import io.druid.common.config.NullHandling; import io.druid.query.filter.ValueMatcher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.IndexedInts; @@ -37,7 +37,7 @@ public class ConstantDimensionSelector implements SingleValueHistoricalDimension public ConstantDimensionSelector(final String value) { - if (Strings.isNullOrEmpty(value)) { + if (NullHandling.isNullOrEquivalent(value)) { // There's an optimized implementation for nulls that callers should use instead. throw new IllegalArgumentException("Use NullDimensionSelector or DimensionSelectorUtils.constantSelector"); } diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java index 5d7a56a5f046..f52e5915d439 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Doubles; import com.google.common.primitives.Floats; +import io.druid.common.config.NullHandling; import io.druid.common.guava.GuavaUtils; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.java.util.common.IAE; @@ -237,6 +238,15 @@ private static Colu return strategyFactory.makeColumnSelectorStrategy(capabilities, selector); } + @Nullable + public static String convertObjectToString(@Nullable Object valObj) + { + if (valObj == null) { + return null; + } + return valObj.toString(); + } + @Nullable public static Long convertObjectToLong(@Nullable Object valObj) { @@ -247,7 +257,7 @@ public static Long convertObjectToLong(@Nullable Object valObj) public static Long convertObjectToLong(@Nullable Object valObj, boolean reportParseExceptions) { if (valObj == null) { - return ZERO_LONG; + return null; } if (valObj instanceof Long) { @@ -275,7 +285,7 @@ public static Float convertObjectToFloat(@Nullable Object valObj) public static Float convertObjectToFloat(@Nullable Object valObj, boolean reportParseExceptions) { if (valObj == null) { - return ZERO_FLOAT; + return null; } if (valObj instanceof Float) { @@ -303,7 +313,7 @@ public static Double convertObjectToDouble(@Nullable Object valObj) public static Double convertObjectToDouble(@Nullable Object valObj, boolean reportParseExceptions) { if (valObj == null) { - return ZERO_DOUBLE; + return null; } if (valObj instanceof Double) { @@ -357,18 +367,18 @@ public static Long getExactLongFromDecimalString(String decimalStr) } } - public static Double nullToZero(@Nullable Double number) + public static Number nullToZeroDouble(@Nullable Number number) { - return number == null ? ZERO_DOUBLE : number; + return number == null ? NullHandling.ZERO_DOUBLE : number; } - public static Long nullToZero(@Nullable Long number) + public static Number nullToZeroLong(@Nullable Number number) { - return number == null ? ZERO_LONG : number; + return number == null ? NullHandling.ZERO_LONG : number; } - public static Float nullToZero(@Nullable Float number) + public static Number nullToZeroFloat(@Nullable Number number) { - return number == null ? ZERO_FLOAT : number; + return number == null ? NullHandling.ZERO_FLOAT : number; } } diff --git a/processing/src/main/java/io/druid/segment/DimensionSelector.java b/processing/src/main/java/io/druid/segment/DimensionSelector.java index 91e099070481..9aee99edf220 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelector.java @@ -150,12 +150,24 @@ default long getLong() } /** + * @deprecated always throws {@link UnsupportedOperationException} + */ + @Deprecated + @Override + default boolean isNull() + { + throw new UnsupportedOperationException("DimensionSelector cannot be operated as numeric ColumnValueSelector" + this.getClass()); + } + + /** + * @deprecated always throws {@link UnsupportedOperationException} * Converts the current result of {@link #getRow()} into null, if the row is empty, a String, if the row has size 1, * or a String[] array, if the row has size > 1, using {@link #lookupName(int)}. * * This method is not the default implementation of {@link #getObject()} to minimize the chance that implementations * "forget" to override it with more optimized version. */ + @Deprecated @Nullable default Object defaultGetObject() { diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java index 1fb6c53cefbc..0765274c70b0 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java @@ -61,6 +61,9 @@ default long getLong() @Override default Double getObject() { + if (isNull()) { + return null; + } return getDouble(); } diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java new file mode 100644 index 000000000000..d5392331335a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java @@ -0,0 +1,134 @@ +/* + * 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.primitives.Ints; +import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.collections.bitmap.MutableBitmap; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.ColumnarDoublesSerializer; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class DoubleColumnSerializerV2 implements GenericColumnSerializer +{ + public static DoubleColumnSerializerV2 create( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + CompressionStrategy compression, + BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new DoubleColumnSerializerV2( + segmentWriteOutMedium, + filenameBase, + IndexIO.BYTE_ORDER, + compression, + bitmapSerdeFactory + ); + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final String filenameBase; + private final ByteOrder byteOrder; + private final CompressionStrategy compression; + private final BitmapSerdeFactory bitmapSerdeFactory; + + private ColumnarDoublesSerializer writer; + private ByteBufferWriter nullValueBitmapWriter; + private MutableBitmap nullRowsBitmap; + private int rowCount = 0; + + private DoubleColumnSerializerV2( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ByteOrder byteOrder, + CompressionStrategy compression, + BitmapSerdeFactory bitmapSerdeFactory + ) + { + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.filenameBase = filenameBase; + this.byteOrder = byteOrder; + this.compression = compression; + this.bitmapSerdeFactory = bitmapSerdeFactory; + } + + @Override + public void open() throws IOException + { + writer = CompressionFactory.getDoubleSerializer( + segmentWriteOutMedium, + StringUtils.format("%s.double_column", filenameBase), + byteOrder, + compression + ); + writer.open(); + nullValueBitmapWriter = new ByteBufferWriter<>( + segmentWriteOutMedium, + bitmapSerdeFactory.getObjectStrategy() + ); + nullValueBitmapWriter.open(); + nullRowsBitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); + } + + @Override + public void serialize(@Nullable Object obj) throws IOException + { + if (obj == null) { + nullRowsBitmap.add(rowCount); + writer.add(0D); + } else { + writer.add(((Number) obj).doubleValue()); + } + rowCount++; + } + + @Override + public long getSerializedSize() throws IOException + { + nullValueBitmapWriter.write(bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(nullRowsBitmap)); + long bitmapSize = nullRowsBitmap.isEmpty() + ? 0L + : nullValueBitmapWriter.getSerializedSize(); + return Integer.BYTES + writer.getSerializedSize() + bitmapSize; + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + channel.write(ByteBuffer.wrap(Ints.toByteArray((int) writer.getSerializedSize()))); + writer.writeTo(channel, smoosher); + if (!nullRowsBitmap.isEmpty()) { + nullValueBitmapWriter.writeTo(channel, smoosher); + } + } + +} diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index 2643ed6ff386..609a40d23eb3 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -21,6 +21,8 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.config.NullHandling; +import io.druid.java.util.common.guava.Comparators; import io.druid.query.dimension.DimensionSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; @@ -28,10 +30,13 @@ import io.druid.segment.incremental.TimeAndDimsHolder; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.List; +import java.util.Objects; public class DoubleDimensionIndexer implements DimensionIndexer { + public static final Comparator DOUBLE_COMPARATOR = Comparators.naturalNullsFirst(); @Override public Double processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions) @@ -93,12 +98,23 @@ public ColumnValueSelector makeColumnValueSelector( final int dimIndex = desc.getIndex(); class IndexerDoubleColumnSelector implements DoubleColumnSelector { + + @Override + public boolean isNull() + { + if (NullHandling.useDefaultValuesForNull()) { + return false; + } + final Object[] dims = currEntry.get().getDims(); + return dimIndex >= dims.length || dims[dimIndex] == null; + } + @Override public double getDouble() { final Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { + if (dimIndex >= dims.length || dims[dimIndex] == null) { return 0.0; } return (Double) dims[dimIndex]; @@ -130,19 +146,19 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public int compareUnsortedEncodedKeyComponents(@Nullable Double lhs, @Nullable Double rhs) { - return Double.compare(DimensionHandlerUtils.nullToZero(lhs), DimensionHandlerUtils.nullToZero(rhs)); + return DOUBLE_COMPARATOR.compare(lhs, rhs); } @Override public boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable Double lhs, @Nullable Double rhs) { - return DimensionHandlerUtils.nullToZero(lhs).equals(DimensionHandlerUtils.nullToZero(rhs)); + return Objects.equals(lhs, rhs); } @Override public int getUnsortedEncodedKeyComponentHashCode(@Nullable Double key) { - return DimensionHandlerUtils.nullToZero(key).hashCode(); + return DimensionHandlerUtils.nullToZeroDouble(key).hashCode(); } @Override diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index 3702df043127..9e7c37ff3770 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -22,7 +22,7 @@ import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressionStrategy; -import io.druid.segment.serde.DoubleGenericColumnPartSerde; +import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; @@ -33,7 +33,7 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9 { protected String dimensionName; protected final IndexSpec indexSpec; - private DoubleColumnSerializer serializer; + private GenericColumnSerializer serializer; public DoubleDimensionMergerV9( String dimensionName, @@ -55,7 +55,14 @@ public DoubleDimensionMergerV9( private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = DoubleColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression); + // If using default values for null use DoubleColumnSerializer to allow rollback to previous versions. + this.serializer = IndexMergerV9.createDoubleColumnSerializer( + segmentWriteOutMedium, + dimensionName, + metCompression, + indexSpec + ); + serializer.open(); } @@ -86,7 +93,6 @@ public void writeIndexes(List segmentRowNumConversions) throws IOExce @Override public boolean canSkip() { - // a double column can never be all null return false; } @@ -95,12 +101,8 @@ public ColumnDescriptor makeColumnDescriptor() throws IOException { final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.DOUBLE); - builder.addSerde( - DoubleGenericColumnPartSerde.serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate(serializer) - .build() - ); + ColumnPartSerde serde = IndexMergerV9.createDoubleColumnPartSerde(serializer, indexSpec); + builder.addSerde(serde); return builder.build(); } } diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java new file mode 100644 index 000000000000..530f652fe065 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java @@ -0,0 +1,128 @@ +/* + * 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.primitives.Ints; +import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.collections.bitmap.MutableBitmap; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.ColumnarFloatsSerializer; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class FloatColumnSerializerV2 implements GenericColumnSerializer +{ + public static FloatColumnSerializerV2 create( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + CompressionStrategy compression, + BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new FloatColumnSerializerV2(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression, bitmapSerdeFactory); + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final String filenameBase; + private final ByteOrder byteOrder; + private final CompressionStrategy compression; + private final BitmapSerdeFactory bitmapSerdeFactory; + + private ColumnarFloatsSerializer writer; + private ByteBufferWriter nullValueBitmapWriter; + private MutableBitmap nullRowsBitmap; + private int rowCount = 0; + + private FloatColumnSerializerV2( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ByteOrder byteOrder, + CompressionStrategy compression, + BitmapSerdeFactory bitmapSerdeFactory + ) + { + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.filenameBase = filenameBase; + this.byteOrder = byteOrder; + this.compression = compression; + this.bitmapSerdeFactory = bitmapSerdeFactory; + } + + @Override + public void open() throws IOException + { + writer = CompressionFactory.getFloatSerializer( + segmentWriteOutMedium, + StringUtils.format("%s.float_column", filenameBase), + byteOrder, + compression + ); + writer.open(); + nullValueBitmapWriter = new ByteBufferWriter<>( + segmentWriteOutMedium, + bitmapSerdeFactory.getObjectStrategy() + ); + nullValueBitmapWriter.open(); + nullRowsBitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); + } + + @Override + public void serialize(@Nullable Object obj) throws IOException + { + if (obj == null) { + nullRowsBitmap.add(rowCount); + writer.add(0L); + } else { + writer.add(((Number) obj).floatValue()); + } + rowCount++; + } + + @Override + + public long getSerializedSize() throws IOException + { + nullValueBitmapWriter.write(bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(nullRowsBitmap)); + long bitmapSize = nullRowsBitmap.isEmpty() + ? 0L + : nullValueBitmapWriter.getSerializedSize(); + return Integer.BYTES + writer.getSerializedSize() + bitmapSize; + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + channel.write(ByteBuffer.wrap(Ints.toByteArray((int) writer.getSerializedSize()))); + writer.writeTo(channel, smoosher); + if (!nullRowsBitmap.isEmpty()) { + nullValueBitmapWriter.writeTo(channel, smoosher); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 6ac82d6abc36..1a05d8bdc06a 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -21,6 +21,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.java.util.common.guava.Comparators; import io.druid.query.dimension.DimensionSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; @@ -28,11 +29,15 @@ import io.druid.segment.incremental.TimeAndDimsHolder; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.List; +import java.util.Objects; public class FloatDimensionIndexer implements DimensionIndexer { + public static final Comparator FLOAT_COMPARATOR = Comparators.naturalNullsFirst(); + @Override public Float processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions) { @@ -99,13 +104,20 @@ public float getFloat() { final Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { + if (dimIndex >= dims.length || dims[dimIndex] == null) { return 0.0f; } return (Float) dims[dimIndex]; } + @Override + public boolean isNull() + { + final Object[] dims = currEntry.get().getDims(); + return dimIndex >= dims.length || dims[dimIndex] == null; + } + @SuppressWarnings("deprecation") @Nullable @Override @@ -133,19 +145,19 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public int compareUnsortedEncodedKeyComponents(@Nullable Float lhs, @Nullable Float rhs) { - return DimensionHandlerUtils.nullToZero(lhs).compareTo(DimensionHandlerUtils.nullToZero(rhs)); + return FLOAT_COMPARATOR.compare(lhs, rhs); } @Override public boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable Float lhs, @Nullable Float rhs) { - return DimensionHandlerUtils.nullToZero(lhs).equals(DimensionHandlerUtils.nullToZero(rhs)); + return Objects.equals(lhs, rhs); } @Override public int getUnsortedEncodedKeyComponentHashCode(@Nullable Float key) { - return DimensionHandlerUtils.nullToZero(key).hashCode(); + return DimensionHandlerUtils.nullToZeroFloat(key).hashCode(); } @Override diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index 0a2070f4cf1f..fda2fd4030cf 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -22,7 +22,7 @@ import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressionStrategy; -import io.druid.segment.serde.FloatGenericColumnPartSerde; +import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; @@ -33,7 +33,7 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9 { protected String dimensionName; protected final IndexSpec indexSpec; - private FloatColumnSerializer serializer; + private GenericColumnSerializer serializer; public FloatDimensionMergerV9( String dimensionName, @@ -55,7 +55,12 @@ public FloatDimensionMergerV9( private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = FloatColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression); + this.serializer = IndexMergerV9.createFloatColumnSerializer( + segmentWriteOutMedium, + dimensionName, + metCompression, + indexSpec + ); serializer.open(); } @@ -95,12 +100,8 @@ public ColumnDescriptor makeColumnDescriptor() throws IOException { final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.FLOAT); - builder.addSerde( - FloatGenericColumnPartSerde.serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate(serializer) - .build() - ); + ColumnPartSerde serde = IndexMergerV9.createFloatColumnPartSerde(serializer, indexSpec); + builder.addSerde(serde); return builder.build(); } } diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 7c94f7ff85a7..7c88dbaac627 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -84,6 +84,7 @@ public class IndexIO public static final byte V8_VERSION = 0x8; public static final byte V9_VERSION = 0x9; public static final int CURRENT_VERSION_ID = V9_VERSION; + public static BitmapSerdeFactory LEGACY_FACTORY = new BitmapSerde.LegacyBitmapSerdeFactory(); public static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder(); @@ -162,13 +163,11 @@ public void validateTwoSegments(final IndexableAdapter adapter1, final Indexable if (rb1.getRowNum() != rb2.getRowNum()) { throw new SegmentValidationException("Row number mismatch: [%d] vs [%d]", rb1.getRowNum(), rb2.getRowNum()); } - if (rb1.compareTo(rb2) != 0) { - try { - validateRowValues(dimHandlers, rb1, adapter1, rb2, adapter2); - } - catch (SegmentValidationException ex) { - throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2); - } + try { + validateRowValues(dimHandlers, rb1, adapter1, rb2, adapter2); + } + catch (SegmentValidationException ex) { + throw new SegmentValidationException(ex, "Validation failure on row %d: [%s] vs [%s]", row, rb1, rb2); } } if (it2.hasNext()) { @@ -477,7 +476,11 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException metric, new ColumnBuilder() .setType(ValueType.FLOAT) - .setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType)) + .setGenericColumn(new FloatGenericColumnSupplier( + metricHolder.floatType, + LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap() + )) .build() ); } else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) { @@ -507,7 +510,11 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException Column.TIME_COLUMN_NAME, new ColumnBuilder() .setType(ValueType.LONG) - .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) + .setGenericColumn(new LongGenericColumnSupplier( + index.timestamps, + LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap() + )) .build() ); return new SimpleQueryableIndex( diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 644bcfea6466..3611a15eb7c1 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -21,13 +21,14 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; import com.google.inject.ImplementedBy; +import io.druid.common.config.NullHandling; import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.ByteBufferUtils; import io.druid.java.util.common.ISE; @@ -52,10 +53,10 @@ import java.nio.IntBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import java.util.Objects; import java.util.PriorityQueue; import java.util.Set; import java.util.stream.Collectors; @@ -431,16 +432,10 @@ class DictionaryMergeIterator implements CloseableIterator DictionaryMergeIterator(Indexed[] dimValueLookups, boolean useDirect) { + final Ordering stringOrdering = Comparators.naturalNullsFirst(); pQueue = new PriorityQueue<>( dimValueLookups.length, - new Comparator>>() - { - @Override - public int compare(Pair> lhs, Pair> rhs) - { - return lhs.rhs.peek().compareTo(rhs.rhs.peek()); - } - } + (lhs, rhs) -> stringOrdering.compare(lhs.rhs.peek(), rhs.rhs.peek()) ); conversions = new IntBuffer[dimValueLookups.length]; for (int i = 0; i < conversions.length; i++) { @@ -461,14 +456,7 @@ public int compare(Pair> lhs, Pair iter = Iterators.peekingIterator( Iterators.transform( indexed.iterator(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return Strings.nullToEmpty(input); - } - } + input -> NullHandling.nullToEmptyIfNeeded(input) ) ); if (iter.hasNext()) { @@ -492,7 +480,7 @@ public String next() } final String value = writeTranslate(smallest, counter); - while (!pQueue.isEmpty() && value.equals(pQueue.peek().rhs.peek())) { + while (!pQueue.isEmpty() && Objects.equals(value, pQueue.peek().rhs.peek())) { writeTranslate(pQueue.remove(), counter); } counter++; diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index 6d9be2ced61c..f2918efed660 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -33,6 +33,7 @@ import com.google.common.primitives.Longs; import com.google.inject.Inject; import io.druid.collections.CombiningIterable; +import io.druid.common.config.NullHandling; import io.druid.io.ZeroCopyByteArrayOutputStream; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.IAE; @@ -58,12 +59,16 @@ import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.loading.MMappedQueryableSegmentizerFactory; +import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; import io.druid.segment.serde.DoubleGenericColumnPartSerde; +import io.druid.segment.serde.DoubleGenericColumnPartSerdeV2; import io.druid.segment.serde.FloatGenericColumnPartSerde; +import io.druid.segment.serde.FloatGenericColumnPartSerdeV2; import io.druid.segment.serde.LongGenericColumnPartSerde; +import io.druid.segment.serde.LongGenericColumnPartSerdeV2; import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; @@ -196,7 +201,7 @@ public Metadata apply(IndexableAdapter input) handlers, mergers ); - final LongColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); + final GenericColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); final ArrayList metWriters = setupMetricsWriters( segmentWriteOutMedium, mergedMetrics, @@ -211,8 +216,16 @@ public Metadata apply(IndexableAdapter input) /************ Create Inverted Indexes and Finalize Build Columns *************/ final String section = "build inverted index and columns"; progress.startSection(section); - makeTimeColumn(v9Smoosher, progress, timeWriter); - makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsValueTypes, metricTypeNames, metWriters); + makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec); + makeMetricsColumns( + v9Smoosher, + progress, + mergedMetrics, + metricsValueTypes, + metricTypeNames, + metWriters, + indexSpec + ); for (int i = 0; i < mergedDimensions.size(); i++) { DimensionMergerV9 merger = (DimensionMergerV9) mergers.get(i); @@ -323,7 +336,8 @@ private void makeMetricsColumns( final List mergedMetrics, final Map metricsValueTypes, final Map metricTypeNames, - final List metWriters + final List metWriters, + final IndexSpec indexSpec ) throws IOException { final String section = "make metric columns"; @@ -340,33 +354,15 @@ private void makeMetricsColumns( switch (type) { case LONG: builder.setValueType(ValueType.LONG); - builder.addSerde( - LongGenericColumnPartSerde - .serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate((LongColumnSerializer) writer) - .build() - ); + builder.addSerde(createLongColumnPartSerde(writer, indexSpec)); break; case FLOAT: builder.setValueType(ValueType.FLOAT); - builder.addSerde( - FloatGenericColumnPartSerde - .serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate((FloatColumnSerializer) writer) - .build() - ); + builder.addSerde(createFloatColumnPartSerde(writer, indexSpec)); break; case DOUBLE: builder.setValueType(ValueType.DOUBLE); - builder.addSerde( - DoubleGenericColumnPartSerde - .serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate((DoubleColumnSerializer) writer) - .build() - ); + builder.addSerde(createDoubleColumnPartSerde(writer, indexSpec)); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -389,10 +385,62 @@ private void makeMetricsColumns( progress.stopSection(section); } + static ColumnPartSerde createLongColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec) + { + // If using default values for null use LongGenericColumnPartSerde to allow rollback to previous versions. + if (NullHandling.useDefaultValuesForNull()) { + return LongGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate(serializer) + .build(); + } else { + return LongGenericColumnPartSerdeV2.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withBitmapSerdeFactory(indexSpec.getBitmapSerdeFactory()) + .withDelegate(serializer) + .build(); + } + } + + static ColumnPartSerde createDoubleColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec) + { + // If using default values for null use DoubleGenericColumnPartSerde to allow rollback to previous versions. + if (NullHandling.useDefaultValuesForNull()) { + return DoubleGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate(serializer) + .build(); + } else { + return DoubleGenericColumnPartSerdeV2.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withBitmapSerdeFactory(indexSpec.getBitmapSerdeFactory()) + .withDelegate(serializer) + .build(); + } + } + + static ColumnPartSerde createFloatColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec) + { + // If using default values for null use FloatGenericColumnPartSerde to allow rollback to previous versions. + if (NullHandling.useDefaultValuesForNull()) { + return FloatGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate(serializer) + .build(); + } else { + return FloatGenericColumnPartSerdeV2.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withBitmapSerdeFactory(indexSpec.getBitmapSerdeFactory()) + .withDelegate(serializer) + .build(); + } + } + private void makeTimeColumn( final FileSmoosher v9Smoosher, final ProgressIndicator progress, - final LongColumnSerializer timeWriter + final GenericColumnSerializer timeWriter, + final IndexSpec indexSpec ) throws IOException { final String section = "make time column"; @@ -402,12 +450,7 @@ private void makeTimeColumn( final ColumnDescriptor serdeficator = ColumnDescriptor .builder() .setValueType(ValueType.LONG) - .addSerde( - LongGenericColumnPartSerde.serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate(timeWriter) - .build() - ) + .addSerde(createLongColumnPartSerde(timeWriter, indexSpec)) .build(); makeColumn(v9Smoosher, Column.TIME_COLUMN_NAME, serdeficator); log.info("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); @@ -435,7 +478,7 @@ private void mergeIndexesAndWriteColumns( final List adapters, final ProgressIndicator progress, final Iterable theRows, - final LongColumnSerializer timeWriter, + final GenericColumnSerializer timeWriter, final ArrayList metWriters, final List rowNumConversions, final List mergers @@ -498,13 +541,15 @@ private void mergeIndexesAndWriteColumns( progress.stopSection(section); } - private LongColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec) throws IOException + private GenericColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec) + throws IOException { - LongColumnSerializer timeWriter = LongColumnSerializer.create( + GenericColumnSerializer timeWriter = createLongColumnSerializer( segmentWriteOutMedium, "little_end_time", CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY, - indexSpec.getLongEncoding() + indexSpec.getLongEncoding(), + indexSpec ); // we will close this writer after we added all the timestamps timeWriter.open(); @@ -527,13 +572,13 @@ private ArrayList setupMetricsWriters( GenericColumnSerializer writer; switch (type) { case LONG: - writer = LongColumnSerializer.create(segmentWriteOutMedium, metric, metCompression, longEncoding); + writer = createLongColumnSerializer(segmentWriteOutMedium, metric, metCompression, longEncoding, indexSpec); break; case FLOAT: - writer = FloatColumnSerializer.create(segmentWriteOutMedium, metric, metCompression); + writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, metCompression, indexSpec); break; case DOUBLE: - writer = DoubleColumnSerializer.create(segmentWriteOutMedium, metric, metCompression); + writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, metCompression, indexSpec); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -553,6 +598,81 @@ private ArrayList setupMetricsWriters( return metWriters; } + static GenericColumnSerializer createLongColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, + String columnName, + CompressionStrategy metCompression, + CompressionFactory.LongEncodingStrategy longEncoding, + IndexSpec indexSpec + ) + { + // If using default values for null use LongColumnSerializer to allow rollback to previous versions. + if (NullHandling.useDefaultValuesForNull()) { + return LongColumnSerializer.create( + segmentWriteOutMedium, + columnName, + metCompression, + longEncoding + ); + } else { + return LongColumnSerializerV2.create( + segmentWriteOutMedium, + columnName, + metCompression, + longEncoding, + indexSpec.getBitmapSerdeFactory() + ); + } + } + + static GenericColumnSerializer createDoubleColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, + String columnName, + CompressionStrategy metCompression, + IndexSpec indexSpec + ) + { + // If using default values for null use DoubleColumnSerializer to allow rollback to previous versions. + if (NullHandling.useDefaultValuesForNull()) { + return DoubleColumnSerializer.create( + segmentWriteOutMedium, + columnName, + metCompression + ); + } else { + return DoubleColumnSerializerV2.create( + segmentWriteOutMedium, + columnName, + metCompression, + indexSpec.getBitmapSerdeFactory() + ); + } + } + + static GenericColumnSerializer createFloatColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, + String columnName, + CompressionStrategy metCompression, + IndexSpec indexSpec + ) + { + // If using default values for null use FloatColumnSerializer to allow rollback to previous versions. + if (NullHandling.useDefaultValuesForNull()) { + return FloatColumnSerializer.create( + segmentWriteOutMedium, + columnName, + metCompression + ); + } else { + return FloatColumnSerializerV2.create( + segmentWriteOutMedium, + columnName, + metCompression, + indexSpec.getBitmapSerdeFactory() + ); + } + } + private void writeDimValueAndSetupDimConversion( final List indexes, final ProgressIndicator progress, diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java new file mode 100644 index 000000000000..4c979fd710d3 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java @@ -0,0 +1,142 @@ +/* + * 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.primitives.Ints; +import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.collections.bitmap.MutableBitmap; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.ColumnarLongsSerializer; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** + * Unsafe for concurrent use from multiple threads. + */ +public class LongColumnSerializerV2 implements GenericColumnSerializer +{ + public static LongColumnSerializerV2 create( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + CompressionStrategy compression, + CompressionFactory.LongEncodingStrategy encoding, + BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new LongColumnSerializerV2( + segmentWriteOutMedium, + filenameBase, + IndexIO.BYTE_ORDER, + compression, + encoding, + bitmapSerdeFactory + ); + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final String filenameBase; + private final ByteOrder byteOrder; + private final CompressionStrategy compression; + private final CompressionFactory.LongEncodingStrategy encoding; + private final BitmapSerdeFactory bitmapSerdeFactory; + + private ColumnarLongsSerializer writer; + private ByteBufferWriter nullValueBitmapWriter; + private MutableBitmap nullRowsBitmap; + private int rowCount = 0; + + private LongColumnSerializerV2( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ByteOrder byteOrder, + CompressionStrategy compression, + CompressionFactory.LongEncodingStrategy encoding, + BitmapSerdeFactory bitmapSerdeFactory + ) + { + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.filenameBase = filenameBase; + this.byteOrder = byteOrder; + this.compression = compression; + this.encoding = encoding; + this.bitmapSerdeFactory = bitmapSerdeFactory; + } + + @Override + public void open() throws IOException + { + writer = CompressionFactory.getLongSerializer( + segmentWriteOutMedium, + StringUtils.format("%s.long_column", filenameBase), + byteOrder, + encoding, + compression + ); + writer.open(); + nullValueBitmapWriter = new ByteBufferWriter<>( + segmentWriteOutMedium, + bitmapSerdeFactory.getObjectStrategy() + ); + nullValueBitmapWriter.open(); + nullRowsBitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); + } + + @Override + public void serialize(@Nullable Object obj) throws IOException + { + if (obj == null) { + nullRowsBitmap.add(rowCount); + writer.add(0L); + } else { + writer.add(((Number) obj).longValue()); + } + rowCount++; + } + + @Override + public long getSerializedSize() throws IOException + { + nullValueBitmapWriter.write(bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(nullRowsBitmap)); + long bitmapSize = nullRowsBitmap.isEmpty() + ? 0L + : nullValueBitmapWriter.getSerializedSize(); + return Integer.BYTES + writer.getSerializedSize() + bitmapSize; + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + channel.write(ByteBuffer.wrap(Ints.toByteArray((int) writer.getSerializedSize()))); + writer.writeTo(channel, smoosher); + if (!nullRowsBitmap.isEmpty()) { + nullValueBitmapWriter.writeTo(channel, smoosher); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 29afdde65b9c..ac8cf1a156c9 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -21,6 +21,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.java.util.common.guava.Comparators; import io.druid.query.dimension.DimensionSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.Indexed; @@ -28,10 +29,13 @@ import io.druid.segment.incremental.TimeAndDimsHolder; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.List; +import java.util.Objects; public class LongDimensionIndexer implements DimensionIndexer { + public static final Comparator LONG_COMPARATOR = Comparators.naturalNullsFirst(); @Override public Long processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions) @@ -99,13 +103,20 @@ public long getLong() { final Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { - return 0L; + if (dimIndex >= dims.length || dims[dimIndex] == null) { + return 0; } return (Long) dims[dimIndex]; } + @Override + public boolean isNull() + { + final Object[] dims = currEntry.get().getDims(); + return dimIndex >= dims.length || dims[dimIndex] == null; + } + @SuppressWarnings("deprecation") @Nullable @Override @@ -133,19 +144,19 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public int compareUnsortedEncodedKeyComponents(@Nullable Long lhs, @Nullable Long rhs) { - return DimensionHandlerUtils.nullToZero(lhs).compareTo(DimensionHandlerUtils.nullToZero(rhs)); + return LONG_COMPARATOR.compare(lhs, rhs); } @Override public boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable Long lhs, @Nullable Long rhs) { - return DimensionHandlerUtils.nullToZero(lhs).equals(DimensionHandlerUtils.nullToZero(rhs)); + return Objects.equals(lhs, rhs); } @Override public int getUnsortedEncodedKeyComponentHashCode(@Nullable Long key) { - return DimensionHandlerUtils.nullToZero(key).hashCode(); + return DimensionHandlerUtils.nullToZeroLong(key).hashCode(); } @Override diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index ce515ad1020b..f7c01805044c 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -24,7 +24,7 @@ import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; -import io.druid.segment.serde.LongGenericColumnPartSerde; +import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; @@ -35,7 +35,7 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 { protected String dimensionName; protected final IndexSpec indexSpec; - protected LongColumnSerializer serializer; + protected GenericColumnSerializer serializer; LongDimensionMergerV9( String dimensionName, @@ -58,7 +58,14 @@ protected void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedi { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - this.serializer = LongColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression, longEncoding); + + this.serializer = IndexMergerV9.createLongColumnSerializer( + segmentWriteOutMedium, + dimensionName, + metCompression, + longEncoding, + indexSpec + ); serializer.open(); } @@ -89,7 +96,6 @@ public void writeIndexes(List segmentRowNumConversions) throws IOExce @Override public boolean canSkip() { - // a long column can never be all null return false; } @@ -98,12 +104,8 @@ public ColumnDescriptor makeColumnDescriptor() throws IOException { final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.LONG); - builder.addSerde( - LongGenericColumnPartSerde.serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate(serializer) - .build() - ); + ColumnPartSerde serde = IndexMergerV9.createLongColumnPartSerde(serializer, indexSpec); + builder.addSerde(serde); return builder.build(); } } diff --git a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java index 26ba95715743..c19208711a42 100644 --- a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java @@ -85,6 +85,12 @@ public Class classOfObject() return Object.class; } + @Override + public boolean isNull() + { + return true; + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java index 7c8969305444..56ad73fcae1e 100644 --- a/processing/src/main/java/io/druid/segment/NullDimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/NullDimensionSelector.java @@ -20,7 +20,7 @@ package io.druid.segment; import com.google.common.base.Predicate; -import com.google.common.base.Strings; +import io.druid.common.config.NullHandling; import io.druid.query.filter.ValueMatcher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.IndexedInts; @@ -103,7 +103,7 @@ public IdLookup idLookup() @Override public int lookupId(String name) { - return Strings.isNullOrEmpty(name) ? 0 : -1; + return NullHandling.isNullOrEquivalent(name) ? 0 : -1; } @Nullable diff --git a/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java b/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java index 2cf5a6c2173a..c9d739183235 100644 --- a/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java @@ -76,4 +76,17 @@ default long getLong() } return ((Number) value).longValue(); } + + /** + * @deprecated This method is marked as deprecated in ObjectColumnSelector to minimize the probability of accidential + * calling. "Polymorphism" of ObjectColumnSelector should be used only when operating on {@link ColumnValueSelector} + * objects. + */ + @Deprecated + @Override + default boolean isNull() + { + T value = getObject(); + return value == null; + } } diff --git a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java index 675077021632..431c050bde8a 100644 --- a/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/io/druid/segment/QueryableIndexIndexableAdapter.java @@ -39,8 +39,8 @@ import io.druid.segment.column.FloatsColumn; import io.druid.segment.column.LongsColumn; import io.druid.segment.column.ValueType; -import io.druid.segment.data.ImmutableBitmapValues; import io.druid.segment.data.BitmapValues; +import io.druid.segment.data.ImmutableBitmapValues; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.ListIndexed; @@ -270,11 +270,14 @@ public Rowboat next() Object[] metricArray = new Object[numMetrics]; for (int i = 0; i < metricArray.length; ++i) { if (metrics[i] instanceof FloatsColumn) { - metricArray[i] = ((GenericColumn) metrics[i]).getFloatSingleValueRow(currRow); + GenericColumn genericColumn = (GenericColumn) metrics[i]; + metricArray[i] = genericColumn.isNull(currRow) ? null : genericColumn.getFloatSingleValueRow(currRow); } else if (metrics[i] instanceof DoublesColumn) { - metricArray[i] = ((GenericColumn) metrics[i]).getDoubleSingleValueRow(currRow); + GenericColumn genericColumn = (GenericColumn) metrics[i]; + metricArray[i] = genericColumn.isNull(currRow) ? null : genericColumn.getDoubleSingleValueRow(currRow); } else if (metrics[i] instanceof LongsColumn) { - metricArray[i] = ((GenericColumn) metrics[i]).getLongSingleValueRow(currRow); + GenericColumn genericColumn = (GenericColumn) metrics[i]; + metricArray[i] = genericColumn.isNull(currRow) ? null : genericColumn.getLongSingleValueRow(currRow); } else if (metrics[i] instanceof ComplexColumn) { metricArray[i] = ((ComplexColumn) metrics[i]).getRowValue(currRow); } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index cd4db8a3db46..fe3387f5548b 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -77,6 +77,20 @@ public int compareSortedEncodedKeyComponents(int[] lhs, int[] rhs) return retVal; } + private boolean isNUllRow(int[] row, Indexed encodings) + { + if (row == null) { + return true; + } + for (int i : row) { + if (encodings.get(i) != null) { + // Non-Null value + return false; + } + } + return true; + } + @Override public void validateSortedEncodedKeyComponents( int[] lhs, @@ -86,7 +100,7 @@ public void validateSortedEncodedKeyComponents( ) throws SegmentValidationException { if (lhs == null || rhs == null) { - if (lhs != null || rhs != null) { + if (!isNUllRow(lhs, lhsEncodings) || !isNUllRow(rhs, rhsEncodings)) { throw new SegmentValidationException( "Expected nulls, found %s and %s", Arrays.toString(lhs), diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index af994ecfd6e9..08e4b8592cab 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -21,11 +21,11 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; -import com.google.common.base.Strings; import com.google.common.collect.Lists; import com.google.common.primitives.Ints; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Comparators; @@ -58,13 +58,18 @@ public class StringDimensionIndexer implements DimensionIndexer { - private static final Function STRING_TRANSFORMER = o -> o != null ? o.toString() : null; + private static final Function EMPTY_TO_NULL_IF_NEEDED = o -> o != null + ? NullHandling.emptyToNullIfNeeded(o.toString()) + : null; + + private static final int ABSENT_VALUE_ID = -1; private static final int[] EMPTY_INT_ARRAY = new int[]{}; private static class DimensionDictionary { private String minValue = null; private String maxValue = null; + private int idForNull = ABSENT_VALUE_ID; private final Object2IntMap valueToId = new Object2IntOpenHashMap<>(); @@ -80,37 +85,59 @@ public DimensionDictionary() public int getId(String value) { synchronized (lock) { - return valueToId.getInt(Strings.nullToEmpty(value)); + if (value == null) { + return idForNull; + } + return valueToId.getInt(value); } } public String getValue(int id) { synchronized (lock) { - return Strings.emptyToNull(idToValue.get(id)); + if (id == idForNull) { + return null; + } + return idToValue.get(id); + } + } + + public boolean contains(String value) + { + synchronized (lock) { + if (value == null) { + return idForNull != ABSENT_VALUE_ID; + } + return valueToId.containsKey(value); } } public int size() { synchronized (lock) { - return valueToId.size(); + return idToValue.size(); } } public int add(String originalValue) { - String value = Strings.nullToEmpty(originalValue); synchronized (lock) { - int prev = valueToId.getInt(value); + if (originalValue == null) { + if (idForNull == ABSENT_VALUE_ID) { + idForNull = size(); + idToValue.add(null); + } + return idForNull; + } + int prev = valueToId.getInt(originalValue); if (prev >= 0) { return prev; } final int index = size(); - valueToId.put(value, index); - idToValue.add(value); - minValue = minValue == null || minValue.compareTo(value) > 0 ? value : minValue; - maxValue = maxValue == null || maxValue.compareTo(value) < 0 ? value : maxValue; + valueToId.put(originalValue, index); + idToValue.add(originalValue); + minValue = minValue == null || minValue.compareTo(originalValue) > 0 ? originalValue : minValue; + maxValue = maxValue == null || maxValue.compareTo(originalValue) < 0 ? originalValue : maxValue; return index; } } @@ -145,9 +172,10 @@ private static class SortedDimensionDictionary public SortedDimensionDictionary(List idToValue, int length) { - Object2IntSortedMap sortedMap = new Object2IntRBTreeMap<>(); + Object2IntSortedMap sortedMap = new Object2IntRBTreeMap<>(Comparators.naturalNullsFirst()); for (int id = 0; id < length; id++) { - sortedMap.put(idToValue.get(id), id); + String value = idToValue.get(id); + sortedMap.put(value, id); } this.sortedVals = Lists.newArrayList(sortedMap.keySet()); this.idToIndex = new int[length]; @@ -173,7 +201,7 @@ public int getSortedIdFromUnsortedId(int id) public String getValueFromSortedId(int index) { - return Strings.emptyToNull(sortedVals.get(index)); + return sortedVals.get(index); } } @@ -194,19 +222,22 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole final int oldDictSize = dimLookup.size(); if (dimValues == null) { - dimLookup.add(null); - encodedDimensionValues = null; + if (!dimLookup.contains(null)) { + encodedDimensionValues = new int[]{dimLookup.add(null)}; + } else { + encodedDimensionValues = new int[]{dimLookup.getId(null)}; + } } else if (dimValues instanceof List) { List dimValuesList = (List) dimValues; if (dimValuesList.isEmpty()) { dimLookup.add(null); encodedDimensionValues = EMPTY_INT_ARRAY; } else if (dimValuesList.size() == 1) { - encodedDimensionValues = new int[]{dimLookup.add(STRING_TRANSFORMER.apply(dimValuesList.get(0)))}; + encodedDimensionValues = new int[]{dimLookup.add(EMPTY_TO_NULL_IF_NEEDED.apply(dimValuesList.get(0)))}; } else { final String[] dimensionValues = new String[dimValuesList.size()]; for (int i = 0; i < dimValuesList.size(); i++) { - dimensionValues[i] = STRING_TRANSFORMER.apply(dimValuesList.get(i)); + dimensionValues[i] = EMPTY_TO_NULL_IF_NEEDED.apply(dimValuesList.get(i)); } if (multiValueHandling.needSorting()) { // Sort multival row by their unencoded values first. @@ -231,7 +262,7 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole encodedDimensionValues = pos == retVal.length ? retVal : Arrays.copyOf(retVal, pos); } } else { - encodedDimensionValues = new int[]{dimLookup.add(STRING_TRANSFORMER.apply(dimValues))}; + encodedDimensionValues = new int[]{dimLookup.add(EMPTY_TO_NULL_IF_NEEDED.apply(dimValues))}; } // If dictionary size has changed, the sorted lookup is no longer valid. @@ -280,7 +311,7 @@ public String get(int index) public int indexOf(String value) { int id = getEncodedValue(value, false); - return id < 0 ? -1 : getSortedEncodedValueFromUnsorted(id); + return id < 0 ? ABSENT_VALUE_ID : getSortedEncodedValueFromUnsorted(id); } @Override @@ -583,22 +614,20 @@ public Object convertUnsortedEncodedKeyComponentToActualArrayOrList(int[] key, b return null; } if (key.length == 1) { - String val = getActualValue(key[0], false); - val = Strings.nullToEmpty(val); - return val; + return getActualValue(key[0], false); } else { if (asList) { List rowVals = new ArrayList<>(key.length); for (int id : key) { String val = getActualValue(id, false); - rowVals.add(Strings.nullToEmpty(val)); + rowVals.add(NullHandling.nullToEmptyIfNeeded(val)); } return rowVals; } else { String[] rowArray = new String[key.length]; for (int i = 0; i < key.length; i++) { String val = getActualValue(key[i], false); - rowArray[i] = Strings.nullToEmpty(val); + rowArray[i] = NullHandling.nullToEmptyIfNeeded(val); } return rowArray; } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 8e97666bc184..ca738effb011 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -20,7 +20,6 @@ package io.druid.segment; import com.google.common.base.Splitter; -import com.google.common.base.Strings; import com.google.common.collect.Lists; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; @@ -28,6 +27,7 @@ import io.druid.collections.spatial.ImmutableRTree; import io.druid.collections.spatial.RTree; import io.druid.collections.spatial.split.LinearGutmanSplitStrategy; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; @@ -63,8 +63,11 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 { private static final Logger log = new Logger(StringDimensionMergerV9.class); - private static final Indexed EMPTY_STR_DIM_VAL = new ArrayIndexed<>(new String[]{""}, String.class); - private static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0}; + protected static final Indexed NULL_STR_DIM_VAL = new ArrayIndexed<>( + new String[]{(String) null}, + String.class + ); + protected static final int[] NULL_STR_DIM_ARRAY = new int[]{0}; private static final Splitter SPLITTER = Splitter.on(","); private ColumnarIntsSerializer encodedValueWriter; @@ -145,7 +148,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE */ if (convertMissingValues && !hasNull) { hasNull = true; - dimValueLookups[adapters.size()] = dimValueLookup = EMPTY_STR_DIM_VAL; + dimValueLookups[adapters.size()] = dimValueLookup = NULL_STR_DIM_VAL; numMergeIndex++; } @@ -184,7 +187,7 @@ private void writeDictionary(Iterable dictionaryValues) throws IOExcepti { for (String value : dictionaryValues) { dictionaryWriter.write(value); - value = Strings.emptyToNull(value); + value = NullHandling.emptyToNullIfNeeded(value); if (dictionarySize == 0) { firstDictionaryValue = value; } @@ -231,7 +234,7 @@ public int[] convertSegmentRowValuesToMergedRowValues(int[] segmentRow, int segm // For strings, convert missing values to null/empty if conversion flag is set // But if bitmap/dictionary is not used, always convert missing to 0 if (dimVals == null) { - return convertMissingValues ? EMPTY_STR_DIM_ARRAY : null; + return convertMissingValues ? NULL_STR_DIM_ARRAY : null; } int[] newDimVals = new int[dimVals.length]; diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java index 5601d39f4c93..55c499698ab9 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarDoubles; @@ -28,10 +29,12 @@ public class DoublesColumn implements GenericColumn { private final ColumnarDoubles column; + private final ImmutableBitmap nullValueBitmap; - public DoublesColumn(ColumnarDoubles columnarDoubles) + public DoublesColumn(ColumnarDoubles columnarDoubles, ImmutableBitmap nullValueBitmap) { column = columnarDoubles; + this.nullValueBitmap = nullValueBitmap; } @Override @@ -43,7 +46,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset); + return column.makeColumnValueSelector(offset, nullValueBitmap); } @Override @@ -64,6 +67,12 @@ public double getDoubleSingleValueRow(int rowNum) return column.get(rowNum); } + @Override + public boolean isNull(int rowNum) + { + return nullValueBitmap.get(rowNum); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index 5bc381c19c17..51a869b1c98d 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarFloats; @@ -29,10 +30,15 @@ public class FloatsColumn implements GenericColumn { private final ColumnarFloats column; + private final ImmutableBitmap nullValueBitmap; - public FloatsColumn(final ColumnarFloats column) + public FloatsColumn( + final ColumnarFloats column, + ImmutableBitmap nullValueBitmap + ) { this.column = column; + this.nullValueBitmap = nullValueBitmap; } @Override @@ -44,7 +50,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset); + return column.makeColumnValueSelector(offset, nullValueBitmap); } @Override @@ -65,6 +71,12 @@ public double getDoubleSingleValueRow(int rowNum) return (double) column.get(rowNum); } + @Override + public boolean isNull(int rowNum) + { + return nullValueBitmap.get(rowNum); + } + @Override public void close() { 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 2c277db7a0d2..c00a3054e9ad 100644 --- a/processing/src/main/java/io/druid/segment/column/GenericColumn.java +++ b/processing/src/main/java/io/druid/segment/column/GenericColumn.java @@ -37,6 +37,9 @@ public interface GenericColumn extends BaseColumn, HotLoopCallee @CalledFromHotLoop double getDoubleSingleValueRow(int rowNum); + @CalledFromHotLoop + boolean isNull(int rowNum); + @Override void close(); } diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 4f67a4b9e565..2b5c5b864713 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarLongs; @@ -29,10 +30,15 @@ public class LongsColumn implements GenericColumn { private final ColumnarLongs column; + private final ImmutableBitmap nullValueBitmap; - public LongsColumn(final ColumnarLongs column) + public LongsColumn( + final ColumnarLongs column, + ImmutableBitmap nullValueBitmap + ) { this.column = column; + this.nullValueBitmap = nullValueBitmap; } @Override @@ -44,7 +50,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset); + return column.makeColumnValueSelector(offset, nullValueBitmap); } @Override @@ -65,6 +71,12 @@ public double getDoubleSingleValueRow(int rowNum) return (double) column.get(rowNum); } + @Override + public boolean isNull(int rowNum) + { + return nullValueBitmap.get(rowNum); + } + @Override public void close() { diff --git a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index 3d67beb22a2b..b2dfca272137 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -21,7 +21,6 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; -import com.google.common.base.Strings; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.ValueMatcher; @@ -86,10 +85,10 @@ public IndexedInts getMultiValueRow(int rowNum) } @Override + @Nullable public String lookupName(int id) { - //Empty to Null will ensure that null and empty are equivalent for extraction function - return Strings.emptyToNull(cachedLookups.get(id)); + return cachedLookups.get(id); } @Override @@ -177,6 +176,13 @@ public ValueMatcher makeValueMatcher(Predicate predicate) return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); } + @Override + public boolean isNull() + { + IndexedInts row = getRow(); + return row == null || row.size() == 0; + } + @Nullable @Override public Object getObject() @@ -282,6 +288,12 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } + @Override + public boolean isNull() + { + return getObject() == null; + } + @Override public Object getObject() { diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java index 407ef6947407..e4ac821d5455 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DoubleColumnSelector; @@ -38,7 +39,7 @@ public interface ColumnarDoubles extends Closeable @Override void close(); - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector { @@ -54,6 +55,12 @@ public double getDouble(int offset) return ColumnarDoubles.this.get(offset); } + @Override + public boolean isNull() + { + return nullValueBitmap.get(offset.getOffset()); + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java index 582b254ca0d1..b0a701047a89 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.FloatColumnSelector; @@ -39,7 +40,7 @@ public interface ColumnarFloats extends Closeable @Override void close(); - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector { @@ -49,6 +50,12 @@ public float getFloat() return ColumnarFloats.this.get(offset.getOffset()); } + @Override + public boolean isNull() + { + return nullValueBitmap.get(offset.getOffset()); + } + @Override public double getDouble(int offset) { diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java index 5a60fc50744b..de67a2d25071 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java @@ -19,6 +19,7 @@ package io.druid.segment.data; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.LongColumnSelector; @@ -39,7 +40,7 @@ public interface ColumnarLongs extends Closeable @Override void close(); - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector { @@ -55,6 +56,12 @@ public double getDouble(int offset) return ColumnarLongs.this.get(offset); } + @Override + public boolean isNull() + { + return nullValueBitmap.get(offset.getOffset()); + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 86ef78b65759..2e08985e050c 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -19,9 +19,9 @@ package io.druid.segment.data; -import com.google.common.base.Strings; import com.google.common.primitives.Ints; import io.druid.collections.ResourceHolder; +import io.druid.common.config.NullHandling; import io.druid.common.utils.SerializerUtils; import io.druid.io.Channels; import io.druid.java.util.common.IAE; @@ -35,8 +35,7 @@ import io.druid.segment.serde.MetaSerdeHelper; import io.druid.segment.serde.Serializer; import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; -import it.unimi.dsi.fastutil.bytes.ByteArrays; - +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; @@ -98,18 +97,20 @@ public Class getClazz() } @Override + @Nullable public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { - return StringUtils.fromUtf8(buffer, numBytes); + if (numBytes < 0) { + // nulBytes will be -1 for null values. + return null; + } + return NullHandling.emptyToNullIfNeeded(StringUtils.fromUtf8Nullable(buffer, numBytes)); } @Override public byte[] toBytes(String val) { - if (Strings.isNullOrEmpty(val)) { - return ByteArrays.EMPTY_ARRAY; - } - return StringUtils.toUtf8(val); + return StringUtils.toUtf8Nullable(NullHandling.nullToEmptyIfNeeded(val)); } @Override @@ -309,8 +310,6 @@ private int indexOf(Indexed indexed, T value) throw new UnsupportedOperationException("Reverse lookup not allowed."); } - value = (value != null && value.equals("")) ? null : value; - int minIndex = 0; int maxIndex = size - 1; while (minIndex <= maxIndex) { @@ -370,11 +369,8 @@ public GenericIndexed.BufferIndexed singleThreaded() private T copyBufferAndGet(ByteBuffer valueBuffer, int startOffset, int endOffset) { - final int size = endOffset - startOffset; - if (size == 0) { - return null; - } ByteBuffer copyValueBuffer = valueBuffer.asReadOnlyBuffer(); + final int size = endOffset > startOffset ? endOffset - startOffset : copyValueBuffer.get(startOffset - Ints.BYTES); copyValueBuffer.position(startOffset); // fromByteBuffer must not modify the buffer limit return strategy.fromByteBuffer(copyValueBuffer, size); @@ -413,11 +409,11 @@ public int size() T bufferedIndexedGet(ByteBuffer copyValueBuffer, int startOffset, int endOffset) { - final int size = endOffset - startOffset; + final int size = endOffset > startOffset + ? endOffset - startOffset + : copyValueBuffer.get(startOffset - Ints.BYTES); lastReadSize = size; - if (size == 0) { - return null; - } + // ObjectStrategy.fromByteBuffer() is allowed to reset the limit of the buffer. So if the limit is changed, // position() call in the next line could throw an exception, if the position is set beyond the new limit. clear() // sets the limit to the maximum possible, the capacity. It is safe to reset the limit to capacity, because the @@ -496,11 +492,11 @@ private static GenericIndexed fromIterableVersionOne( allowReverseLookup = false; } - // for compatibility with the format, but this field is unused - valuesOut.writeInt(0); + valuesOut.writeInt(next == null ? -1 : 0); if (next != null) { strategy.writeTo(next, valuesOut); } + headerOut.writeInt(Ints.checkedCast(valuesOut.size())); if (prevVal instanceof Closeable) { @@ -563,7 +559,7 @@ public T get(final int index) final int endOffset; if (index == 0) { - startOffset = 4; + startOffset = Ints.BYTES; endOffset = headerBuffer.getInt(0); } else { int headerPosition = (index - 1) * Integer.BYTES; diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index cd19949443ec..20e6374a328e 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -214,9 +214,8 @@ public void write(T objectToWrite) throws IOException } ++numWritten; - // for compatibility with the format (see GenericIndexed javadoc for description of the format), but this field is - // unused. - valuesOut.writeInt(0); + + valuesOut.writeInt(objectToWrite == null ? -1 : 0); if (objectToWrite != null) { strategy.writeTo(objectToWrite, valuesOut); } diff --git a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java index fb4135e2ab23..811dd14cae8a 100644 --- a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java @@ -22,6 +22,7 @@ import io.druid.guice.annotations.ExtensionPoint; import io.druid.segment.writeout.WriteOutBytes; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Comparator; @@ -44,8 +45,11 @@ public interface ObjectStrategy extends Comparator * @param numBytes number of bytes used to store the value, starting at buffer.position() * @return an object created from the given byte buffer representation */ + @Nullable T fromByteBuffer(ByteBuffer buffer, int numBytes); - byte[] toBytes(T val); + + @Nullable + byte[] toBytes(@Nullable T val); /** * Reads 4-bytes numBytes from the given buffer, and then delegates to {@link #fromByteBuffer(ByteBuffer, int)}. @@ -62,6 +66,9 @@ default T fromByteBufferWithSize(ByteBuffer buffer) default void writeTo(T val, WriteOutBytes out) throws IOException { - out.write(toBytes(val)); + byte[] bytes = toBytes(val); + if (bytes != null) { + out.write(toBytes(val)); + } } } diff --git a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java index cb76d50189d7..4125927f8937 100644 --- a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -79,6 +79,9 @@ public Class getClazz() @Override public ImmutableBitmap fromByteBuffer(ByteBuffer buffer, int numBytes) { + if (numBytes == 0) { + return null; + } buffer.limit(buffer.position() + numBytes); return new WrappedImmutableRoaringBitmap(new ImmutableRoaringBitmap(buffer)); } 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 dd5daa9281e8..da60fe9b9e51 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.query.BitmapResultFactory; import io.druid.query.ColumnSelectorPlus; @@ -132,7 +133,7 @@ public static ValueMatcher makeValueMatcher( columnSelectorFactory ); - return selector.getColumnSelectorStrategy().makeValueMatcher(selector.getSelector(), value); + return selector.getColumnSelectorStrategy().makeValueMatcher(selector.getSelector(), NullHandling.emptyToNullIfNeeded(value)); } /** 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 d2f19696129c..8655b9f7d559 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -151,22 +151,28 @@ public ColumnValueSelector makeColumnValueSelector(final String column) final ComplexMetricExtractor extractor = serde.getExtractor(); return new ColumnValueSelector() { + @Override + public boolean isNull() + { + return in.get().getMetric(column) == null; + } + @Override public long getLong() { - return in.get().getMetric(column).longValue(); + return DimensionHandlerUtils.nullToZeroLong(in.get().getMetric(column)).longValue(); } @Override public float getFloat() { - return in.get().getMetric(column).floatValue(); + return DimensionHandlerUtils.nullToZeroFloat(in.get().getMetric(column)).floatValue(); } @Override public double getDouble() { - return in.get().getMetric(column).doubleValue(); + return DimensionHandlerUtils.nullToZeroDouble(in.get().getMetric(column)).doubleValue(); } @Override @@ -461,6 +467,9 @@ protected abstract Integer addToFacts( protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); + protected abstract boolean isNull(int rowOffset, int aggOffset); + + @Override public void close() { @@ -1400,6 +1409,12 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("index", IncrementalIndex.this); } + + @Override + public boolean isNull() + { + return IncrementalIndex.this.isNull(currEntry.getValue(), metricIndex); + } } private class ObjectMetricColumnSelector implements ObjectColumnSelector @@ -1461,6 +1476,12 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("index", IncrementalIndex.this); } + + @Override + public boolean isNull() + { + return IncrementalIndex.this.isNull(currEntry.getValue(), metricIndex); + } } private class DoubleMetricColumnSelector implements DoubleColumnSelector @@ -1480,6 +1501,12 @@ public double getDouble() return getMetricDoubleValue(currEntry.getValue(), metricIndex); } + @Override + public boolean isNull() + { + return IncrementalIndex.this.isNull(currEntry.getValue(), metricIndex); + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index d965b80aa120..23afec83166a 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -121,6 +121,12 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { // nothing to inspect } + + @Override + public boolean isNull() + { + return false; + } } return new TimeLongColumnSelector(); } 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 3d45604b69c9..d377634e0bc8 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OffheapIncrementalIndex.java @@ -313,6 +313,15 @@ public double getMetricDoubleValue(int rowOffset, int aggOffset) return agg.getDouble(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } + @Override + public boolean isNull(int rowOffset, int aggOffset) + { + BufferAggregator agg = getAggs()[aggOffset]; + int[] indexAndOffset = indexAndOffsets.get(rowOffset); + ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); + return agg.isNull(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + } + /** * NOTE: This is NOT thread-safe with add... so make sure all the adding is DONE before closing */ 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 c41b63a3376b..1a377ea826be 100644 --- a/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/OnheapIncrementalIndex.java @@ -286,6 +286,12 @@ protected double getMetricDoubleValue(int rowOffset, int aggOffset) return concurrentGet(rowOffset)[aggOffset].getDouble(); } + @Override + public boolean isNull(int rowOffset, int aggOffset) + { + return concurrentGet(rowOffset)[aggOffset].isNull(); + } + /** * Clear out maps to allow GC * NOTE: This is NOT thread-safe with add... so make sure all the adding is DONE before closing diff --git a/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java b/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java index ea204ced0d11..72d20f9c2355 100644 --- a/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/BitmapIndexColumnPartSupplier.java @@ -22,6 +22,7 @@ import com.google.common.base.Supplier; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.segment.column.BitmapIndex; import io.druid.segment.data.GenericIndexed; @@ -77,7 +78,7 @@ public BitmapFactory getBitmapFactory() public int getIndex(String value) { // GenericIndexed.indexOf satisfies contract needed by BitmapIndex.indexOf - return dictionary.indexOf(value); + return dictionary.indexOf(NullHandling.emptyToNullIfNeeded(value)); } @Override 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 4c7d7b1fd86c..6662e60269bc 100644 --- a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -34,7 +34,10 @@ @JsonSubTypes.Type(name = "float", value = FloatGenericColumnPartSerde.class), @JsonSubTypes.Type(name = "long", value = LongGenericColumnPartSerde.class), @JsonSubTypes.Type(name = "double", value = DoubleGenericColumnPartSerde.class), - @JsonSubTypes.Type(name = "stringDictionary", value = DictionaryEncodedColumnPartSerde.class) + @JsonSubTypes.Type(name = "stringDictionary", value = DictionaryEncodedColumnPartSerde.class), + @JsonSubTypes.Type(name = "floatV2", value = FloatGenericColumnPartSerdeV2.class), + @JsonSubTypes.Type(name = "longV2", value = LongGenericColumnPartSerdeV2.class), + @JsonSubTypes.Type(name = "doubleV2", value = DoubleGenericColumnPartSerdeV2.class), }) public interface ColumnPartSerde { diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java index 3a545ec675e4..e467683cbf49 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; -import io.druid.segment.DoubleColumnSerializer; +import io.druid.segment.IndexIO; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; @@ -65,7 +65,7 @@ public static SerializerBuilder serializerBuilder() public static class SerializerBuilder { private ByteOrder byteOrder = null; - private DoubleColumnSerializer delegate = null; + private Serializer delegate = null; public SerializerBuilder withByteOrder(final ByteOrder byteOrder) { @@ -73,7 +73,7 @@ public SerializerBuilder withByteOrder(final ByteOrder byteOrder) return this; } - public SerializerBuilder withDelegate(final DoubleColumnSerializer delegate) + public SerializerBuilder withDelegate(final Serializer delegate) { this.delegate = delegate; return this; @@ -105,7 +105,8 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo ); builder.setType(ValueType.DOUBLE) .setHasMultipleValues(false) - .setGenericColumn(new DoubleGenericColumnSupplier(column)); + .setGenericColumn(new DoubleGenericColumnSupplier(column, IndexIO.LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap())); } }; diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java new file mode 100644 index 000000000000..b371dd0dee33 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java @@ -0,0 +1,162 @@ +/* + * 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 com.google.common.base.Supplier; +import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerde; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ColumnarDoubles; +import io.druid.segment.data.CompressedColumnarDoublesSuppliers; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class DoubleGenericColumnPartSerdeV2 implements ColumnPartSerde +{ + private final ByteOrder byteOrder; + private Serializer serialize; + private final BitmapSerdeFactory bitmapSerdeFactory; + + @JsonCreator + public static DoubleGenericColumnPartSerdeV2 getDoubleGenericColumnPartSerde( + @JsonProperty("byteOrder") ByteOrder byteOrder, + @Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new DoubleGenericColumnPartSerdeV2(byteOrder, + bitmapSerdeFactory != null + ? bitmapSerdeFactory + : new BitmapSerde.LegacyBitmapSerdeFactory(), null + ); + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() + { + return bitmapSerdeFactory; + } + + public DoubleGenericColumnPartSerdeV2( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + Serializer serialize + ) + { + this.byteOrder = byteOrder; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.serialize = serialize; + } + + @Override + public Serializer getSerializer() + { + return serialize; + } + + @Override + public Deserializer getDeserializer() + { + return (buffer, builder, columnConfig) -> { + int offset = buffer.getInt(); + int initialPos = buffer.position(); + final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( + buffer, + byteOrder + ); + + buffer.position(initialPos + offset); + final ImmutableBitmap bitmap; + if (buffer.hasRemaining()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + builder.setType(ValueType.DOUBLE) + .setHasMultipleValues(false) + .setGenericColumn(new DoubleGenericColumnSupplier(column, bitmap)); + }; + } + + public static SerializerBuilder serializerBuilder() + { + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private ByteOrder byteOrder = null; + private Serializer delegate = null; + private BitmapSerdeFactory bitmapSerdeFactory = null; + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withDelegate(final Serializer delegate) + { + this.delegate = delegate; + return this; + } + + public SerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) + { + this.bitmapSerdeFactory = bitmapSerdeFactory; + return this; + } + + public DoubleGenericColumnPartSerdeV2 build() + { + return new DoubleGenericColumnPartSerdeV2( + byteOrder, + bitmapSerdeFactory, + new Serializer() + { + @Override + public long getSerializedSize() throws IOException + { + return delegate.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher fileSmoosher) throws IOException + { + delegate.writeTo(channel, fileSmoosher); + } + } + ); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java index 148bbcbbd609..ae4ea7a50ba5 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java @@ -20,6 +20,7 @@ package io.druid.segment.serde; import com.google.common.base.Supplier; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.DoublesColumn; import io.druid.segment.data.ColumnarDoubles; @@ -28,15 +29,20 @@ public class DoubleGenericColumnSupplier implements Supplier { private final Supplier column; + private final ImmutableBitmap nullValueBitmap; - public DoubleGenericColumnSupplier(Supplier column) + public DoubleGenericColumnSupplier( + Supplier column, + ImmutableBitmap nullValueBitmap + ) { this.column = column; + this.nullValueBitmap = nullValueBitmap; } @Override public GenericColumn get() { - return new DoublesColumn(column.get()); + return new DoublesColumn(column.get(), nullValueBitmap); } } 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 c62c5e744623..9d3810c53df9 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.segment.FloatColumnSerializer; +import io.druid.segment.IndexIO; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; @@ -65,7 +65,7 @@ public static SerializerBuilder serializerBuilder() public static class SerializerBuilder { private ByteOrder byteOrder = null; - private FloatColumnSerializer delegate = null; + private Serializer delegate = null; public SerializerBuilder withByteOrder(final ByteOrder byteOrder) { @@ -73,7 +73,7 @@ public SerializerBuilder withByteOrder(final ByteOrder byteOrder) return this; } - public SerializerBuilder withDelegate(final FloatColumnSerializer delegate) + public SerializerBuilder withDelegate(final Serializer delegate) { this.delegate = delegate; return this; @@ -105,7 +105,11 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo ); builder.setType(ValueType.FLOAT) .setHasMultipleValues(false) - .setGenericColumn(new FloatGenericColumnSupplier(column)); + .setGenericColumn(new FloatGenericColumnSupplier( + column, + IndexIO.LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap() + )); } }; } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java new file mode 100644 index 000000000000..604c5abb5038 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java @@ -0,0 +1,161 @@ +/* + * 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.collections.bitmap.ImmutableBitmap; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerde; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.CompressedColumnarFloatsSupplier; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** + */ +public class FloatGenericColumnPartSerdeV2 implements ColumnPartSerde +{ + @JsonCreator + public static FloatGenericColumnPartSerdeV2 createDeserializer( + @JsonProperty("byteOrder") ByteOrder byteOrder, + @Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new FloatGenericColumnPartSerdeV2( + byteOrder, + bitmapSerdeFactory != null ? bitmapSerdeFactory : new BitmapSerde.LegacyBitmapSerdeFactory(), + null + ); + } + + private final ByteOrder byteOrder; + private final BitmapSerdeFactory bitmapSerdeFactory; + private Serializer serializer; + + private FloatGenericColumnPartSerdeV2( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + Serializer serializer + ) + { + this.byteOrder = byteOrder; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.serializer = serializer; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() + { + return bitmapSerdeFactory; + } + + public static SerializerBuilder serializerBuilder() + { + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private ByteOrder byteOrder = null; + private Serializer delegate = null; + private BitmapSerdeFactory bitmapSerdeFactory = null; + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withDelegate(final Serializer delegate) + { + this.delegate = delegate; + return this; + } + + public SerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) + { + this.bitmapSerdeFactory = bitmapSerdeFactory; + return this; + } + + public FloatGenericColumnPartSerdeV2 build() + { + return new FloatGenericColumnPartSerdeV2( + byteOrder, bitmapSerdeFactory, + new Serializer() + { + @Override + public long getSerializedSize() throws IOException + { + return delegate.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher fileSmoosher) throws IOException + { + delegate.writeTo(channel, fileSmoosher); + } + } + ); + } + + } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + public Deserializer getDeserializer() + { + return (buffer, builder, columnConfig) -> { + int offset = buffer.getInt(); + int initialPos = buffer.position(); + final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( + buffer, + byteOrder + ); + buffer.position(initialPos + offset); + final ImmutableBitmap bitmap; + if (buffer.hasRemaining()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + builder.setType(ValueType.FLOAT) + .setHasMultipleValues(false) + .setGenericColumn(new FloatGenericColumnSupplier(column, bitmap)); + + }; + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java index 98568c8d2bd7..0d40a02cadb4 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java @@ -20,6 +20,7 @@ package io.druid.segment.serde; import com.google.common.base.Supplier; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.FloatsColumn; import io.druid.segment.data.CompressedColumnarFloatsSupplier; @@ -29,15 +30,20 @@ public class FloatGenericColumnSupplier implements Supplier { private final CompressedColumnarFloatsSupplier column; + private final ImmutableBitmap nullValueBitmap; - public FloatGenericColumnSupplier(CompressedColumnarFloatsSupplier column) + public FloatGenericColumnSupplier( + CompressedColumnarFloatsSupplier column, + ImmutableBitmap nullValueBitmap + ) { this.column = column; + this.nullValueBitmap = nullValueBitmap; } @Override public GenericColumn get() { - return new FloatsColumn(column.get()); + return new FloatsColumn(column.get(), nullValueBitmap); } } 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 a73fde430e13..bc926713b9bd 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.segment.LongColumnSerializer; +import io.druid.segment.IndexIO; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; @@ -65,7 +65,7 @@ public static SerializerBuilder serializerBuilder() public static class SerializerBuilder { private ByteOrder byteOrder = null; - private LongColumnSerializer delegate = null; + private Serializer delegate = null; public SerializerBuilder withByteOrder(final ByteOrder byteOrder) { @@ -73,7 +73,7 @@ public SerializerBuilder withByteOrder(final ByteOrder byteOrder) return this; } - public SerializerBuilder withDelegate(final LongColumnSerializer delegate) + public SerializerBuilder withDelegate(final Serializer delegate) { this.delegate = delegate; return this; @@ -105,7 +105,11 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo ); builder.setType(ValueType.LONG) .setHasMultipleValues(false) - .setGenericColumn(new LongGenericColumnSupplier(column)); + .setGenericColumn(new LongGenericColumnSupplier( + column, + IndexIO.LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap() + )); } }; } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java new file mode 100644 index 000000000000..0d95a86ded72 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java @@ -0,0 +1,159 @@ +/* + * 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.collections.bitmap.ImmutableBitmap; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.BitmapSerde; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.CompressedColumnarLongsSupplier; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** + */ +public class LongGenericColumnPartSerdeV2 implements ColumnPartSerde +{ + @JsonCreator + public static LongGenericColumnPartSerdeV2 createDeserializer( + @JsonProperty("byteOrder") ByteOrder byteOrder, + @Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory + ) + { + return new LongGenericColumnPartSerdeV2( + byteOrder, + bitmapSerdeFactory != null ? bitmapSerdeFactory : new BitmapSerde.LegacyBitmapSerdeFactory(), + null + ); + } + + private final ByteOrder byteOrder; + private final BitmapSerdeFactory bitmapSerdeFactory; + private Serializer serializer; + + private LongGenericColumnPartSerdeV2( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, Serializer serializer + ) + { + this.byteOrder = byteOrder; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.serializer = serializer; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() + { + return bitmapSerdeFactory; + } + + public static SerializerBuilder serializerBuilder() + { + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private ByteOrder byteOrder = null; + private Serializer delegate = null; + private BitmapSerdeFactory bitmapSerdeFactory = null; + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withDelegate(final Serializer delegate) + { + this.delegate = delegate; + return this; + } + + public SerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) + { + this.bitmapSerdeFactory = bitmapSerdeFactory; + return this; + } + + public LongGenericColumnPartSerdeV2 build() + { + return new LongGenericColumnPartSerdeV2( + byteOrder, bitmapSerdeFactory, + new Serializer() + { + @Override + public long getSerializedSize() throws IOException + { + return delegate.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + delegate.writeTo(channel, smoosher); + } + } + ); + } + } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + public Deserializer getDeserializer() + { + return (buffer, builder, columnConfig) -> { + int offset = buffer.getInt(); + int initialPos = buffer.position(); + final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( + buffer, + byteOrder + ); + buffer.position(initialPos + offset); + final ImmutableBitmap bitmap; + if (buffer.hasRemaining()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + builder.setType(ValueType.LONG) + .setHasMultipleValues(false) + .setGenericColumn(new LongGenericColumnSupplier(column, bitmap)); + + }; + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java index c5861bb5b477..e71657b2a46a 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java @@ -20,6 +20,7 @@ package io.druid.segment.serde; import com.google.common.base.Supplier; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.LongsColumn; import io.druid.segment.data.CompressedColumnarLongsSupplier; @@ -29,15 +30,17 @@ public class LongGenericColumnSupplier implements Supplier { private final CompressedColumnarLongsSupplier column; + private final ImmutableBitmap nullValueBitmap; - public LongGenericColumnSupplier(CompressedColumnarLongsSupplier column) + public LongGenericColumnSupplier(CompressedColumnarLongsSupplier column, ImmutableBitmap nullValueBitmap) { this.column = column; + this.nullValueBitmap = nullValueBitmap; } @Override public GenericColumn get() { - return new LongsColumn(column.get()); + return new LongsColumn(column.get(), nullValueBitmap); } } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java index 7cbb277721a9..0fd23ebffcdc 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java @@ -75,4 +75,10 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) inspector.visit("expression", expression); inspector.visit("bindings", bindings); } + + @Override + public boolean isNull() + { + return getObject() == null; + } } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java index 2c2a3a80a93f..5f8652643aa5 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java @@ -21,10 +21,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Strings; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Maps; +import io.druid.common.config.NullHandling; import io.druid.math.expr.Expr; import io.druid.math.expr.ExprEval; import io.druid.math.expr.Parser; @@ -77,6 +77,12 @@ public double getDouble() return baseSelector.getDouble(); } + @Override + public boolean isNull() + { + return baseSelector.getObject().isNull(); + } + @Override public float getFloat() { @@ -198,7 +204,7 @@ class DefaultExpressionDimensionSelector extends BaseSingleValueDimensionSelecto @Override protected String getValue() { - return Strings.emptyToNull(baseSelector.getObject().asString()); + return NullHandling.emptyToNullIfNeeded(baseSelector.getObject().asString()); } @Override @@ -214,7 +220,7 @@ class ExtractionExpressionDimensionSelector extends BaseSingleValueDimensionSele @Override protected String getValue() { - return extractionFn.apply(Strings.emptyToNull(baseSelector.getObject().asString())); + return extractionFn.apply(NullHandling.emptyToNullIfNeeded(baseSelector.getObject().asString())); } @Override diff --git a/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java index 873b4a331470..f2051ae9f64b 100644 --- a/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java @@ -143,4 +143,10 @@ private ExprEval eval(final long value) bindings.set(value); return expression.eval(bindings); } + + @Override + public boolean isNull() + { + return getObject().isNull(); + } } diff --git a/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java index fba53f7d3d10..71557231cfe8 100644 --- a/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java @@ -133,6 +133,12 @@ private ExprEval eval() return expression.eval(bindings); } + @Override + public boolean isNull() + { + return eval().isNull(); + } + public static class LruEvalCache { private final Expr expression; diff --git a/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java b/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java index 967086f155f8..8721a5ad7780 100644 --- a/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/MetricManipulatorFnsTest.java @@ -50,6 +50,12 @@ public long getLong() { return longVal; } + + @Override + public boolean isNull() + { + return false; + } } ); LongMinAggregatorFactory longMinAggregatorFactory = new LongMinAggregatorFactory(NAME, FIELD); @@ -89,6 +95,12 @@ public long getLong() { return longVal; } + + @Override + public boolean isNull() + { + return false; + } } ); constructorArrays.add( diff --git a/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelectorImpl.java b/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelectorImpl.java index 9e7e1dd93215..4f2cac74f8cb 100644 --- a/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelectorImpl.java +++ b/processing/src/test/java/io/druid/query/aggregation/TestDoubleColumnSelectorImpl.java @@ -39,6 +39,12 @@ public double getDouble() return doubles[index]; } + @Override + public boolean isNull() + { + return false; + } + public void increment() { ++index; diff --git a/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java index 8c0aa532d8f5..dc1c68caeb03 100644 --- a/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java +++ b/processing/src/test/java/io/druid/query/aggregation/TestFloatColumnSelector.java @@ -38,6 +38,12 @@ public float getFloat() return floats[index]; } + @Override + public boolean isNull() + { + return false; + } + public void increment() { ++index; diff --git a/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java b/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java index 2a78bb500521..aca323a66c8b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java +++ b/processing/src/test/java/io/druid/query/aggregation/TestLongColumnSelector.java @@ -38,6 +38,12 @@ public long getLong() return longs[index]; } + @Override + public boolean isNull() + { + return false; + } + public void increment() { ++index; 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 8428981da063..eeb59e2035c6 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 @@ -85,6 +85,14 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { // don't inspect in tests } + + @Override + public boolean isNull() + { + return row.get().getMetric(columnName) == null; + } + + }; } diff --git a/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java index 278a1d759098..032ea51c4b38 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java @@ -19,12 +19,12 @@ package io.druid.segment; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMultiset; import com.google.common.collect.Sets; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.ISE; import io.druid.java.util.common.guava.Comparators; @@ -89,9 +89,15 @@ public void testStringColumnNullHandling() throws Exception nullFlavors.add(mMissing); nullFlavors.add(mEmptyList); nullFlavors.add(mNull); - nullFlavors.add(mEmptyString); nullFlavors.add(mListOfNull); - nullFlavors.add(mListOfEmptyString); + + if (NullHandling.useDefaultValuesForNull()) { + nullFlavors.add(mEmptyString); + nullFlavors.add(mListOfEmptyString); + } else { + nonNullFlavors.add(mEmptyString); + nonNullFlavors.add(mListOfEmptyString); + } Set> allValues = new HashSet<>(); allValues.addAll(nonNullFlavors); @@ -174,7 +180,7 @@ public void testStringColumnNullHandling() throws Exception final List expectedNullRows = new ArrayList<>(); for (int i = 0; i < index.getNumRows(); i++) { final List row = getRow(dictionaryColumn, i); - if (row.isEmpty() || row.stream().anyMatch(Strings::isNullOrEmpty)) { + if (row.isEmpty() || row.stream().anyMatch(NullHandling::isNullOrEquivalent)) { expectedNullRows.add(i); } } @@ -209,19 +215,16 @@ private static List normalize(final Object value, final boolean hasMulti final List retVal = new ArrayList<>(); if (value == null) { - if (!hasMultipleValues) { - // nulls become nulls in single valued columns, but are empty lists in multi valued columns - retVal.add(null); - } + retVal.add(null); } else if (value instanceof String) { - retVal.add(Strings.emptyToNull(((String) value))); + retVal.add(NullHandling.emptyToNullIfNeeded(((String) value))); } else if (value instanceof List) { final List list = (List) value; if (list.isEmpty() && !hasMultipleValues) { // empty lists become nulls in single valued columns - retVal.add(null); + retVal.add(NullHandling.emptyToNullIfNeeded(null)); } else { - retVal.addAll(list.stream().map(Strings::emptyToNull).collect(Collectors.toList())); + retVal.addAll(list.stream().map(NullHandling::emptyToNullIfNeeded).collect(Collectors.toList())); } } else { throw new ISE("didn't expect class[%s]", value.getClass()); diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 73feff0787b3..73759dbd0cb5 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.collections.StupidPool; +import io.druid.common.config.NullHandling; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; @@ -537,9 +538,13 @@ public void testCursoringAndSnapshot() throws Exception // no null id, so should get empty dims array Assert.assertEquals(0, rowD.size()); IndexedInts rowE = dimSelector3E.getRow(); - Assert.assertEquals(1, rowE.size()); - // the null id - Assert.assertEquals(0, rowE.get(0)); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(1, rowE.size()); + // the null id + Assert.assertEquals(0, rowE.get(0)); + } else { + Assert.assertEquals(0, rowE.size()); + } cursor.advance(); rowNumInCursor++; } diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index ca6ffe9fb66f..7dd331e08ecb 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import io.druid.collections.StupidPool; +import io.druid.common.config.NullHandling; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; diff --git a/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java index d132e1a6673f..bc31c22605ff 100644 --- a/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -22,6 +22,7 @@ import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.druid.common.config.NullHandling; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; @@ -113,7 +114,12 @@ public void testObjectSelector() Assert.assertEquals(null, selector.getObject()); CURRENT_ROW.set(ROW1); - Assert.assertEquals(4.0d, selector.getObject()); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(4.0d, selector.getObject()); + } else { + // y is null for row1 + Assert.assertEquals(null, selector.getObject()); + } CURRENT_ROW.set(ROW2); Assert.assertEquals(5.1d, selector.getObject()); @@ -131,7 +137,12 @@ public void testLongSelector() Assert.assertEquals(0L, selector.getLong()); CURRENT_ROW.set(ROW1); - Assert.assertEquals(4L, selector.getLong()); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(4L, selector.getLong()); + } else { + // y is null for row1 + Assert.assertTrue(selector.isNull()); + } CURRENT_ROW.set(ROW2); Assert.assertEquals(5L, selector.getLong()); @@ -149,7 +160,12 @@ public void testLongSelectorUsingStringFunction() Assert.assertEquals(0L, selector.getLong()); CURRENT_ROW.set(ROW1); - Assert.assertEquals(4L, selector.getLong()); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(4L, selector.getLong()); + } else { + // y is null for row1 + Assert.assertTrue(selector.isNull()); + } CURRENT_ROW.set(ROW2); Assert.assertEquals(0L, selector.getLong()); @@ -167,7 +183,12 @@ public void testFloatSelector() Assert.assertEquals(0.0f, selector.getFloat(), 0.0f); CURRENT_ROW.set(ROW1); - Assert.assertEquals(4.0f, selector.getFloat(), 0.0f); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(4.0f, selector.getFloat(), 0.0f); + } else { + // y is null for row1 + Assert.assertTrue(selector.isNull()); + } CURRENT_ROW.set(ROW2); Assert.assertEquals(5.1f, selector.getFloat(), 0.0f); @@ -195,10 +216,18 @@ public void testDimensionSelector() Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW1); - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); - Assert.assertEquals("4.0", selector.lookupName(selector.getRow().get(0))); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(false, nullMatcher.matches()); + Assert.assertEquals(false, fiveMatcher.matches()); + Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals("4.0", selector.lookupName(selector.getRow().get(0))); + } else { + // y is null in row1 + Assert.assertEquals(true, nullMatcher.matches()); + Assert.assertEquals(false, fiveMatcher.matches()); + Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); + } CURRENT_ROW.set(ROW2); Assert.assertEquals(false, nullMatcher.matches()); @@ -229,7 +258,10 @@ public void testDimensionSelectorUsingStringFunction() CURRENT_ROW.set(ROW1); Assert.assertEquals(1, selector.getRow().size()); - Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0))); + Assert.assertEquals( + NullHandling.useDefaultValuesForNull() ? "4" : null, + selector.lookupName(selector.getRow().get(0)) + ); CURRENT_ROW.set(ROW2); Assert.assertEquals(1, selector.getRow().size()); @@ -259,10 +291,18 @@ public void testDimensionSelectorWithExtraction() Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW1); - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); - Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0))); + if (NullHandling.useDefaultValuesForNull()) { + Assert.assertEquals(false, nullMatcher.matches()); + Assert.assertEquals(false, fiveMatcher.matches()); + Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0))); + } else { + // y is null in row1 + Assert.assertEquals(true, nullMatcher.matches()); + Assert.assertEquals(false, fiveMatcher.matches()); + Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); + } CURRENT_ROW.set(ROW2); Assert.assertEquals(false, nullMatcher.matches()); diff --git a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java index 943b50258ca6..53e4464fe137 100644 --- a/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/VirtualColumnsTest.java @@ -379,6 +379,12 @@ public Class classOfObject() { return String.class; } + + @Override + public boolean isNull() + { + return selector.isNull(); + } }; return dimensionSpec.decorate(dimensionSelector); @@ -397,6 +403,12 @@ public long getLong() { return theLong; } + + @Override + public boolean isNull() + { + return false; + } }; } diff --git a/services/src/main/java/io/druid/cli/DumpSegment.java b/services/src/main/java/io/druid/cli/DumpSegment.java index 794e9181cbae..3d82fbf8d3ce 100644 --- a/services/src/main/java/io/druid/cli/DumpSegment.java +++ b/services/src/main/java/io/druid/cli/DumpSegment.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Strings; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; @@ -41,6 +40,7 @@ import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.RoaringBitmapFactory; +import io.druid.common.config.NullHandling; import io.druid.guice.DruidProcessingModule; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; @@ -364,18 +364,20 @@ public Object apply(final OutputStream out) jg.writeFieldName(columnName); jg.writeStartObject(); for (int i = 0; i < bitmapIndex.getCardinality(); i++) { - jg.writeFieldName(Strings.nullToEmpty(bitmapIndex.getValue(i))); - final ImmutableBitmap bitmap = bitmapIndex.getBitmap(i); - if (decompressBitmaps) { - jg.writeStartArray(); - final IntIterator iterator = bitmap.iterator(); - while (iterator.hasNext()) { - final int rowNum = iterator.next(); - jg.writeNumber(rowNum); + String val = NullHandling.nullToEmptyIfNeeded(bitmapIndex.getValue(i)); + if (val != null) { + final ImmutableBitmap bitmap = bitmapIndex.getBitmap(i); + if (decompressBitmaps) { + jg.writeStartArray(); + final IntIterator iterator = bitmap.iterator(); + while (iterator.hasNext()) { + final int rowNum = iterator.next(); + jg.writeNumber(rowNum); + } + jg.writeEndArray(); + } else { + jg.writeBinary(bitmapSerdeFactory.getObjectStrategy().toBytes(bitmap)); } - jg.writeEndArray(); - } else { - jg.writeBinary(bitmapSerdeFactory.getObjectStrategy().toBytes(bitmap)); } } jg.writeEndObject(); @@ -552,5 +554,11 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("delegate", delegate); } + + @Override + public boolean isNull() + { + return delegate.isNull(); + } } } From 2c78fa58bb1019215d90fe006820ee7c8f6a72be Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 23 Jan 2018 17:52:35 +0530 Subject: [PATCH 02/19] fix teamcity error - remove unused method --- .../java/io/druid/segment/DimensionHandlerUtils.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java index f52e5915d439..63ca29678e38 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java @@ -238,15 +238,6 @@ private static Colu return strategyFactory.makeColumnSelectorStrategy(capabilities, selector); } - @Nullable - public static String convertObjectToString(@Nullable Object valObj) - { - if (valObj == null) { - return null; - } - return valObj.toString(); - } - @Nullable public static Long convertObjectToLong(@Nullable Object valObj) { From 6e956a6950678ffd42e7418445dd41946bf145fb Mon Sep 17 00:00:00 2001 From: Nishant Date: Mon, 29 Jan 2018 10:54:11 +0530 Subject: [PATCH 03/19] review comments --- .../io/druid/common/config/NullHandling.java | 19 +++++++++------ .../main/java/io/druid/math/expr/Expr.java | 2 +- .../java/io/druid/math/expr/Function.java | 2 +- .../java/io/druid/math/expr/EvalTest.java | 2 +- .../java/io/druid/math/expr/FunctionTest.java | 2 +- .../query/aggregation/BufferAggregator.java | 3 ++- .../aggregation/ObjectAggregateCombiner.java | 2 +- .../RowBasedColumnSelectorFactory.java | 24 ++++++++++++++----- .../BaseNullableColumnValueSelector.java | 10 ++++++++ .../ColumnSelectorBitmapIndexSelector.java | 9 ++++--- .../segment/ConstantColumnValueSelector.java | 2 +- .../druid/segment/DimensionHandlerUtils.java | 18 ++++++++------ .../io/druid/segment/DimensionSelector.java | 6 +---- .../druid/segment/DoubleDimensionIndexer.java | 4 ++-- .../druid/segment/FloatDimensionIndexer.java | 2 +- .../java/io/druid/segment/IndexMergerV9.java | 12 +++++----- .../druid/segment/LongDimensionIndexer.java | 2 +- .../segment/incremental/IncrementalIndex.java | 6 ++--- .../segment/IndexMergerNullHandlingTest.java | 2 +- .../IncrementalIndexStorageAdapterTest.java | 2 +- .../virtual/ExpressionVirtualColumnTest.java | 14 +++++------ 21 files changed, 88 insertions(+), 57 deletions(-) diff --git a/common/src/main/java/io/druid/common/config/NullHandling.java b/common/src/main/java/io/druid/common/config/NullHandling.java index 75df6e980e73..67061f497521 100644 --- a/common/src/main/java/io/druid/common/config/NullHandling.java +++ b/common/src/main/java/io/druid/common/config/NullHandling.java @@ -53,16 +53,21 @@ public class NullHandling Boolean.valueOf(System.getProperty(NULL_HANDLING_CONFIG_STRING, "true")) ); - public static boolean useDefaultValuesForNull() + public static boolean replaceWithDefault() { return INSTANCE.isUseDefaultValuesForNull(); } + public static boolean sqlCompatible() + { + return !replaceWithDefault(); + } + @Nullable public static String nullToEmptyIfNeeded(@Nullable String value) { //CHECKSTYLE.OFF: Regexp - return useDefaultValuesForNull() ? Strings.nullToEmpty(value) : value; + return replaceWithDefault() ? Strings.nullToEmpty(value) : value; //CHECKSTYLE.ON: Regexp } @@ -70,32 +75,32 @@ public static String nullToEmptyIfNeeded(@Nullable String value) public static String emptyToNullIfNeeded(@Nullable String value) { //CHECKSTYLE.OFF: Regexp - return useDefaultValuesForNull() ? Strings.emptyToNull(value) : value; + return replaceWithDefault() ? Strings.emptyToNull(value) : value; //CHECKSTYLE.ON: Regexp } @Nullable public static String defaultStringValue() { - return useDefaultValuesForNull() ? "" : null; + return replaceWithDefault() ? "" : null; } @Nullable public static Long defaultLongValue() { - return useDefaultValuesForNull() ? ZERO_LONG : null; + return replaceWithDefault() ? ZERO_LONG : null; } @Nullable public static Float defaultFloatValue() { - return useDefaultValuesForNull() ? ZERO_FLOAT : null; + return replaceWithDefault() ? ZERO_FLOAT : null; } @Nullable public static Double defaultDoubleValue() { - return useDefaultValuesForNull() ? ZERO_DOUBLE : null; + return replaceWithDefault() ? ZERO_DOUBLE : null; } public static boolean isNullOrEquivalent(@Nullable String value) diff --git a/common/src/main/java/io/druid/math/expr/Expr.java b/common/src/main/java/io/druid/math/expr/Expr.java index 0ab1f9f82bf7..0b1deb188740 100644 --- a/common/src/main/java/io/druid/math/expr/Expr.java +++ b/common/src/main/java/io/druid/math/expr/Expr.java @@ -365,7 +365,7 @@ public ExprEval eval(ObjectBinding bindings) // Result of any Binary expressions is null if any of the argument is null. // e.g "select null * 2 as c;" or "select null + 1 as c;" will return null as per Standard SQL spec. - if (!NullHandling.useDefaultValuesForNull() && (leftVal.isNull() || rightVal.isNull())) { + if (NullHandling.sqlCompatible() && (leftVal.isNull() || rightVal.isNull())) { return ExprEval.of(null); } diff --git a/common/src/main/java/io/druid/math/expr/Function.java b/common/src/main/java/io/druid/math/expr/Function.java index 9a589ebc9fd3..42b7902ba4bd 100644 --- a/common/src/main/java/io/druid/math/expr/Function.java +++ b/common/src/main/java/io/druid/math/expr/Function.java @@ -74,7 +74,7 @@ abstract class SingleParamMath extends SingleParam @Override protected final ExprEval eval(ExprEval param) { - if (!NullHandling.useDefaultValuesForNull() && param.isNull()) { + if (NullHandling.sqlCompatible() && param.isNull()) { return ExprEval.of(null); } if (param.type() == ExprType.LONG) { diff --git a/common/src/test/java/io/druid/math/expr/EvalTest.java b/common/src/test/java/io/druid/math/expr/EvalTest.java index 68ef37f309cf..800bc5740c28 100644 --- a/common/src/test/java/io/druid/math/expr/EvalTest.java +++ b/common/src/test/java/io/druid/math/expr/EvalTest.java @@ -140,7 +140,7 @@ public void testLongEval() Assert.assertEquals(1271055781L, evalLong("unix_timestamp('2010-04-12T07:03:01')", bindings)); Assert.assertEquals(1271023381L, evalLong("unix_timestamp('2010-04-12T07:03:01+09:00')", bindings)); Assert.assertEquals(1271023381L, evalLong("unix_timestamp('2010-04-12T07:03:01.419+09:00')", bindings)); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals("NULL", eval("nvl(if(x == 9223372036854775807, '', 'x'), 'NULL')", bindings).asString()); } else { Assert.assertEquals("", eval("nvl(if(x == 9223372036854775807, '', 'x'), 'NULL')", bindings).asString()); diff --git a/common/src/test/java/io/druid/math/expr/FunctionTest.java b/common/src/test/java/io/druid/math/expr/FunctionTest.java index 99c153da550c..9ea6efb2c6cb 100644 --- a/common/src/test/java/io/druid/math/expr/FunctionTest.java +++ b/common/src/test/java/io/druid/math/expr/FunctionTest.java @@ -56,7 +56,7 @@ public void testCaseSearched() public void testConcat() { assertExpr("concat(x,' ',y)", "foo 2"); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { assertExpr("concat(x,' ',nonexistent,' ',y)", "foo 2"); } else { assertExpr("concat(x,' ',nonexistent,' ',y)", null); 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 b32ea821fa26..3f62d836df01 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -190,7 +190,8 @@ default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, By * @param position offset within the byte buffer at which the aggregate value is stored * * @return true if the aggrgeated value is null otherwise false. - * For backwards compatibility, isNull() may return false even if {@link BufferAggregator#get(ByteBuffer, int)} returns null. Users of this method should account for this case. + * The default implementation always return false to enable smooth backward compatibility, + * re-implement if your aggregator is nullable. */ default boolean isNull(ByteBuffer buf, int position) { diff --git a/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java index d486ecad82bb..bc9a594ee8d0 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java @@ -27,7 +27,7 @@ public abstract class ObjectAggregateCombiner implements AggregateCombiner, ObjectColumnSelector { @Override - public boolean isNull() + public final boolean isNull() { return false; } diff --git a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java index 856491080071..15e78e6d3be8 100644 --- a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java @@ -416,27 +416,39 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) return new ColumnValueSelector() { @Override - public double getDouble() + public boolean isNull() { - return DimensionHandlerUtils.nullToZeroDouble(row.get().getMetric(columnName)).doubleValue(); + return row.get().getRaw(columnName) == null; } @Override - public boolean isNull() + public double getDouble() { - return row.get().getRaw(columnName) == null; + Number metric = row.get().getMetric(columnName); + if (NullHandling.sqlCompatible() && metric == null) { + throw new IllegalStateException("Cannot return double for Null Value"); + } + return DimensionHandlerUtils.nullToZero(metric).doubleValue(); } @Override public float getFloat() { - return DimensionHandlerUtils.nullToZeroDouble(row.get().getMetric(columnName)).floatValue(); + Number metric = row.get().getMetric(columnName); + if (NullHandling.sqlCompatible() && metric == null) { + throw new IllegalStateException("Cannot return float for Null Value"); + } + return DimensionHandlerUtils.nullToZero(metric).floatValue(); } @Override public long getLong() { - return DimensionHandlerUtils.nullToZeroDouble(row.get().getMetric(columnName)).longValue(); + Number metric = row.get().getMetric(columnName); + if (NullHandling.sqlCompatible() && metric == null) { + throw new IllegalStateException("Cannot return long for Null Value"); + } + return DimensionHandlerUtils.nullToZero(metric).longValue(); } @Nullable diff --git a/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java b/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java index 3d2c6a5b990f..89b47efd06f2 100644 --- a/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java @@ -21,8 +21,18 @@ import io.druid.guice.annotations.PublicApi; +/** + * Null value checking polymorphic "part" of the {@link ColumnValueSelector} interface for primitive values. + * Users of {@link BaseLongColumnValueSelector#getLong()}, {@link BaseDoubleColumnValueSelector#getDouble()} + * and {@link BaseFloatColumnValueSelector#getFloat()} are required to check the nullability of the primitive + * types returned. + */ @PublicApi public interface BaseNullableColumnValueSelector { + /** + * returns true if selected primitive value is null for {@link BaseFloatColumnValueSelector}, + * {@link BaseLongColumnValueSelector} and {@link BaseDoubleColumnValueSelector} otherwise false. + */ boolean isNull(); } diff --git a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java index 7026b0cd7042..bee78e103048 100644 --- a/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/io/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -174,12 +174,15 @@ public BitmapFactory getBitmapFactory() return bitmapFactory; } + /** + * Return -2 for non-null values to match what the {@link BitmapIndex} implementation in + * {@link io.druid.segment.serde.BitmapIndexColumnPartSupplier} + * would return for {@link BitmapIndex#getIndex(String)} when there is only a single index, for the null value. + * i.e., return an 'insertion point' of 1 for non-null values (see {@link BitmapIndex} interface) + */ @Override public int getIndex(String value) { - // Return -2 for non-null values to match what the BitmapIndex implementation in BitmapIndexColumnPartSupplier - // would return for getIndex() when there is only a single index, for the null value. - // i.e., return an 'insertion point' of 1 for non-null values (see BitmapIndex interface) return NullHandling.isNullOrEquivalent(value) ? 0 : -2; } diff --git a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java index 087ce74de2ae..b28755cedbe0 100644 --- a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java @@ -90,6 +90,6 @@ public void inspectRuntimeShape(final RuntimeShapeInspector inspector) @Override public boolean isNull() { - return objectValue == null; + return false; } } diff --git a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java index 63ca29678e38..10d556bbd263 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandlerUtils.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Doubles; import com.google.common.primitives.Floats; -import io.druid.common.config.NullHandling; import io.druid.common.guava.GuavaUtils; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; import io.druid.java.util.common.IAE; @@ -358,18 +357,23 @@ public static Long getExactLongFromDecimalString(String decimalStr) } } - public static Number nullToZeroDouble(@Nullable Number number) + public static Double nullToZero(@Nullable Double number) { - return number == null ? NullHandling.ZERO_DOUBLE : number; + return number == null ? ZERO_DOUBLE : number; } - public static Number nullToZeroLong(@Nullable Number number) + public static Long nullToZero(@Nullable Long number) { - return number == null ? NullHandling.ZERO_LONG : number; + return number == null ? ZERO_LONG : number; } - public static Number nullToZeroFloat(@Nullable Number number) + public static Float nullToZero(@Nullable Float number) { - return number == null ? NullHandling.ZERO_FLOAT : number; + return number == null ? ZERO_FLOAT : number; + } + + public static Number nullToZero(@Nullable Number number) + { + return number == null ? ZERO_DOUBLE : number; } } diff --git a/processing/src/main/java/io/druid/segment/DimensionSelector.java b/processing/src/main/java/io/druid/segment/DimensionSelector.java index 9aee99edf220..095f2dabc7dc 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelector.java @@ -149,18 +149,14 @@ default long getLong() return 0L; } - /** - * @deprecated always throws {@link UnsupportedOperationException} - */ @Deprecated @Override default boolean isNull() { - throw new UnsupportedOperationException("DimensionSelector cannot be operated as numeric ColumnValueSelector" + this.getClass()); + return false; } /** - * @deprecated always throws {@link UnsupportedOperationException} * Converts the current result of {@link #getRow()} into null, if the row is empty, a String, if the row has size 1, * or a String[] array, if the row has size > 1, using {@link #lookupName(int)}. * diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index 609a40d23eb3..88c1be58cb61 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -102,7 +102,7 @@ class IndexerDoubleColumnSelector implements DoubleColumnSelector @Override public boolean isNull() { - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return false; } final Object[] dims = currEntry.get().getDims(); @@ -158,7 +158,7 @@ public boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable Double lhs, @Nul @Override public int getUnsortedEncodedKeyComponentHashCode(@Nullable Double key) { - return DimensionHandlerUtils.nullToZeroDouble(key).hashCode(); + return DimensionHandlerUtils.nullToZero(key).hashCode(); } @Override diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 1a05d8bdc06a..57670e250186 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -157,7 +157,7 @@ public boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable Float lhs, @Null @Override public int getUnsortedEncodedKeyComponentHashCode(@Nullable Float key) { - return DimensionHandlerUtils.nullToZeroFloat(key).hashCode(); + return DimensionHandlerUtils.nullToZero(key).hashCode(); } @Override diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index f2918efed660..5266617a9184 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -388,7 +388,7 @@ private void makeMetricsColumns( static ColumnPartSerde createLongColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec) { // If using default values for null use LongGenericColumnPartSerde to allow rollback to previous versions. - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return LongGenericColumnPartSerde.serializerBuilder() .withByteOrder(IndexIO.BYTE_ORDER) .withDelegate(serializer) @@ -405,7 +405,7 @@ static ColumnPartSerde createLongColumnPartSerde(GenericColumnSerializer seriali static ColumnPartSerde createDoubleColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec) { // If using default values for null use DoubleGenericColumnPartSerde to allow rollback to previous versions. - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return DoubleGenericColumnPartSerde.serializerBuilder() .withByteOrder(IndexIO.BYTE_ORDER) .withDelegate(serializer) @@ -422,7 +422,7 @@ static ColumnPartSerde createDoubleColumnPartSerde(GenericColumnSerializer seria static ColumnPartSerde createFloatColumnPartSerde(GenericColumnSerializer serializer, IndexSpec indexSpec) { // If using default values for null use FloatGenericColumnPartSerde to allow rollback to previous versions. - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return FloatGenericColumnPartSerde.serializerBuilder() .withByteOrder(IndexIO.BYTE_ORDER) .withDelegate(serializer) @@ -607,7 +607,7 @@ static GenericColumnSerializer createLongColumnSerializer( ) { // If using default values for null use LongColumnSerializer to allow rollback to previous versions. - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return LongColumnSerializer.create( segmentWriteOutMedium, columnName, @@ -633,7 +633,7 @@ static GenericColumnSerializer createDoubleColumnSerializer( ) { // If using default values for null use DoubleColumnSerializer to allow rollback to previous versions. - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return DoubleColumnSerializer.create( segmentWriteOutMedium, columnName, @@ -657,7 +657,7 @@ static GenericColumnSerializer createFloatColumnSerializer( ) { // If using default values for null use FloatColumnSerializer to allow rollback to previous versions. - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { return FloatColumnSerializer.create( segmentWriteOutMedium, columnName, diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index ac8cf1a156c9..962df7cdd618 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -156,7 +156,7 @@ public boolean checkUnsortedEncodedKeyComponentsEqual(@Nullable Long lhs, @Nulla @Override public int getUnsortedEncodedKeyComponentHashCode(@Nullable Long key) { - return DimensionHandlerUtils.nullToZeroLong(key).hashCode(); + return DimensionHandlerUtils.nullToZero(key).hashCode(); } @Override 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 8655b9f7d559..d56ba07b4827 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -160,19 +160,19 @@ public boolean isNull() @Override public long getLong() { - return DimensionHandlerUtils.nullToZeroLong(in.get().getMetric(column)).longValue(); + return DimensionHandlerUtils.nullToZero(in.get().getMetric(column)).longValue(); } @Override public float getFloat() { - return DimensionHandlerUtils.nullToZeroFloat(in.get().getMetric(column)).floatValue(); + return DimensionHandlerUtils.nullToZero(in.get().getMetric(column)).floatValue(); } @Override public double getDouble() { - return DimensionHandlerUtils.nullToZeroDouble(in.get().getMetric(column)).doubleValue(); + return DimensionHandlerUtils.nullToZero(in.get().getMetric(column)).doubleValue(); } @Override diff --git a/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java index 032ea51c4b38..a7d5ec9f957c 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java @@ -91,7 +91,7 @@ public void testStringColumnNullHandling() throws Exception nullFlavors.add(mNull); nullFlavors.add(mListOfNull); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { nullFlavors.add(mEmptyString); nullFlavors.add(mListOfEmptyString); } else { diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 73759dbd0cb5..2ea4882e2016 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -538,7 +538,7 @@ public void testCursoringAndSnapshot() throws Exception // no null id, so should get empty dims array Assert.assertEquals(0, rowD.size()); IndexedInts rowE = dimSelector3E.getRow(); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(1, rowE.size()); // the null id Assert.assertEquals(0, rowE.get(0)); diff --git a/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java index bc31c22605ff..55ce4d17be33 100644 --- a/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/io/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -114,7 +114,7 @@ public void testObjectSelector() Assert.assertEquals(null, selector.getObject()); CURRENT_ROW.set(ROW1); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(4.0d, selector.getObject()); } else { // y is null for row1 @@ -137,7 +137,7 @@ public void testLongSelector() Assert.assertEquals(0L, selector.getLong()); CURRENT_ROW.set(ROW1); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(4L, selector.getLong()); } else { // y is null for row1 @@ -160,7 +160,7 @@ public void testLongSelectorUsingStringFunction() Assert.assertEquals(0L, selector.getLong()); CURRENT_ROW.set(ROW1); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(4L, selector.getLong()); } else { // y is null for row1 @@ -183,7 +183,7 @@ public void testFloatSelector() Assert.assertEquals(0.0f, selector.getFloat(), 0.0f); CURRENT_ROW.set(ROW1); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(4.0f, selector.getFloat(), 0.0f); } else { // y is null for row1 @@ -216,7 +216,7 @@ public void testDimensionSelector() Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW1); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(false, nullMatcher.matches()); Assert.assertEquals(false, fiveMatcher.matches()); Assert.assertEquals(true, nonNullMatcher.matches()); @@ -259,7 +259,7 @@ public void testDimensionSelectorUsingStringFunction() CURRENT_ROW.set(ROW1); Assert.assertEquals(1, selector.getRow().size()); Assert.assertEquals( - NullHandling.useDefaultValuesForNull() ? "4" : null, + NullHandling.replaceWithDefault() ? "4" : null, selector.lookupName(selector.getRow().get(0)) ); @@ -291,7 +291,7 @@ public void testDimensionSelectorWithExtraction() Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW1); - if (NullHandling.useDefaultValuesForNull()) { + if (NullHandling.replaceWithDefault()) { Assert.assertEquals(false, nullMatcher.matches()); Assert.assertEquals(false, fiveMatcher.matches()); Assert.assertEquals(true, nonNullMatcher.matches()); From 961d69467deb9e0b30fd14f0173ba11f29945528 Mon Sep 17 00:00:00 2001 From: Nishant Date: Thu, 1 Feb 2018 00:10:37 +0530 Subject: [PATCH 04/19] review comments --- .../io/druid/common/config/NullHandling.java | 2 +- .../druid/common/utils/SerializerUtils.java | 2 +- .../query/aggregation/BufferAggregator.java | 5 ++-- .../io/druid/segment/DimensionSelector.java | 1 - .../druid/segment/DoubleColumnSelector.java | 3 +++ .../segment/DoubleColumnSerializerV2.java | 11 +++++--- .../druid/segment/DoubleDimensionIndexer.java | 5 +++- .../io/druid/segment/FloatColumnSelector.java | 6 +++++ .../segment/FloatColumnSerializerV2.java | 21 ++++++++++++---- .../druid/segment/FloatDimensionIndexer.java | 25 ++++++++++++------- .../druid/segment/FloatDimensionMergerV9.java | 1 + .../io/druid/segment/LongColumnSelector.java | 6 +++++ .../druid/segment/LongColumnSerializerV2.java | 10 +++++--- .../druid/segment/LongDimensionIndexer.java | 22 ++++++++++------ .../druid/segment/LongDimensionMergerV9.java | 2 +- 15 files changed, 86 insertions(+), 36 deletions(-) diff --git a/common/src/main/java/io/druid/common/config/NullHandling.java b/common/src/main/java/io/druid/common/config/NullHandling.java index 67061f497521..538d6de94640 100644 --- a/common/src/main/java/io/druid/common/config/NullHandling.java +++ b/common/src/main/java/io/druid/common/config/NullHandling.java @@ -105,7 +105,7 @@ public static Double defaultDoubleValue() public static boolean isNullOrEquivalent(@Nullable String value) { - return INSTANCE.isUseDefaultValuesForNull() ? Strings.isNullOrEmpty(value) : value == null; + return replaceWithDefault() ? Strings.isNullOrEmpty(value) : value == null; } } diff --git a/common/src/main/java/io/druid/common/utils/SerializerUtils.java b/common/src/main/java/io/druid/common/utils/SerializerUtils.java index e274b1f0d8dd..45abf9814f69 100644 --- a/common/src/main/java/io/druid/common/utils/SerializerUtils.java +++ b/common/src/main/java/io/druid/common/utils/SerializerUtils.java @@ -142,7 +142,7 @@ private void writeInt(OutputStream out, int intValue) throws IOException out.write(Ints.toByteArray(intValue)); } - private void writeInt(WritableByteChannel out, int intValue) throws IOException + public static void writeInt(WritableByteChannel out, int intValue) throws IOException { final ByteBuffer buffer = ByteBuffer.allocate(Integer.BYTES); buffer.putInt(intValue); 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 3f62d836df01..2345a4dbf2eb 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -189,10 +189,9 @@ default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, By * @param buf byte buffer storing the byte array representation of the aggregate * @param position offset within the byte buffer at which the aggregate value is stored * - * @return true if the aggrgeated value is null otherwise false. - * The default implementation always return false to enable smooth backward compatibility, - * re-implement if your aggregator is nullable. + * @return true if the aggregated value is primitive long/double/float and aggregated value is null otherwise false. */ + @CalledFromHotLoop default boolean isNull(ByteBuffer buf, int position) { return false; diff --git a/processing/src/main/java/io/druid/segment/DimensionSelector.java b/processing/src/main/java/io/druid/segment/DimensionSelector.java index 095f2dabc7dc..2623dcc73290 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelector.java @@ -149,7 +149,6 @@ default long getLong() return 0L; } - @Deprecated @Override default boolean isNull() { diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java index 0765274c70b0..3ecbb90908a4 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java @@ -19,6 +19,8 @@ package io.druid.segment; +import javax.annotation.Nullable; + /** * This interface is convenient for implementation of "double-sourcing" {@link ColumnValueSelector}s, it provides * default implementations for all {@link ColumnValueSelector}'s methods except {@link #getDouble()}. @@ -59,6 +61,7 @@ default long getLong() */ @Deprecated @Override + @Nullable default Double getObject() { if (isNull()) { diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java index d5392331335a..b0a95edcfba1 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java @@ -19,9 +19,9 @@ package io.druid.segment; -import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.data.BitmapSerdeFactory; @@ -33,10 +33,15 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; +/** + * Column Serializer for double column. + * The column is serialized in two parts, first a bitmap indicating the nullability of row values + * and second the actual row values. + * This class is Unsafe for concurrent use from multiple threads. + */ public class DoubleColumnSerializerV2 implements GenericColumnSerializer { public static DoubleColumnSerializerV2 create( @@ -124,7 +129,7 @@ public long getSerializedSize() throws IOException @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - channel.write(ByteBuffer.wrap(Ints.toByteArray((int) writer.getSerializedSize()))); + SerializerUtils.writeInt(channel, (int) writer.getSerializedSize()); writer.writeTo(channel, smoosher); if (!nullRowsBitmap.isEmpty()) { nullValueBitmapWriter.writeTo(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index 88c1be58cb61..5b69f8f0415b 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -36,7 +36,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer { - public static final Comparator DOUBLE_COMPARATOR = Comparators.naturalNullsFirst(); + public static final Comparator DOUBLE_COMPARATOR = Comparators.naturalNullsFirst(); @Override public Double processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions) @@ -115,6 +115,9 @@ public double getDouble() final Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (NullHandling.sqlCompatible()) { + throw new IllegalStateException("Cannot return double for Null Value"); + } return 0.0; } return (Double) dims[dimIndex]; diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java index 0e890138936b..2e0a394ffc27 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java @@ -19,6 +19,8 @@ package io.druid.segment; +import javax.annotation.Nullable; + /** * This interface is convenient for implementation of "float-sourcing" {@link ColumnValueSelector}s, it provides default * implementations for all {@link ColumnValueSelector}'s methods except {@link #getFloat()}. @@ -59,8 +61,12 @@ default long getLong() */ @Deprecated @Override + @Nullable default Float getObject() { + if (isNull()) { + return null; + } return getFloat(); } diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java index 530f652fe065..9b872a89f9c4 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java @@ -19,9 +19,9 @@ package io.druid.segment; -import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.data.BitmapSerdeFactory; @@ -33,10 +33,15 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; +/** + * Column Serializer for float column. + * The column is serialized in two parts, first a bitmap indicating the nullability of row values + * and second the actual row values. + * This class is Unsafe for concurrent use from multiple threads. + */ public class FloatColumnSerializerV2 implements GenericColumnSerializer { public static FloatColumnSerializerV2 create( @@ -46,7 +51,13 @@ public static FloatColumnSerializerV2 create( BitmapSerdeFactory bitmapSerdeFactory ) { - return new FloatColumnSerializerV2(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression, bitmapSerdeFactory); + return new FloatColumnSerializerV2( + segmentWriteOutMedium, + filenameBase, + IndexIO.BYTE_ORDER, + compression, + bitmapSerdeFactory + ); } private final SegmentWriteOutMedium segmentWriteOutMedium; @@ -98,7 +109,7 @@ public void serialize(@Nullable Object obj) throws IOException { if (obj == null) { nullRowsBitmap.add(rowCount); - writer.add(0L); + writer.add(0f); } else { writer.add(((Number) obj).floatValue()); } @@ -119,7 +130,7 @@ public long getSerializedSize() throws IOException @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - channel.write(ByteBuffer.wrap(Ints.toByteArray((int) writer.getSerializedSize()))); + SerializerUtils.writeInt(channel, (int) writer.getSerializedSize()); writer.writeTo(channel, smoosher); if (!nullRowsBitmap.isEmpty()) { nullValueBitmapWriter.writeTo(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 57670e250186..02c7f4c87944 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -21,6 +21,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.guava.Comparators; import io.druid.query.dimension.DimensionSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -35,8 +36,7 @@ public class FloatDimensionIndexer implements DimensionIndexer { - - public static final Comparator FLOAT_COMPARATOR = Comparators.naturalNullsFirst(); + public static final Comparator FLOAT_COMPARATOR = Comparators.naturalNullsFirst(); @Override public Float processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boolean reportParseExceptions) @@ -99,25 +99,32 @@ public ColumnValueSelector makeColumnValueSelector( final int dimIndex = desc.getIndex(); class IndexerFloatColumnSelector implements FloatColumnSelector { + + @Override + public boolean isNull() + { + if (NullHandling.replaceWithDefault()) { + return false; + } + final Object[] dims = currEntry.get().getDims(); + return dimIndex >= dims.length || dims[dimIndex] == null; + } + @Override public float getFloat() { final Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (NullHandling.sqlCompatible()) { + throw new IllegalStateException("Cannot return float for Null Value"); + } return 0.0f; } return (Float) dims[dimIndex]; } - @Override - public boolean isNull() - { - final Object[] dims = currEntry.get().getDims(); - return dimIndex >= dims.length || dims[dimIndex] == null; - } - @SuppressWarnings("deprecation") @Nullable @Override diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index fda2fd4030cf..497a3bf5e985 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -55,6 +55,7 @@ public FloatDimensionMergerV9( private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); + // If using default values for null use FloatColumnSerializer to allow rollback to previous versions. this.serializer = IndexMergerV9.createFloatColumnSerializer( segmentWriteOutMedium, dimensionName, diff --git a/processing/src/main/java/io/druid/segment/LongColumnSelector.java b/processing/src/main/java/io/druid/segment/LongColumnSelector.java index 9f849fe9063e..8a46b7c556dd 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSelector.java @@ -19,6 +19,8 @@ package io.druid.segment; +import javax.annotation.Nullable; + /** * This interface is convenient for implementation of "long-sourcing" {@link ColumnValueSelector}s, it provides default * implementations for all {@link ColumnValueSelector}'s methods except {@link #getLong()}. @@ -59,8 +61,12 @@ default double getDouble() */ @Deprecated @Override + @Nullable default Long getObject() { + if (isNull()) { + return null; + } return getLong(); } diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java index 4c979fd710d3..37004a7e2cdf 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java @@ -19,9 +19,9 @@ package io.druid.segment; -import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.data.BitmapSerdeFactory; @@ -33,12 +33,14 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; /** - * Unsafe for concurrent use from multiple threads. + * Column Serializer for long column. + * The column is serialized in two parts, first a bitmap indicating the nullability of row values + * and second the actual row values. + * This class is Unsafe for concurrent use from multiple threads. */ public class LongColumnSerializerV2 implements GenericColumnSerializer { @@ -133,7 +135,7 @@ public long getSerializedSize() throws IOException @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - channel.write(ByteBuffer.wrap(Ints.toByteArray((int) writer.getSerializedSize()))); + SerializerUtils.writeInt(channel, (int) writer.getSerializedSize()); writer.writeTo(channel, smoosher); if (!nullRowsBitmap.isEmpty()) { nullValueBitmapWriter.writeTo(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 962df7cdd618..412e868db1ed 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -21,6 +21,7 @@ import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.MutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.guava.Comparators; import io.druid.query.dimension.DimensionSpec; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -98,25 +99,32 @@ public ColumnValueSelector makeColumnValueSelector( final int dimIndex = desc.getIndex(); class IndexerLongColumnSelector implements LongColumnSelector { + + @Override + public boolean isNull() + { + if (NullHandling.replaceWithDefault()) { + return false; + } + final Object[] dims = currEntry.get().getDims(); + return dimIndex >= dims.length || dims[dimIndex] == null; + } + @Override public long getLong() { final Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (NullHandling.sqlCompatible()) { + throw new IllegalStateException("Cannot return long for Null Value"); + } return 0; } return (Long) dims[dimIndex]; } - @Override - public boolean isNull() - { - final Object[] dims = currEntry.get().getDims(); - return dimIndex >= dims.length || dims[dimIndex] == null; - } - @SuppressWarnings("deprecation") @Nullable @Override diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index f7c01805044c..5138b072fee5 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -58,7 +58,7 @@ protected void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedi { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - + // If using default values for null use LongColumnSerializer to allow rollback to previous versions. this.serializer = IndexMergerV9.createLongColumnSerializer( segmentWriteOutMedium, dimensionName, From aaba00baaa0dbf4c802ddb0b05dd400154c2d5c1 Mon Sep 17 00:00:00 2001 From: Nishant Date: Thu, 1 Feb 2018 01:38:54 +0530 Subject: [PATCH 05/19] more comments --- .../java/io/druid/segment/BaseNullableColumnValueSelector.java | 2 ++ .../src/main/java/io/druid/segment/DimensionSelector.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java b/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java index 89b47efd06f2..2536935f619f 100644 --- a/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/BaseNullableColumnValueSelector.java @@ -20,6 +20,7 @@ package io.druid.segment; import io.druid.guice.annotations.PublicApi; +import io.druid.query.monomorphicprocessing.CalledFromHotLoop; /** * Null value checking polymorphic "part" of the {@link ColumnValueSelector} interface for primitive values. @@ -34,5 +35,6 @@ public interface BaseNullableColumnValueSelector * returns true if selected primitive value is null for {@link BaseFloatColumnValueSelector}, * {@link BaseLongColumnValueSelector} and {@link BaseDoubleColumnValueSelector} otherwise false. */ + @CalledFromHotLoop boolean isNull(); } diff --git a/processing/src/main/java/io/druid/segment/DimensionSelector.java b/processing/src/main/java/io/druid/segment/DimensionSelector.java index 2623dcc73290..3180ddb38885 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelector.java @@ -149,6 +149,7 @@ default long getLong() return 0L; } + @Deprecated @Override default boolean isNull() { @@ -162,7 +163,6 @@ default boolean isNull() * This method is not the default implementation of {@link #getObject()} to minimize the chance that implementations * "forget" to override it with more optimized version. */ - @Deprecated @Nullable default Object defaultGetObject() { From c5a5d65ab3950d50fe9989f526fdaf5ec91605b3 Mon Sep 17 00:00:00 2001 From: Nishant Date: Sat, 3 Feb 2018 02:08:08 +0530 Subject: [PATCH 06/19] review comments --- .../RowBasedColumnSelectorFactory.java | 6 ++-- .../druid/segment/DoubleDimensionIndexer.java | 2 +- .../segment/DoubleDimensionMergerV9.java | 4 --- .../druid/segment/FloatDimensionIndexer.java | 2 +- .../druid/segment/FloatDimensionMergerV9.java | 4 --- .../java/io/druid/segment/IndexMergerV9.java | 33 +++++++------------ .../druid/segment/LongDimensionIndexer.java | 2 +- .../druid/segment/LongDimensionMergerV9.java | 7 ---- .../druid/segment/StringDimensionHandler.java | 4 +-- .../io/druid/segment/data/GenericIndexed.java | 10 ++++-- 10 files changed, 27 insertions(+), 47 deletions(-) diff --git a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java index 15e78e6d3be8..a5116f99221b 100644 --- a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java @@ -426,7 +426,7 @@ public double getDouble() { Number metric = row.get().getMetric(columnName); if (NullHandling.sqlCompatible() && metric == null) { - throw new IllegalStateException("Cannot return double for Null Value"); + throw new IllegalStateException("Cannot return null value as double"); } return DimensionHandlerUtils.nullToZero(metric).doubleValue(); } @@ -436,7 +436,7 @@ public float getFloat() { Number metric = row.get().getMetric(columnName); if (NullHandling.sqlCompatible() && metric == null) { - throw new IllegalStateException("Cannot return float for Null Value"); + throw new IllegalStateException("Cannot return null value as float"); } return DimensionHandlerUtils.nullToZero(metric).floatValue(); } @@ -446,7 +446,7 @@ public long getLong() { Number metric = row.get().getMetric(columnName); if (NullHandling.sqlCompatible() && metric == null) { - throw new IllegalStateException("Cannot return long for Null Value"); + throw new IllegalStateException("Cannot return null value as long"); } return DimensionHandlerUtils.nullToZero(metric).longValue(); } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index 5b69f8f0415b..16fbb9cbd923 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -116,7 +116,7 @@ public double getDouble() if (dimIndex >= dims.length || dims[dimIndex] == null) { if (NullHandling.sqlCompatible()) { - throw new IllegalStateException("Cannot return double for Null Value"); + throw new IllegalStateException("Cannot return null value as double"); } return 0.0; } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index 9e7c37ff3770..459ef19af605 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -21,7 +21,6 @@ import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressionStrategy; import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; @@ -54,12 +53,9 @@ public DoubleDimensionMergerV9( private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { - final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - // If using default values for null use DoubleColumnSerializer to allow rollback to previous versions. this.serializer = IndexMergerV9.createDoubleColumnSerializer( segmentWriteOutMedium, dimensionName, - metCompression, indexSpec ); diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 02c7f4c87944..4a8bbf94819d 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -117,7 +117,7 @@ public float getFloat() if (dimIndex >= dims.length || dims[dimIndex] == null) { if (NullHandling.sqlCompatible()) { - throw new IllegalStateException("Cannot return float for Null Value"); + throw new IllegalStateException("Cannot return null value as float"); } return 0.0f; } diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index 497a3bf5e985..574c80f261fe 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -21,7 +21,6 @@ import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressionStrategy; import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; @@ -54,12 +53,9 @@ public FloatDimensionMergerV9( private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { - final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - // If using default values for null use FloatColumnSerializer to allow rollback to previous versions. this.serializer = IndexMergerV9.createFloatColumnSerializer( segmentWriteOutMedium, dimensionName, - metCompression, indexSpec ); serializer.open(); diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index 5266617a9184..00e7e39021bf 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -52,8 +52,6 @@ import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; @@ -547,8 +545,6 @@ private GenericColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWri GenericColumnSerializer timeWriter = createLongColumnSerializer( segmentWriteOutMedium, "little_end_time", - CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY, - indexSpec.getLongEncoding(), indexSpec ); // we will close this writer after we added all the timestamps @@ -565,20 +561,19 @@ private ArrayList setupMetricsWriters( ) throws IOException { ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); - final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); + for (String metric : mergedMetrics) { ValueType type = metricsValueTypes.get(metric); GenericColumnSerializer writer; switch (type) { case LONG: - writer = createLongColumnSerializer(segmentWriteOutMedium, metric, metCompression, longEncoding, indexSpec); + writer = createLongColumnSerializer(segmentWriteOutMedium, metric, indexSpec); break; case FLOAT: - writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, metCompression, indexSpec); + writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, indexSpec); break; case DOUBLE: - writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, metCompression, indexSpec); + writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, indexSpec); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -601,8 +596,6 @@ private ArrayList setupMetricsWriters( static GenericColumnSerializer createLongColumnSerializer( SegmentWriteOutMedium segmentWriteOutMedium, String columnName, - CompressionStrategy metCompression, - CompressionFactory.LongEncodingStrategy longEncoding, IndexSpec indexSpec ) { @@ -611,15 +604,15 @@ static GenericColumnSerializer createLongColumnSerializer( return LongColumnSerializer.create( segmentWriteOutMedium, columnName, - metCompression, - longEncoding + indexSpec.getMetricCompression(), + indexSpec.getLongEncoding() ); } else { return LongColumnSerializerV2.create( segmentWriteOutMedium, columnName, - metCompression, - longEncoding, + indexSpec.getMetricCompression(), + indexSpec.getLongEncoding(), indexSpec.getBitmapSerdeFactory() ); } @@ -628,7 +621,6 @@ static GenericColumnSerializer createLongColumnSerializer( static GenericColumnSerializer createDoubleColumnSerializer( SegmentWriteOutMedium segmentWriteOutMedium, String columnName, - CompressionStrategy metCompression, IndexSpec indexSpec ) { @@ -637,13 +629,13 @@ static GenericColumnSerializer createDoubleColumnSerializer( return DoubleColumnSerializer.create( segmentWriteOutMedium, columnName, - metCompression + indexSpec.getMetricCompression() ); } else { return DoubleColumnSerializerV2.create( segmentWriteOutMedium, columnName, - metCompression, + indexSpec.getMetricCompression(), indexSpec.getBitmapSerdeFactory() ); } @@ -652,7 +644,6 @@ static GenericColumnSerializer createDoubleColumnSerializer( static GenericColumnSerializer createFloatColumnSerializer( SegmentWriteOutMedium segmentWriteOutMedium, String columnName, - CompressionStrategy metCompression, IndexSpec indexSpec ) { @@ -661,13 +652,13 @@ static GenericColumnSerializer createFloatColumnSerializer( return FloatColumnSerializer.create( segmentWriteOutMedium, columnName, - metCompression + indexSpec.getMetricCompression() ); } else { return FloatColumnSerializerV2.create( segmentWriteOutMedium, columnName, - metCompression, + indexSpec.getMetricCompression(), indexSpec.getBitmapSerdeFactory() ); } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 412e868db1ed..38fece2471c1 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -117,7 +117,7 @@ public long getLong() if (dimIndex >= dims.length || dims[dimIndex] == null) { if (NullHandling.sqlCompatible()) { - throw new IllegalStateException("Cannot return long for Null Value"); + throw new IllegalStateException("Cannot return null value as long"); } return 0; } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index 5138b072fee5..122a198dcbac 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -22,8 +22,6 @@ import com.google.common.base.Throwables; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.CompressionStrategy; import io.druid.segment.serde.ColumnPartSerde; import io.druid.segment.writeout.SegmentWriteOutMedium; @@ -56,14 +54,9 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 protected void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { - final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - // If using default values for null use LongColumnSerializer to allow rollback to previous versions. this.serializer = IndexMergerV9.createLongColumnSerializer( segmentWriteOutMedium, dimensionName, - metCompression, - longEncoding, indexSpec ); serializer.open(); diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index fe3387f5548b..79caecd57130 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -77,7 +77,7 @@ public int compareSortedEncodedKeyComponents(int[] lhs, int[] rhs) return retVal; } - private boolean isNUllRow(int[] row, Indexed encodings) + private boolean isNullRow(int[] row, Indexed encodings) { if (row == null) { return true; @@ -100,7 +100,7 @@ public void validateSortedEncodedKeyComponents( ) throws SegmentValidationException { if (lhs == null || rhs == null) { - if (!isNUllRow(lhs, lhsEncodings) || !isNUllRow(rhs, rhsEncodings)) { + if (!isNullRow(lhs, lhsEncodings) || !isNullRow(rhs, rhsEncodings)) { throw new SegmentValidationException( "Expected nulls, found %s and %s", Arrays.toString(lhs), diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 2e08985e050c..63a49d04508e 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -52,7 +52,7 @@ *

* byte 1: version (0x1) * byte 2 == 0x1 =>; allowReverseLookup - * bytes 3-6 =>; numBytesUsed + * bytes 3-6 =>; numBytesUsed, It will be -1 for null values. * bytes 7-10 =>; numElements * bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values * bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes @@ -80,6 +80,8 @@ public class GenericIndexed implements Indexed, Serializer static final byte REVERSE_LOOKUP_ALLOWED = 0x1; static final byte REVERSE_LOOKUP_DISALLOWED = 0x0; + static final int NULL_VALUE_SIZE_MARKER = -1; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper .firstWriteByte((GenericIndexed x) -> VERSION_ONE) .writeByte(x -> x.allowReverseLookup ? REVERSE_LOOKUP_ALLOWED : REVERSE_LOOKUP_DISALLOWED) @@ -96,12 +98,14 @@ public Class getClazz() return String.class; } + /** + * numBytes will be {@link NULL_VALUE_SIZE_MARKER} for null values. + */ @Override @Nullable public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { if (numBytes < 0) { - // nulBytes will be -1 for null values. return null; } return NullHandling.emptyToNullIfNeeded(StringUtils.fromUtf8Nullable(buffer, numBytes)); @@ -492,7 +496,7 @@ private static GenericIndexed fromIterableVersionOne( allowReverseLookup = false; } - valuesOut.writeInt(next == null ? -1 : 0); + valuesOut.writeInt(next == null ? NULL_VALUE_SIZE_MARKER : 0); if (next != null) { strategy.writeTo(next, valuesOut); } From 606b98051303ee2b549e2b9b1c43b77ef85dee61 Mon Sep 17 00:00:00 2001 From: Nishant Date: Mon, 5 Feb 2018 23:51:41 +0530 Subject: [PATCH 07/19] review comments --- .../aggregation/ObjectAggregateCombiner.java | 8 ++---- .../java/io/druid/segment/IndexMerger.java | 2 +- .../druid/segment/ObjectColumnSelector.java | 18 ++++++------- .../druid/segment/StringDimensionHandler.java | 7 ++--- .../druid/segment/StringDimensionIndexer.java | 27 ++++++++++--------- .../column/SimpleDictionaryEncodedColumn.java | 7 ----- .../segment/incremental/IncrementalIndex.java | 2 +- .../segment/TestObjectColumnSelector.java | 2 +- 8 files changed, 31 insertions(+), 42 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java index bc9a594ee8d0..b7709702f4e1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/ObjectAggregateCombiner.java @@ -24,11 +24,7 @@ /** * Specialization of {@link AggregateCombiner} for object aggregations. */ -public abstract class ObjectAggregateCombiner implements AggregateCombiner, ObjectColumnSelector +public abstract class ObjectAggregateCombiner extends ObjectColumnSelector implements AggregateCombiner { - @Override - public final boolean isNull() - { - return false; - } + // No methods } diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 3611a15eb7c1..75150b767f50 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -456,7 +456,7 @@ class DictionaryMergeIterator implements CloseableIterator final PeekingIterator iter = Iterators.peekingIterator( Iterators.transform( indexed.iterator(), - input -> NullHandling.nullToEmptyIfNeeded(input) + NullHandling::nullToEmptyIfNeeded ) ); if (iter.hasNext()) { diff --git a/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java b/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java index c9d739183235..d1e377a13abd 100644 --- a/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/ObjectColumnSelector.java @@ -27,7 +27,7 @@ * This interface should appear ONLY in "implements" clause or anonymous class creation, but NOT in "user" code, where * {@link BaseObjectColumnValueSelector} must be used instead. */ -public interface ObjectColumnSelector extends ColumnValueSelector +public abstract class ObjectColumnSelector implements ColumnValueSelector { /** * @deprecated This method is marked as deprecated in ObjectColumnSelector to minimize the probability of accidential @@ -36,7 +36,7 @@ public interface ObjectColumnSelector extends ColumnValueSelector */ @Deprecated @Override - default float getFloat() + public float getFloat() { T value = getObject(); if (value == null) { @@ -52,7 +52,7 @@ default float getFloat() */ @Deprecated @Override - default double getDouble() + public double getDouble() { T value = getObject(); if (value == null) { @@ -68,7 +68,7 @@ default double getDouble() */ @Deprecated @Override - default long getLong() + public long getLong() { T value = getObject(); if (value == null) { @@ -78,15 +78,13 @@ default long getLong() } /** - * @deprecated This method is marked as deprecated in ObjectColumnSelector to minimize the probability of accidential - * calling. "Polymorphism" of ObjectColumnSelector should be used only when operating on {@link ColumnValueSelector} - * objects. + * @deprecated This method is marked as deprecated in ObjectColumnSelector since it always returns false. + * There is no need to call this method. */ @Deprecated @Override - default boolean isNull() + public final boolean isNull() { - T value = getObject(); - return value == null; + return false; } } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index 79caecd57130..596b67fc07a2 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -28,6 +28,7 @@ import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; +import javax.annotation.Nullable; import java.io.Closeable; import java.lang.reflect.Array; import java.util.Arrays; @@ -77,13 +78,13 @@ public int compareSortedEncodedKeyComponents(int[] lhs, int[] rhs) return retVal; } - private boolean isNullRow(int[] row, Indexed encodings) + private boolean isNullRow(@Nullable int[] row, Indexed encodings) { if (row == null) { return true; } - for (int i : row) { - if (encodings.get(i) != null) { + for (int value : row) { + if (encodings.get(value) != null) { // Non-Null value return false; } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 08e4b8592cab..85f5f479b44c 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -54,16 +54,19 @@ import java.util.BitSet; import java.util.Iterator; import java.util.List; -import java.util.function.Function; public class StringDimensionIndexer implements DimensionIndexer { - private static final Function EMPTY_TO_NULL_IF_NEEDED = o -> o != null - ? NullHandling.emptyToNullIfNeeded(o.toString()) - : null; + + private static String emptytoNullIfNeeded(Object o) + { + return o != null + ? NullHandling.emptyToNullIfNeeded(o.toString()) + : null; + } private static final int ABSENT_VALUE_ID = -1; - private static final int[] EMPTY_INT_ARRAY = new int[]{}; + private static final int[] EMPTY_INT_ARRAY = IntArrays.EMPTY_ARRAY; private static class DimensionDictionary { @@ -115,6 +118,7 @@ public boolean contains(String value) public int size() { synchronized (lock) { + // using idToValue rather than valueToId because the valueToId doesn't account null value, if it is present. return idToValue.size(); } } @@ -222,22 +226,19 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole final int oldDictSize = dimLookup.size(); if (dimValues == null) { - if (!dimLookup.contains(null)) { - encodedDimensionValues = new int[]{dimLookup.add(null)}; - } else { - encodedDimensionValues = new int[]{dimLookup.getId(null)}; - } + final int nullId = dimLookup.getId(null); + encodedDimensionValues = nullId == ABSENT_VALUE_ID ? new int[]{dimLookup.add(null)} : new int[]{nullId}; } else if (dimValues instanceof List) { List dimValuesList = (List) dimValues; if (dimValuesList.isEmpty()) { dimLookup.add(null); encodedDimensionValues = EMPTY_INT_ARRAY; } else if (dimValuesList.size() == 1) { - encodedDimensionValues = new int[]{dimLookup.add(EMPTY_TO_NULL_IF_NEEDED.apply(dimValuesList.get(0)))}; + encodedDimensionValues = new int[]{dimLookup.add(emptytoNullIfNeeded(dimValuesList.get(0)))}; } else { final String[] dimensionValues = new String[dimValuesList.size()]; for (int i = 0; i < dimValuesList.size(); i++) { - dimensionValues[i] = EMPTY_TO_NULL_IF_NEEDED.apply(dimValuesList.get(i)); + dimensionValues[i] = emptytoNullIfNeeded(dimValuesList.get(i)); } if (multiValueHandling.needSorting()) { // Sort multival row by their unencoded values first. @@ -262,7 +263,7 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole encodedDimensionValues = pos == retVal.length ? retVal : Arrays.copyOf(retVal, pos); } } else { - encodedDimensionValues = new int[]{dimLookup.add(EMPTY_TO_NULL_IF_NEEDED.apply(dimValues))}; + encodedDimensionValues = new int[]{dimLookup.add(emptytoNullIfNeeded(dimValues))}; } // If dictionary size has changed, the sorted lookup is no longer valid. diff --git a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index b2dfca272137..705512ca54d2 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -176,13 +176,6 @@ public ValueMatcher makeValueMatcher(Predicate predicate) return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); } - @Override - public boolean isNull() - { - IndexedInts row = getRow(); - return row == null || row.size() == 0; - } - @Nullable @Override public Object getObject() 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 d56ba07b4827..a78a780a123d 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -1417,7 +1417,7 @@ public boolean isNull() } } - private class ObjectMetricColumnSelector implements ObjectColumnSelector + private class ObjectMetricColumnSelector extends ObjectColumnSelector { private final TimeAndDimsHolder currEntry; private final int metricIndex; diff --git a/processing/src/test/java/io/druid/segment/TestObjectColumnSelector.java b/processing/src/test/java/io/druid/segment/TestObjectColumnSelector.java index 34dc1e1375cc..5bc4618b2851 100644 --- a/processing/src/test/java/io/druid/segment/TestObjectColumnSelector.java +++ b/processing/src/test/java/io/druid/segment/TestObjectColumnSelector.java @@ -21,7 +21,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; -public abstract class TestObjectColumnSelector implements ObjectColumnSelector +public abstract class TestObjectColumnSelector extends ObjectColumnSelector { @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) From 3da96e434d17528fdc943696790abd2027a1a84b Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 6 Feb 2018 00:01:03 +0530 Subject: [PATCH 08/19] Optimize isNull method --- .../src/main/java/io/druid/segment/column/DoublesColumn.java | 4 +++- .../src/main/java/io/druid/segment/column/FloatsColumn.java | 4 +++- .../src/main/java/io/druid/segment/column/LongsColumn.java | 4 +++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java index 55c499698ab9..dc650de72acc 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java @@ -30,11 +30,13 @@ public class DoublesColumn implements GenericColumn { private final ColumnarDoubles column; private final ImmutableBitmap nullValueBitmap; + private final boolean hasNulls; public DoublesColumn(ColumnarDoubles columnarDoubles, ImmutableBitmap nullValueBitmap) { column = columnarDoubles; this.nullValueBitmap = nullValueBitmap; + this.hasNulls = !nullValueBitmap.isEmpty(); } @Override @@ -70,7 +72,7 @@ public double getDoubleSingleValueRow(int rowNum) @Override public boolean isNull(int rowNum) { - return nullValueBitmap.get(rowNum); + return hasNulls && nullValueBitmap.get(rowNum); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index 51a869b1c98d..f16813a9be1b 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -31,6 +31,7 @@ public class FloatsColumn implements GenericColumn { private final ColumnarFloats column; private final ImmutableBitmap nullValueBitmap; + private final boolean hasNulls; public FloatsColumn( final ColumnarFloats column, @@ -39,6 +40,7 @@ public FloatsColumn( { this.column = column; this.nullValueBitmap = nullValueBitmap; + this.hasNulls = !nullValueBitmap.isEmpty(); } @Override @@ -74,7 +76,7 @@ public double getDoubleSingleValueRow(int rowNum) @Override public boolean isNull(int rowNum) { - return nullValueBitmap.get(rowNum); + return hasNulls && nullValueBitmap.get(rowNum); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 2b5c5b864713..9d90e8f385f3 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -31,6 +31,7 @@ public class LongsColumn implements GenericColumn { private final ColumnarLongs column; private final ImmutableBitmap nullValueBitmap; + private final boolean hasNulls; public LongsColumn( final ColumnarLongs column, @@ -39,6 +40,7 @@ public LongsColumn( { this.column = column; this.nullValueBitmap = nullValueBitmap; + this.hasNulls = !nullValueBitmap.isEmpty(); } @Override @@ -74,7 +76,7 @@ public double getDoubleSingleValueRow(int rowNum) @Override public boolean isNull(int rowNum) { - return nullValueBitmap.get(rowNum); + return hasNulls && nullValueBitmap.get(rowNum); } @Override From 1054d6bb6e75c1aa880ed87663c507d3e2425cb9 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 6 Feb 2018 00:03:29 +0530 Subject: [PATCH 09/19] Optimize isNull in ColumnarFloats/Longs/Doubles --- .../src/main/java/io/druid/segment/data/ColumnarDoubles.java | 3 ++- .../src/main/java/io/druid/segment/data/ColumnarFloats.java | 3 ++- .../src/main/java/io/druid/segment/data/ColumnarLongs.java | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java index e4ac821d5455..9392d1137d45 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java @@ -41,6 +41,7 @@ public interface ColumnarDoubles extends Closeable default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { + final boolean hasNulls = !nullValueBitmap.isEmpty(); class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector { @Override @@ -58,7 +59,7 @@ public double getDouble(int offset) @Override public boolean isNull() { - return nullValueBitmap.get(offset.getOffset()); + return hasNulls && nullValueBitmap.get(offset.getOffset()); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java index b0a701047a89..89591655a536 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java @@ -42,6 +42,7 @@ public interface ColumnarFloats extends Closeable default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { + final boolean hasNulls = !nullValueBitmap.isEmpty(); class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector { @Override @@ -53,7 +54,7 @@ public float getFloat() @Override public boolean isNull() { - return nullValueBitmap.get(offset.getOffset()); + return hasNulls && nullValueBitmap.get(offset.getOffset()); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java index de67a2d25071..68a5610c5c95 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java @@ -42,6 +42,7 @@ public interface ColumnarLongs extends Closeable default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { + final boolean hasNulls = !nullValueBitmap.isEmpty(); class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector { @Override @@ -59,7 +60,7 @@ public double getDouble(int offset) @Override public boolean isNull() { - return nullValueBitmap.get(offset.getOffset()); + return hasNulls && nullValueBitmap.get(offset.getOffset()); } @Override From 6ef4d68dcefb6559b2b4a4ee80bd5c57bfeb4a34 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 6 Feb 2018 00:54:40 +0530 Subject: [PATCH 10/19] review comment - separate classes for null and non-null columns fix intellij inspection --- .../druid/segment/column/DoublesColumn.java | 13 ++--- .../column/DoublesColumnWithNulls.java | 52 +++++++++++++++++++ .../io/druid/segment/column/FloatsColumn.java | 16 ++---- .../segment/column/FloatsColumnWithNulls.java | 52 +++++++++++++++++++ .../io/druid/segment/column/LongsColumn.java | 16 ++---- .../segment/column/LongsColumnWithNulls.java | 52 +++++++++++++++++++ .../druid/segment/data/ColumnarDoubles.java | 35 ++++++++++++- .../io/druid/segment/data/ColumnarFloats.java | 32 ++++++++++++ .../io/druid/segment/data/ColumnarLongs.java | 32 ++++++++++++ .../serde/DoubleGenericColumnSupplier.java | 9 +++- .../serde/FloatGenericColumnSupplier.java | 9 +++- .../serde/LongGenericColumnSupplier.java | 7 ++- 12 files changed, 285 insertions(+), 40 deletions(-) create mode 100644 processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java create mode 100644 processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java create mode 100644 processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java index dc650de72acc..a2f8e4d5481a 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java @@ -19,7 +19,6 @@ package io.druid.segment.column; -import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarDoubles; @@ -28,15 +27,11 @@ public class DoublesColumn implements GenericColumn { - private final ColumnarDoubles column; - private final ImmutableBitmap nullValueBitmap; - private final boolean hasNulls; + final ColumnarDoubles column; - public DoublesColumn(ColumnarDoubles columnarDoubles, ImmutableBitmap nullValueBitmap) + public DoublesColumn(ColumnarDoubles columnarDoubles) { column = columnarDoubles; - this.nullValueBitmap = nullValueBitmap; - this.hasNulls = !nullValueBitmap.isEmpty(); } @Override @@ -48,7 +43,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset, nullValueBitmap); + return column.makeColumnValueSelector(offset); } @Override @@ -72,7 +67,7 @@ public double getDoubleSingleValueRow(int rowNum) @Override public boolean isNull(int rowNum) { - return hasNulls && nullValueBitmap.get(rowNum); + return false; } @Override diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java new file mode 100644 index 000000000000..8bfd6d9ef08f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java @@ -0,0 +1,52 @@ +/* + * 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.collections.bitmap.ImmutableBitmap; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.data.ColumnarDoubles; +import io.druid.segment.data.ReadableOffset; + +/** + * DoublesColumn with null values. + */ +public class DoublesColumnWithNulls extends DoublesColumn +{ + private final ImmutableBitmap nullValueBitmap; + + public DoublesColumnWithNulls(ColumnarDoubles columnarDoubles, ImmutableBitmap nullValueBitmap) + { + super(columnarDoubles); + this.nullValueBitmap = nullValueBitmap; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return column.makeColumnValueSelector(offset, nullValueBitmap); + } + + @Override + public boolean isNull(int rowNum) + { + return nullValueBitmap.get(rowNum); + } + +} diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index f16813a9be1b..5525fcd6b533 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -19,7 +19,6 @@ package io.druid.segment.column; -import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarFloats; @@ -29,18 +28,11 @@ */ public class FloatsColumn implements GenericColumn { - private final ColumnarFloats column; - private final ImmutableBitmap nullValueBitmap; - private final boolean hasNulls; + final ColumnarFloats column; - public FloatsColumn( - final ColumnarFloats column, - ImmutableBitmap nullValueBitmap - ) + public FloatsColumn(final ColumnarFloats column) { this.column = column; - this.nullValueBitmap = nullValueBitmap; - this.hasNulls = !nullValueBitmap.isEmpty(); } @Override @@ -52,7 +44,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset, nullValueBitmap); + return column.makeColumnValueSelector(offset); } @Override @@ -76,7 +68,7 @@ public double getDoubleSingleValueRow(int rowNum) @Override public boolean isNull(int rowNum) { - return hasNulls && nullValueBitmap.get(rowNum); + return false; } @Override diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java new file mode 100644 index 000000000000..bb64107f85b6 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java @@ -0,0 +1,52 @@ +/* + * 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.collections.bitmap.ImmutableBitmap; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.data.ColumnarFloats; +import io.druid.segment.data.ReadableOffset; + +/** + * FloatsColumn with null values. + */ +public class FloatsColumnWithNulls extends FloatsColumn +{ + private final ImmutableBitmap nullValueBitmap; + + public FloatsColumnWithNulls(ColumnarFloats columnarFloats, ImmutableBitmap nullValueBitmap) + { + super(columnarFloats); + this.nullValueBitmap = nullValueBitmap; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return column.makeColumnValueSelector(offset, nullValueBitmap); + } + + @Override + public boolean isNull(int rowNum) + { + return nullValueBitmap.get(rowNum); + } + +} diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 9d90e8f385f3..7657087d98a0 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -19,7 +19,6 @@ package io.druid.segment.column; -import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarLongs; @@ -29,18 +28,11 @@ */ public class LongsColumn implements GenericColumn { - private final ColumnarLongs column; - private final ImmutableBitmap nullValueBitmap; - private final boolean hasNulls; + final ColumnarLongs column; - public LongsColumn( - final ColumnarLongs column, - ImmutableBitmap nullValueBitmap - ) + public LongsColumn(final ColumnarLongs column) { this.column = column; - this.nullValueBitmap = nullValueBitmap; - this.hasNulls = !nullValueBitmap.isEmpty(); } @Override @@ -52,7 +44,7 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset, nullValueBitmap); + return column.makeColumnValueSelector(offset); } @Override @@ -76,7 +68,7 @@ public double getDoubleSingleValueRow(int rowNum) @Override public boolean isNull(int rowNum) { - return hasNulls && nullValueBitmap.get(rowNum); + return false; } @Override diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java new file mode 100644 index 000000000000..e8a2f22adaea --- /dev/null +++ b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java @@ -0,0 +1,52 @@ +/* + * 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.collections.bitmap.ImmutableBitmap; +import io.druid.segment.ColumnValueSelector; +import io.druid.segment.data.ColumnarLongs; +import io.druid.segment.data.ReadableOffset; + +/** + * LongsColumn with null values. + */ +public class LongsColumnWithNulls extends LongsColumn +{ + private final ImmutableBitmap nullValueBitmap; + + public LongsColumnWithNulls(ColumnarLongs columnarLongs, ImmutableBitmap nullValueBitmap) + { + super(columnarLongs); + this.nullValueBitmap = nullValueBitmap; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return column.makeColumnValueSelector(offset, nullValueBitmap); + } + + @Override + public boolean isNull(int rowNum) + { + return nullValueBitmap.get(rowNum); + } + +} diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java index 9392d1137d45..f9ab57f49918 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java @@ -39,9 +39,40 @@ public interface ColumnarDoubles extends Closeable @Override void close(); + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector + { + @Override + public double getDouble() + { + return ColumnarDoubles.this.get(offset.getOffset()); + } + + @Override + public double getDouble(int offset) + { + return ColumnarDoubles.this.get(offset); + } + + @Override + public boolean isNull() + { + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarDoubles.this); + inspector.visit("offset", offset); + } + } + return new HistoricalDoubleColumnSelector(); + } + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { - final boolean hasNulls = !nullValueBitmap.isEmpty(); class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector { @Override @@ -59,7 +90,7 @@ public double getDouble(int offset) @Override public boolean isNull() { - return hasNulls && nullValueBitmap.get(offset.getOffset()); + return nullValueBitmap.get(offset.getOffset()); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java index 89591655a536..ba073e2b1bdf 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java @@ -40,6 +40,38 @@ public interface ColumnarFloats extends Closeable @Override void close(); + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector + { + @Override + public float getFloat() + { + return ColumnarFloats.this.get(offset.getOffset()); + } + + @Override + public boolean isNull() + { + return false; + } + + @Override + public double getDouble(int offset) + { + return ColumnarFloats.this.get(offset); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarFloats.this); + inspector.visit("offset", offset); + } + } + return new HistoricalFloatColumnSelector(); + } + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { final boolean hasNulls = !nullValueBitmap.isEmpty(); diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java index 68a5610c5c95..55f30f3d0e67 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java @@ -72,4 +72,36 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) } return new HistoricalLongColumnSelector(); } + + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector + { + @Override + public long getLong() + { + return ColumnarLongs.this.get(offset.getOffset()); + } + + @Override + public double getDouble(int offset) + { + return ColumnarLongs.this.get(offset); + } + + @Override + public boolean isNull() + { + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarLongs.this); + inspector.visit("offset", offset); + } + } + return new HistoricalLongColumnSelector(); + } } diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java index ae4ea7a50ba5..f0e36e132f38 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java @@ -21,8 +21,9 @@ import com.google.common.base.Supplier; import io.druid.collections.bitmap.ImmutableBitmap; -import io.druid.segment.column.GenericColumn; import io.druid.segment.column.DoublesColumn; +import io.druid.segment.column.DoublesColumnWithNulls; +import io.druid.segment.column.GenericColumn; import io.druid.segment.data.ColumnarDoubles; @@ -43,6 +44,10 @@ public DoubleGenericColumnSupplier( @Override public GenericColumn get() { - return new DoublesColumn(column.get(), nullValueBitmap); + if (nullValueBitmap.isEmpty()) { + return new DoublesColumn(column.get()); + } else { + return new DoublesColumnWithNulls(column.get(), nullValueBitmap); + } } } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java index 0d40a02cadb4..98973f72fc25 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java @@ -21,8 +21,9 @@ import com.google.common.base.Supplier; import io.druid.collections.bitmap.ImmutableBitmap; -import io.druid.segment.column.GenericColumn; import io.druid.segment.column.FloatsColumn; +import io.druid.segment.column.FloatsColumnWithNulls; +import io.druid.segment.column.GenericColumn; import io.druid.segment.data.CompressedColumnarFloatsSupplier; /** @@ -44,6 +45,10 @@ public FloatGenericColumnSupplier( @Override public GenericColumn get() { - return new FloatsColumn(column.get(), nullValueBitmap); + if (nullValueBitmap.isEmpty()) { + return new FloatsColumn(column.get()); + } else { + return new FloatsColumnWithNulls(column.get(), nullValueBitmap); + } } } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java index e71657b2a46a..1b74395c6c88 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java @@ -23,6 +23,7 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.LongsColumn; +import io.druid.segment.column.LongsColumnWithNulls; import io.druid.segment.data.CompressedColumnarLongsSupplier; /** @@ -41,6 +42,10 @@ public LongGenericColumnSupplier(CompressedColumnarLongsSupplier column, Immutab @Override public GenericColumn get() { - return new LongsColumn(column.get(), nullValueBitmap); + if (nullValueBitmap.isEmpty()) { + return new LongsColumn(column.get()); + } else { + return new LongsColumnWithNulls(column.get(), nullValueBitmap); + } } } From 5564b183c694cf0e8597846919cc669be9e9eda4 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 6 Feb 2018 16:31:38 +0530 Subject: [PATCH 11/19] remove unused import --- .../java/io/druid/segment/incremental/IncrementalIndexTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index 7dd331e08ecb..ca6ffe9fb66f 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import io.druid.collections.StupidPool; -import io.druid.common.config.NullHandling; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionSchema; import io.druid.data.input.impl.DimensionsSpec; From c0bf599c7f77156a81ac00aaa8f3a064d9d36d72 Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 6 Feb 2018 22:40:53 +0530 Subject: [PATCH 12/19] More Review comments --- .../io/druid/common/config/NullHandling.java | 4 +- .../query/aggregation/BufferAggregator.java | 3 +- .../druid/segment/DoubleDimensionIndexer.java | 3 - .../druid/segment/FloatDimensionIndexer.java | 3 - .../druid/segment/LongDimensionIndexer.java | 3 - .../druid/segment/NilColumnValueSelector.java | 37 ++++++- .../druid/segment/StringDimensionIndexer.java | 10 -- .../druid/segment/column/DoublesColumn.java | 4 +- .../column/DoublesColumnWithNulls.java | 7 ++ .../io/druid/segment/column/FloatsColumn.java | 6 +- .../segment/column/FloatsColumnWithNulls.java | 6 +- .../io/druid/segment/column/LongsColumn.java | 6 +- .../segment/column/LongsColumnWithNulls.java | 7 +- .../column/SimpleDictionaryEncodedColumn.java | 6 - .../druid/segment/data/ColumnarDoubles.java | 101 ++++++++--------- .../io/druid/segment/data/ColumnarFloats.java | 103 +++++++++--------- .../io/druid/segment/data/ColumnarLongs.java | 101 ++++++++--------- .../io/druid/segment/data/GenericIndexed.java | 14 ++- .../data/RoaringBitmapSerdeFactory.java | 2 + .../java/io/druid/segment/filter/Filters.java | 3 +- 20 files changed, 231 insertions(+), 198 deletions(-) diff --git a/common/src/main/java/io/druid/common/config/NullHandling.java b/common/src/main/java/io/druid/common/config/NullHandling.java index 538d6de94640..20a5d7cf52ef 100644 --- a/common/src/main/java/io/druid/common/config/NullHandling.java +++ b/common/src/main/java/io/druid/common/config/NullHandling.java @@ -26,7 +26,7 @@ /** * Helper class for NullHandling. This class is used to switch between SQL compatible Null Handling behavior - * introduced as part of {@link https://github.com/druid-io/druid/issues/4349} and the old druid behavior + * introduced as part of https://github.com/druid-io/druid/issues/4349 and the old druid behavior * where null values are replaced with default values e.g Null Strings are replaced with empty values. */ public class NullHandling @@ -44,7 +44,7 @@ public class NullHandling /** * INSTANCE is injected using static injection to avoid adding JacksonInject annotations all over the code. - * See {@link io.druid.guice.NullHandlingModule} for details. + * See io.druid.guice.NullHandlingModule for details. * It does not take effect in all unit tests since we don't use Guice Injection. * For tests default system property is supposed to be used only in tests */ 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 2345a4dbf2eb..60e8d841d63e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -181,7 +181,7 @@ default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, By /** * returns true if aggregator's output type is primitive long/double/float and aggregated value is null, * but when aggregated output type is Object, this method always returns false, - * and users are advised to check nullability for the object returned by {@link #get()} + * and users are advised to check nullability for the object returned by {@link #get(ByteBuffer, int)} ()} * method. * The default implementation always return false to enable smooth backward compatibility, * re-implement if your aggregator is nullable. @@ -191,7 +191,6 @@ default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, By * * @return true if the aggregated value is primitive long/double/float and aggregated value is null otherwise false. */ - @CalledFromHotLoop default boolean isNull(ByteBuffer buf, int position) { return false; diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index 16fbb9cbd923..9653495d8968 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -102,9 +102,6 @@ class IndexerDoubleColumnSelector implements DoubleColumnSelector @Override public boolean isNull() { - if (NullHandling.replaceWithDefault()) { - return false; - } final Object[] dims = currEntry.get().getDims(); return dimIndex >= dims.length || dims[dimIndex] == null; } diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index 4a8bbf94819d..ecd986387174 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -103,9 +103,6 @@ class IndexerFloatColumnSelector implements FloatColumnSelector @Override public boolean isNull() { - if (NullHandling.replaceWithDefault()) { - return false; - } final Object[] dims = currEntry.get().getDims(); return dimIndex >= dims.length || dims[dimIndex] == null; } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 38fece2471c1..2327ed6387b8 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -103,9 +103,6 @@ class IndexerLongColumnSelector implements LongColumnSelector @Override public boolean isNull() { - if (NullHandling.replaceWithDefault()) { - return false; - } final Object[] dims = currEntry.get().getDims(); return dimIndex >= dims.length || dims[dimIndex] == null; } diff --git a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java index c19208711a42..75fc17e1e6c8 100644 --- a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java @@ -19,6 +19,7 @@ package io.druid.segment; +import io.druid.common.config.NullHandling; import io.druid.guice.annotations.PublicApi; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -28,9 +29,11 @@ * Represents "absent" column. */ @PublicApi -public final class NilColumnValueSelector implements ColumnValueSelector +public class NilColumnValueSelector implements ColumnValueSelector { - private static final NilColumnValueSelector INSTANCE = new NilColumnValueSelector(); + private static final NilColumnValueSelector INSTANCE = NullHandling.sqlCompatible() + ? new SqlCompatibleNilColumnValueSelector() + : new NilColumnValueSelector(); public static NilColumnValueSelector instance() { @@ -88,7 +91,7 @@ public Class classOfObject() @Override public boolean isNull() { - return true; + return false; } @Override @@ -96,4 +99,32 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { // nothing to inspect } + + private static class SqlCompatibleNilColumnValueSelector extends NilColumnValueSelector + { + @Override + public boolean isNull() + { + return true; + } + + @Override + public double getDouble() + { + throw new IllegalStateException("Cannot return null value as double"); + } + + @Override + public float getFloat() + { + throw new IllegalStateException("Cannot return null value as float"); + } + + @Override + public long getLong() + { + throw new IllegalStateException("Cannot return null value as long"); + } + + } } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index 85f5f479b44c..af977f32a03c 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -105,16 +105,6 @@ public String getValue(int id) } } - public boolean contains(String value) - { - synchronized (lock) { - if (value == null) { - return idForNull != ABSENT_VALUE_ID; - } - return valueToId.containsKey(value); - } - } - public int size() { synchronized (lock) { diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java index a2f8e4d5481a..2e175f4ae4c4 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java @@ -21,6 +21,7 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; +import io.druid.segment.IndexIO; import io.druid.segment.data.ColumnarDoubles; import io.druid.segment.data.ReadableOffset; @@ -43,7 +44,8 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset); + return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap()); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java index 8bfd6d9ef08f..232c56d457fc 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java @@ -20,6 +20,7 @@ package io.druid.segment.column; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarDoubles; import io.druid.segment.data.ReadableOffset; @@ -49,4 +50,10 @@ public boolean isNull(int rowNum) return nullValueBitmap.get(rowNum); } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + super.inspectRuntimeShape(inspector); + inspector.visit("nullValueBitmap", nullValueBitmap); + } } diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index 5525fcd6b533..3aa08d1ca1fd 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -21,11 +21,12 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; +import io.druid.segment.IndexIO; import io.druid.segment.data.ColumnarFloats; import io.druid.segment.data.ReadableOffset; /** -*/ + */ public class FloatsColumn implements GenericColumn { final ColumnarFloats column; @@ -44,7 +45,8 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset); + return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap()); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java index bb64107f85b6..2e9b2476e7c0 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java @@ -20,6 +20,7 @@ package io.druid.segment.column; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarFloats; import io.druid.segment.data.ReadableOffset; @@ -44,9 +45,10 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) } @Override - public boolean isNull(int rowNum) + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return nullValueBitmap.get(rowNum); + super.inspectRuntimeShape(inspector); + inspector.visit("nullValueBitmap", nullValueBitmap); } } diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 7657087d98a0..07f9a8ce0e36 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -21,11 +21,12 @@ import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; +import io.druid.segment.IndexIO; import io.druid.segment.data.ColumnarLongs; import io.druid.segment.data.ReadableOffset; /** -*/ + */ public class LongsColumn implements GenericColumn { final ColumnarLongs column; @@ -44,7 +45,8 @@ public int length() @Override public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) { - return column.makeColumnValueSelector(offset); + return column.makeColumnValueSelector(offset, IndexIO.LEGACY_FACTORY.getBitmapFactory() + .makeEmptyImmutableBitmap()); } @Override diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java index e8a2f22adaea..db88b7c290db 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java @@ -20,6 +20,7 @@ package io.druid.segment.column; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.data.ColumnarLongs; import io.druid.segment.data.ReadableOffset; @@ -44,9 +45,9 @@ public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) } @Override - public boolean isNull(int rowNum) + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return nullValueBitmap.get(rowNum); + super.inspectRuntimeShape(inspector); + inspector.visit("nullValueBitmap", nullValueBitmap); } - } diff --git a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java index 705512ca54d2..1cdf1ccbd244 100644 --- a/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java +++ b/processing/src/main/java/io/druid/segment/column/SimpleDictionaryEncodedColumn.java @@ -281,12 +281,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - @Override - public boolean isNull() - { - return getObject() == null; - } - @Override public Object getObject() { diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java index f9ab57f49918..66e9f2b80d08 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java @@ -34,73 +34,74 @@ public interface ColumnarDoubles extends Closeable { int size(); + double get(int index); @Override void close(); - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { - class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector - { - @Override - public double getDouble() + if (nullValueBitmap.isEmpty()) { + class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector { - return ColumnarDoubles.this.get(offset.getOffset()); - } + @Override + public double getDouble() + { + return ColumnarDoubles.this.get(offset.getOffset()); + } - @Override - public double getDouble(int offset) - { - return ColumnarDoubles.this.get(offset); - } + @Override + public double getDouble(int offset) + { + return ColumnarDoubles.this.get(offset); + } - @Override - public boolean isNull() - { - return false; - } + @Override + public boolean isNull() + { + return false; + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("columnar", ColumnarDoubles.this); - inspector.visit("offset", offset); + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarDoubles.this); + inspector.visit("offset", offset); + } } - } - return new HistoricalDoubleColumnSelector(); - } - - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) - { - class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector - { - @Override - public double getDouble() + return new HistoricalDoubleColumnSelector(); + } else { + class HistoricalDoubleColumnSelectorWithNulls implements DoubleColumnSelector, HistoricalColumnSelector { - return ColumnarDoubles.this.get(offset.getOffset()); - } + @Override + public double getDouble() + { + return ColumnarDoubles.this.get(offset.getOffset()); + } - @Override - public double getDouble(int offset) - { - return ColumnarDoubles.this.get(offset); - } + @Override + public double getDouble(int offset) + { + return ColumnarDoubles.this.get(offset); + } - @Override - public boolean isNull() - { - return nullValueBitmap.get(offset.getOffset()); - } + @Override + public boolean isNull() + { + return nullValueBitmap.get(offset.getOffset()); + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("columnar", ColumnarDoubles.this); - inspector.visit("offset", offset); + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarDoubles.this); + inspector.visit("offset", offset); + inspector.visit("nullValueBitmap", nullValueBitmap); + } } + return new HistoricalDoubleColumnSelectorWithNulls(); } - return new HistoricalDoubleColumnSelector(); } } diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java index ba073e2b1bdf..0a696d0cf780 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java @@ -34,74 +34,75 @@ public interface ColumnarFloats extends Closeable { int size(); + float get(int index); + void fill(int index, float[] toFill); @Override void close(); - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { - class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector - { - @Override - public float getFloat() + if (nullValueBitmap.isEmpty()) { + class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector { - return ColumnarFloats.this.get(offset.getOffset()); - } + @Override + public float getFloat() + { + return ColumnarFloats.this.get(offset.getOffset()); + } - @Override - public boolean isNull() - { - return false; - } + @Override + public boolean isNull() + { + return false; + } - @Override - public double getDouble(int offset) - { - return ColumnarFloats.this.get(offset); - } + @Override + public double getDouble(int offset) + { + return ColumnarFloats.this.get(offset); + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("columnar", ColumnarFloats.this); - inspector.visit("offset", offset); + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarFloats.this); + inspector.visit("offset", offset); + } } - } - return new HistoricalFloatColumnSelector(); - } - - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) - { - final boolean hasNulls = !nullValueBitmap.isEmpty(); - class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector - { - @Override - public float getFloat() + return new HistoricalFloatColumnSelector(); + } else { + class HistoricalFloatColumnSelectorwithNulls implements FloatColumnSelector, HistoricalColumnSelector { - return ColumnarFloats.this.get(offset.getOffset()); - } + @Override + public float getFloat() + { + return ColumnarFloats.this.get(offset.getOffset()); + } - @Override - public boolean isNull() - { - return hasNulls && nullValueBitmap.get(offset.getOffset()); - } + @Override + public boolean isNull() + { + return nullValueBitmap.get(offset.getOffset()); + } - @Override - public double getDouble(int offset) - { - return ColumnarFloats.this.get(offset); - } + @Override + public double getDouble(int offset) + { + return ColumnarFloats.this.get(offset); + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("columnar", ColumnarFloats.this); - inspector.visit("offset", offset); + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarFloats.this); + inspector.visit("offset", offset); + inspector.visit("nullValueBitmap", nullValueBitmap); + } } + return new HistoricalFloatColumnSelectorwithNulls(); } - return new HistoricalFloatColumnSelector(); } } diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java index 55f30f3d0e67..5670035a6afd 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java @@ -34,7 +34,9 @@ public interface ColumnarLongs extends Closeable { int size(); + long get(int index); + void fill(int index, long[] toFill); @Override @@ -42,66 +44,65 @@ public interface ColumnarLongs extends Closeable default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { - final boolean hasNulls = !nullValueBitmap.isEmpty(); - class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector - { - @Override - public long getLong() + if (nullValueBitmap.isEmpty()) { + class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector { - return ColumnarLongs.this.get(offset.getOffset()); - } + @Override + public long getLong() + { + return ColumnarLongs.this.get(offset.getOffset()); + } - @Override - public double getDouble(int offset) - { - return ColumnarLongs.this.get(offset); - } + @Override + public double getDouble(int offset) + { + return ColumnarLongs.this.get(offset); + } - @Override - public boolean isNull() - { - return hasNulls && nullValueBitmap.get(offset.getOffset()); - } + @Override + public boolean isNull() + { + return false; + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("columnar", ColumnarLongs.this); - inspector.visit("offset", offset); + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarLongs.this); + inspector.visit("offset", offset); + } } - } - return new HistoricalLongColumnSelector(); - } - - default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) - { - class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector - { - @Override - public long getLong() + return new HistoricalLongColumnSelector(); + } else { + class HistoricalLongColumnSelectorWithNulls implements LongColumnSelector, HistoricalColumnSelector { - return ColumnarLongs.this.get(offset.getOffset()); - } + @Override + public long getLong() + { + return ColumnarLongs.this.get(offset.getOffset()); + } - @Override - public double getDouble(int offset) - { - return ColumnarLongs.this.get(offset); - } + @Override + public double getDouble(int offset) + { + return ColumnarLongs.this.get(offset); + } - @Override - public boolean isNull() - { - return false; - } + @Override + public boolean isNull() + { + return nullValueBitmap.get(offset.getOffset()); + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("columnar", ColumnarLongs.this); - inspector.visit("offset", offset); + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("columnar", ColumnarLongs.this); + inspector.visit("offset", offset); + inspector.visit("nullValueBitmap", nullValueBitmap); + } } + return new HistoricalLongColumnSelectorWithNulls(); } - return new HistoricalLongColumnSelector(); } } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 63a49d04508e..97d96b377f4f 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -52,11 +52,11 @@ *

* byte 1: version (0x1) * byte 2 == 0x1 =>; allowReverseLookup - * bytes 3-6 =>; numBytesUsed, It will be -1 for null values. + * bytes 3-6 =>; numBytesUsed * bytes 7-10 =>; numElements * bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values * bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes - * for value + * for value. length of value will be -1 for null values. *

* V2 Storage Format * Meta, header and value files are separate and header file stored in native endian byte order. @@ -99,12 +99,16 @@ public Class getClazz() } /** - * numBytes will be {@link NULL_VALUE_SIZE_MARKER} for null values. + * numBytes will be -1 for null values. */ @Override @Nullable public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { + // numBytes will be -1 when the value is null and SQL compatible behavior for null handling is used. + // numBytes will be 0 when the value is empty string, + // For non-sql compliant, legacy null handling, nulls are stored as empty string + // and numBytes will be 0 for null values also. if (numBytes < 0) { return null; } @@ -374,7 +378,9 @@ public GenericIndexed.BufferIndexed singleThreaded() private T copyBufferAndGet(ByteBuffer valueBuffer, int startOffset, int endOffset) { ByteBuffer copyValueBuffer = valueBuffer.asReadOnlyBuffer(); - final int size = endOffset > startOffset ? endOffset - startOffset : copyValueBuffer.get(startOffset - Ints.BYTES); + final int size = endOffset > startOffset + ? endOffset - startOffset + : copyValueBuffer.get(startOffset - Integer.BYTES); copyValueBuffer.position(startOffset); // fromByteBuffer must not modify the buffer limit return strategy.fromByteBuffer(copyValueBuffer, size); diff --git a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java index 4125927f8937..dac4b3536fd1 100644 --- a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -27,6 +27,7 @@ import io.druid.collections.bitmap.WrappedImmutableRoaringBitmap; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; +import javax.annotation.Nullable; import java.nio.ByteBuffer; /** @@ -77,6 +78,7 @@ public Class getClazz() } @Override + @Nullable public ImmutableBitmap fromByteBuffer(ByteBuffer buffer, int numBytes) { if (numBytes == 0) { 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 da60fe9b9e51..684fd2f28fb5 100644 --- a/processing/src/main/java/io/druid/segment/filter/Filters.java +++ b/processing/src/main/java/io/druid/segment/filter/Filters.java @@ -133,7 +133,8 @@ public static ValueMatcher makeValueMatcher( columnSelectorFactory ); - return selector.getColumnSelectorStrategy().makeValueMatcher(selector.getSelector(), NullHandling.emptyToNullIfNeeded(value)); + return selector.getColumnSelectorStrategy() + .makeValueMatcher(selector.getSelector(), NullHandling.emptyToNullIfNeeded(value)); } /** From 182f6ee3fa4e5e0088d03253ba632dfd248e876c Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 6 Feb 2018 22:54:02 +0530 Subject: [PATCH 13/19] improve comment --- .../main/java/io/druid/segment/data/GenericIndexed.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 97d96b377f4f..7ae4b8f4896c 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -105,10 +105,11 @@ public Class getClazz() @Nullable public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { - // numBytes will be -1 when the value is null and SQL compatible behavior for null handling is used. - // numBytes will be 0 when the value is empty string, - // For non-sql compliant, legacy null handling, nulls are stored as empty string - // and numBytes will be 0 for null values also. + // When SQL Compatibility is ON + // 1. numBytes will be -1 for null value, return value will be null + // 2. numBytes will be 0 for empty string, return value will be empty string + // For Legacy null handling when nulls and empty string are considered same - + // numBytes will be 0 for both empty string and null and return value will be null for both. if (numBytes < 0) { return null; } From b685a92966557a37225aaf5d302f880b69b47094 Mon Sep 17 00:00:00 2001 From: Nishant Date: Fri, 9 Feb 2018 23:48:34 +0530 Subject: [PATCH 14/19] More review comments --- .../java/io/druid/math/expr/ExprEval.java | 6 +++++ .../RowBasedColumnSelectorFactory.java | 12 +++------ .../druid/segment/DoubleDimensionIndexer.java | 4 +-- .../druid/segment/FloatDimensionIndexer.java | 4 +-- .../druid/segment/LongDimensionIndexer.java | 4 +-- .../druid/segment/NilColumnValueSelector.java | 7 +---- .../druid/segment/StringDimensionIndexer.java | 3 +-- .../druid/segment/column/DoublesColumn.java | 15 ++++++++++- .../column/DoublesColumnWithNulls.java | 25 +++++++++++++++-- .../io/druid/segment/column/FloatsColumn.java | 15 ++++++++++- .../segment/column/FloatsColumnWithNulls.java | 24 +++++++++++++++-- .../io/druid/segment/column/LongsColumn.java | 15 ++++++++++- .../segment/column/LongsColumnWithNulls.java | 25 +++++++++++++++-- .../druid/segment/data/ColumnarDoubles.java | 27 ++++++++++--------- .../io/druid/segment/data/ColumnarFloats.java | 19 +++++++------ .../io/druid/segment/data/ColumnarLongs.java | 27 ++++++++++--------- .../io/druid/segment/data/GenericIndexed.java | 22 +++++---------- .../data/RoaringBitmapSerdeFactory.java | 3 --- .../segment/incremental/IncrementalIndex.java | 16 ++++++++--- .../serde/DoubleGenericColumnSupplier.java | 7 +---- .../serde/FloatGenericColumnSupplier.java | 7 +---- .../serde/LongGenericColumnSupplier.java | 7 +---- 22 files changed, 188 insertions(+), 106 deletions(-) diff --git a/common/src/main/java/io/druid/math/expr/ExprEval.java b/common/src/main/java/io/druid/math/expr/ExprEval.java index 2aa724c6081b..100a1adcc632 100644 --- a/common/src/main/java/io/druid/math/expr/ExprEval.java +++ b/common/src/main/java/io/druid/math/expr/ExprEval.java @@ -133,18 +133,21 @@ private NumericExprEval(Number value) @Override public final int asInt() { + assert NullHandling.replaceWithDefault() || !isNull(); return value.intValue(); } @Override public final long asLong() { + assert NullHandling.replaceWithDefault() || !isNull(); return value.longValue(); } @Override public final double asDouble() { + assert NullHandling.replaceWithDefault() || !isNull(); return value.doubleValue(); } } @@ -246,6 +249,7 @@ public final ExprType type() public final int asInt() { if (value == null) { + assert NullHandling.replaceWithDefault(); return 0; } @@ -258,6 +262,7 @@ public final long asLong() { // GuavaUtils.tryParseLong handles nulls, no need for special null handling here. final Long theLong = GuavaUtils.tryParseLong(value); + assert NullHandling.replaceWithDefault() || theLong != null; return theLong == null ? 0L : theLong; } @@ -265,6 +270,7 @@ public final long asLong() public final double asDouble() { if (value == null) { + assert NullHandling.replaceWithDefault(); return 0.0; } diff --git a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java index a5116f99221b..d2fad0ad8566 100644 --- a/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/io/druid/query/groupby/RowBasedColumnSelectorFactory.java @@ -425,9 +425,7 @@ public boolean isNull() public double getDouble() { Number metric = row.get().getMetric(columnName); - if (NullHandling.sqlCompatible() && metric == null) { - throw new IllegalStateException("Cannot return null value as double"); - } + assert NullHandling.replaceWithDefault() || metric != null; return DimensionHandlerUtils.nullToZero(metric).doubleValue(); } @@ -435,9 +433,7 @@ public double getDouble() public float getFloat() { Number metric = row.get().getMetric(columnName); - if (NullHandling.sqlCompatible() && metric == null) { - throw new IllegalStateException("Cannot return null value as float"); - } + assert NullHandling.replaceWithDefault() || metric != null; return DimensionHandlerUtils.nullToZero(metric).floatValue(); } @@ -445,9 +441,7 @@ public float getFloat() public long getLong() { Number metric = row.get().getMetric(columnName); - if (NullHandling.sqlCompatible() && metric == null) { - throw new IllegalStateException("Cannot return null value as long"); - } + assert NullHandling.replaceWithDefault() || metric != null; return DimensionHandlerUtils.nullToZero(metric).longValue(); } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java index 9653495d8968..a0a7f30e31f3 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionIndexer.java @@ -112,9 +112,7 @@ public double getDouble() final Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length || dims[dimIndex] == null) { - if (NullHandling.sqlCompatible()) { - throw new IllegalStateException("Cannot return null value as double"); - } + assert NullHandling.replaceWithDefault(); return 0.0; } return (Double) dims[dimIndex]; diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java index ecd986387174..695535cdac5a 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionIndexer.java @@ -113,9 +113,7 @@ public float getFloat() final Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length || dims[dimIndex] == null) { - if (NullHandling.sqlCompatible()) { - throw new IllegalStateException("Cannot return null value as float"); - } + assert NullHandling.replaceWithDefault(); return 0.0f; } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java index 2327ed6387b8..187c3fe2d291 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionIndexer.java @@ -113,9 +113,7 @@ public long getLong() final Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length || dims[dimIndex] == null) { - if (NullHandling.sqlCompatible()) { - throw new IllegalStateException("Cannot return null value as long"); - } + assert NullHandling.replaceWithDefault(); return 0; } diff --git a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java index 75fc17e1e6c8..a687d61cea95 100644 --- a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java @@ -91,7 +91,7 @@ public Class classOfObject() @Override public boolean isNull() { - return false; + return true; } @Override @@ -102,11 +102,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) private static class SqlCompatibleNilColumnValueSelector extends NilColumnValueSelector { - @Override - public boolean isNull() - { - return true; - } @Override public double getDouble() diff --git a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java index af977f32a03c..bb57b05414d4 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionIndexer.java @@ -66,7 +66,6 @@ private static String emptytoNullIfNeeded(Object o) } private static final int ABSENT_VALUE_ID = -1; - private static final int[] EMPTY_INT_ARRAY = IntArrays.EMPTY_ARRAY; private static class DimensionDictionary { @@ -222,7 +221,7 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(Object dimValues, boole List dimValuesList = (List) dimValues; if (dimValuesList.isEmpty()) { dimLookup.add(null); - encodedDimensionValues = EMPTY_INT_ARRAY; + encodedDimensionValues = IntArrays.EMPTY_ARRAY; } else if (dimValuesList.size() == 1) { encodedDimensionValues = new int[]{dimLookup.add(emptytoNullIfNeeded(dimValuesList.get(0)))}; } else { diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java index 2e175f4ae4c4..4d06634d908e 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.IndexIO; @@ -28,9 +29,21 @@ public class DoublesColumn implements GenericColumn { + /** + * Factory method to create DoublesColumn. + */ + public static DoublesColumn create(ColumnarDoubles column, ImmutableBitmap nullValueBitmap) + { + if (nullValueBitmap.isEmpty()) { + return new DoublesColumn(column); + } else { + return new DoublesColumnWithNulls(column, nullValueBitmap); + } + } + final ColumnarDoubles column; - public DoublesColumn(ColumnarDoubles columnarDoubles) + DoublesColumn(ColumnarDoubles columnarDoubles) { column = columnarDoubles; } diff --git a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java index 232c56d457fc..dca4cb57cc41 100644 --- a/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/DoublesColumnWithNulls.java @@ -28,11 +28,11 @@ /** * DoublesColumn with null values. */ -public class DoublesColumnWithNulls extends DoublesColumn +class DoublesColumnWithNulls extends DoublesColumn { private final ImmutableBitmap nullValueBitmap; - public DoublesColumnWithNulls(ColumnarDoubles columnarDoubles, ImmutableBitmap nullValueBitmap) + DoublesColumnWithNulls(ColumnarDoubles columnarDoubles, ImmutableBitmap nullValueBitmap) { super(columnarDoubles); this.nullValueBitmap = nullValueBitmap; @@ -56,4 +56,25 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) super.inspectRuntimeShape(inspector); inspector.visit("nullValueBitmap", nullValueBitmap); } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getFloatSingleValueRow(rowNum); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getLongSingleValueRow(rowNum); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getLongSingleValueRow(rowNum); + } } diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index 3aa08d1ca1fd..dff6c3385f6c 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.IndexIO; @@ -29,9 +30,21 @@ */ public class FloatsColumn implements GenericColumn { + /** + * Factory method to create FloatsColumn. + */ + public static FloatsColumn create(ColumnarFloats column, ImmutableBitmap nullValueBitmap) + { + if (nullValueBitmap.isEmpty()) { + return new FloatsColumn(column); + } else { + return new FloatsColumnWithNulls(column, nullValueBitmap); + } + } + final ColumnarFloats column; - public FloatsColumn(final ColumnarFloats column) + protected FloatsColumn(final ColumnarFloats column) { this.column = column; } diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java index 2e9b2476e7c0..016c911527fc 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumnWithNulls.java @@ -28,11 +28,11 @@ /** * FloatsColumn with null values. */ -public class FloatsColumnWithNulls extends FloatsColumn +class FloatsColumnWithNulls extends FloatsColumn { private final ImmutableBitmap nullValueBitmap; - public FloatsColumnWithNulls(ColumnarFloats columnarFloats, ImmutableBitmap nullValueBitmap) + FloatsColumnWithNulls(ColumnarFloats columnarFloats, ImmutableBitmap nullValueBitmap) { super(columnarFloats); this.nullValueBitmap = nullValueBitmap; @@ -51,4 +51,24 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) inspector.visit("nullValueBitmap", nullValueBitmap); } + @Override + public float getFloatSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getFloatSingleValueRow(rowNum); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getLongSingleValueRow(rowNum); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getLongSingleValueRow(rowNum); + } } diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 07f9a8ce0e36..936d242cac47 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -19,6 +19,7 @@ package io.druid.segment.column; +import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.IndexIO; @@ -29,9 +30,21 @@ */ public class LongsColumn implements GenericColumn { + /** + * Factory method to create LongsColumn. + */ + public static LongsColumn create(ColumnarLongs column, ImmutableBitmap nullValueBitmap) + { + if (nullValueBitmap.isEmpty()) { + return new LongsColumn(column); + } else { + return new LongsColumnWithNulls(column, nullValueBitmap); + } + } + final ColumnarLongs column; - public LongsColumn(final ColumnarLongs column) + protected LongsColumn(final ColumnarLongs column) { this.column = column; } diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java index db88b7c290db..6aad278b016d 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumnWithNulls.java @@ -28,11 +28,11 @@ /** * LongsColumn with null values. */ -public class LongsColumnWithNulls extends LongsColumn +class LongsColumnWithNulls extends LongsColumn { private final ImmutableBitmap nullValueBitmap; - public LongsColumnWithNulls(ColumnarLongs columnarLongs, ImmutableBitmap nullValueBitmap) + LongsColumnWithNulls(ColumnarLongs columnarLongs, ImmutableBitmap nullValueBitmap) { super(columnarLongs); this.nullValueBitmap = nullValueBitmap; @@ -50,4 +50,25 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) super.inspectRuntimeShape(inspector); inspector.visit("nullValueBitmap", nullValueBitmap); } + + @Override + public float getFloatSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getFloatSingleValueRow(rowNum); + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getLongSingleValueRow(rowNum); + } + + @Override + public double getDoubleSingleValueRow(int rowNum) + { + assert !isNull(rowNum); + return super.getLongSingleValueRow(rowNum); + } } diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java index 66e9f2b80d08..a17e1816becc 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarDoubles.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.DoubleColumnSelector; @@ -46,21 +47,21 @@ default ColumnValueSelector makeColumnValueSelector(ReadableOffset offse class HistoricalDoubleColumnSelector implements DoubleColumnSelector, HistoricalColumnSelector { @Override - public double getDouble() + public boolean isNull() { - return ColumnarDoubles.this.get(offset.getOffset()); + return false; } @Override - public double getDouble(int offset) + public double getDouble() { - return ColumnarDoubles.this.get(offset); + return ColumnarDoubles.this.get(offset.getOffset()); } @Override - public boolean isNull() + public double getDouble(int offset) { - return false; + return ColumnarDoubles.this.get(offset); } @Override @@ -75,21 +76,23 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) class HistoricalDoubleColumnSelectorWithNulls implements DoubleColumnSelector, HistoricalColumnSelector { @Override - public double getDouble() + public boolean isNull() { - return ColumnarDoubles.this.get(offset.getOffset()); + return nullValueBitmap.get(offset.getOffset()); } @Override - public double getDouble(int offset) + public double getDouble() { - return ColumnarDoubles.this.get(offset); + assert NullHandling.replaceWithDefault() || !isNull(); + return ColumnarDoubles.this.get(offset.getOffset()); } @Override - public boolean isNull() + public double getDouble(int offset) { - return nullValueBitmap.get(offset.getOffset()); + assert NullHandling.replaceWithDefault() || !nullValueBitmap.get(offset); + return ColumnarDoubles.this.get(offset); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java index 0a696d0cf780..2e3599697c92 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarFloats.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.FloatColumnSelector; @@ -48,15 +49,15 @@ default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset class HistoricalFloatColumnSelector implements FloatColumnSelector, HistoricalColumnSelector { @Override - public float getFloat() + public boolean isNull() { - return ColumnarFloats.this.get(offset.getOffset()); + return false; } @Override - public boolean isNull() + public float getFloat() { - return false; + return ColumnarFloats.this.get(offset.getOffset()); } @Override @@ -77,20 +78,22 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) class HistoricalFloatColumnSelectorwithNulls implements FloatColumnSelector, HistoricalColumnSelector { @Override - public float getFloat() + public boolean isNull() { - return ColumnarFloats.this.get(offset.getOffset()); + return nullValueBitmap.get(offset.getOffset()); } @Override - public boolean isNull() + public float getFloat() { - return nullValueBitmap.get(offset.getOffset()); + assert NullHandling.replaceWithDefault() || !isNull(); + return ColumnarFloats.this.get(offset.getOffset()); } @Override public double getDouble(int offset) { + assert NullHandling.replaceWithDefault() || !nullValueBitmap.get(offset); return ColumnarFloats.this.get(offset); } diff --git a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java index 5670035a6afd..ddcb60a9077b 100644 --- a/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java +++ b/processing/src/main/java/io/druid/segment/data/ColumnarLongs.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import io.druid.collections.bitmap.ImmutableBitmap; +import io.druid.common.config.NullHandling; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.ColumnValueSelector; import io.druid.segment.LongColumnSelector; @@ -48,21 +49,21 @@ default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, class HistoricalLongColumnSelector implements LongColumnSelector, HistoricalColumnSelector { @Override - public long getLong() + public boolean isNull() { - return ColumnarLongs.this.get(offset.getOffset()); + return false; } @Override - public double getDouble(int offset) + public long getLong() { - return ColumnarLongs.this.get(offset); + return ColumnarLongs.this.get(offset.getOffset()); } @Override - public boolean isNull() + public double getDouble(int offset) { - return false; + return ColumnarLongs.this.get(offset); } @Override @@ -77,21 +78,23 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) class HistoricalLongColumnSelectorWithNulls implements LongColumnSelector, HistoricalColumnSelector { @Override - public long getLong() + public boolean isNull() { - return ColumnarLongs.this.get(offset.getOffset()); + return nullValueBitmap.get(offset.getOffset()); } @Override - public double getDouble(int offset) + public long getLong() { - return ColumnarLongs.this.get(offset); + assert NullHandling.replaceWithDefault() || !isNull(); + return ColumnarLongs.this.get(offset.getOffset()); } @Override - public boolean isNull() + public double getDouble(int offset) { - return nullValueBitmap.get(offset.getOffset()); + assert NullHandling.replaceWithDefault() || !nullValueBitmap.get(offset); + return ColumnarLongs.this.get(offset); } @Override diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 7ae4b8f4896c..882cb0f7ae94 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -56,7 +56,9 @@ * bytes 7-10 =>; numElements * bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values * bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes - * for value. length of value will be -1 for null values. + * for value. This field has no meaning, if next offset is strictly greater than the current offset, + * and if they are the same, -1 at this field means null, and 0 at this field means some object + * (potentially non-null - e. g. in the string case, that is serialized as an empty sequence of bytes). *

* V2 Storage Format * Meta, header and value files are separate and header file stored in native endian byte order. @@ -98,21 +100,10 @@ public Class getClazz() return String.class; } - /** - * numBytes will be -1 for null values. - */ @Override @Nullable public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { - // When SQL Compatibility is ON - // 1. numBytes will be -1 for null value, return value will be null - // 2. numBytes will be 0 for empty string, return value will be empty string - // For Legacy null handling when nulls and empty string are considered same - - // numBytes will be 0 for both empty string and null and return value will be null for both. - if (numBytes < 0) { - return null; - } return NullHandling.emptyToNullIfNeeded(StringUtils.fromUtf8Nullable(buffer, numBytes)); } @@ -420,9 +411,10 @@ public int size() T bufferedIndexedGet(ByteBuffer copyValueBuffer, int startOffset, int endOffset) { - final int size = endOffset > startOffset - ? endOffset - startOffset - : copyValueBuffer.get(startOffset - Ints.BYTES); + int size = endOffset - startOffset; + if (size == 0 && copyValueBuffer.get(startOffset - Ints.BYTES) == NULL_VALUE_SIZE_MARKER) { + return null; + } lastReadSize = size; // ObjectStrategy.fromByteBuffer() is allowed to reset the limit of the buffer. So if the limit is changed, diff --git a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java index dac4b3536fd1..7577f2f5bdaf 100644 --- a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -81,9 +81,6 @@ public Class getClazz() @Nullable public ImmutableBitmap fromByteBuffer(ByteBuffer buffer, int numBytes) { - if (numBytes == 0) { - return null; - } buffer.limit(buffer.position() + numBytes); return new WrappedImmutableRoaringBitmap(new ImmutableRoaringBitmap(buffer)); } 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 a78a780a123d..669fa9897f4c 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndex.java @@ -30,6 +30,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.collections.NonBlockingPool; +import io.druid.common.config.NullHandling; import io.druid.common.guava.GuavaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedRow; @@ -160,19 +161,25 @@ public boolean isNull() @Override public long getLong() { - return DimensionHandlerUtils.nullToZero(in.get().getMetric(column)).longValue(); + Number metric = in.get().getMetric(column); + assert NullHandling.replaceWithDefault() || metric != null; + return DimensionHandlerUtils.nullToZero(metric).longValue(); } @Override public float getFloat() { - return DimensionHandlerUtils.nullToZero(in.get().getMetric(column)).floatValue(); + Number metric = in.get().getMetric(column); + assert NullHandling.replaceWithDefault() || metric != null; + return DimensionHandlerUtils.nullToZero(metric).floatValue(); } @Override public double getDouble() { - return DimensionHandlerUtils.nullToZero(in.get().getMetric(column)).doubleValue(); + Number metric = in.get().getMetric(column); + assert NullHandling.replaceWithDefault() || metric != null; + return DimensionHandlerUtils.nullToZero(metric).doubleValue(); } @Override @@ -1401,6 +1408,7 @@ public LongMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) @Override public long getLong() { + assert NullHandling.replaceWithDefault() || !isNull(); return getMetricLongValue(currEntry.getValue(), metricIndex); } @@ -1468,6 +1476,7 @@ public FloatMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) @Override public float getFloat() { + assert NullHandling.replaceWithDefault() || !isNull(); return getMetricFloatValue(currEntry.getValue(), metricIndex); } @@ -1498,6 +1507,7 @@ public DoubleMetricColumnSelector(TimeAndDimsHolder currEntry, int metricIndex) @Override public double getDouble() { + assert NullHandling.replaceWithDefault() || !isNull(); return getMetricDoubleValue(currEntry.getValue(), metricIndex); } diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java index f0e36e132f38..046fad51b848 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnSupplier.java @@ -22,7 +22,6 @@ import com.google.common.base.Supplier; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.DoublesColumn; -import io.druid.segment.column.DoublesColumnWithNulls; import io.druid.segment.column.GenericColumn; import io.druid.segment.data.ColumnarDoubles; @@ -44,10 +43,6 @@ public DoubleGenericColumnSupplier( @Override public GenericColumn get() { - if (nullValueBitmap.isEmpty()) { - return new DoublesColumn(column.get()); - } else { - return new DoublesColumnWithNulls(column.get(), nullValueBitmap); - } + return DoublesColumn.create(column.get(), nullValueBitmap); } } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java index 98973f72fc25..9af3adf5971a 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java @@ -22,7 +22,6 @@ import com.google.common.base.Supplier; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.FloatsColumn; -import io.druid.segment.column.FloatsColumnWithNulls; import io.druid.segment.column.GenericColumn; import io.druid.segment.data.CompressedColumnarFloatsSupplier; @@ -45,10 +44,6 @@ public FloatGenericColumnSupplier( @Override public GenericColumn get() { - if (nullValueBitmap.isEmpty()) { - return new FloatsColumn(column.get()); - } else { - return new FloatsColumnWithNulls(column.get(), nullValueBitmap); - } + return FloatsColumn.create(column.get(), nullValueBitmap); } } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java index 1b74395c6c88..c70d4e4dc15f 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnSupplier.java @@ -23,7 +23,6 @@ import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.LongsColumn; -import io.druid.segment.column.LongsColumnWithNulls; import io.druid.segment.data.CompressedColumnarLongsSupplier; /** @@ -42,10 +41,6 @@ public LongGenericColumnSupplier(CompressedColumnarLongsSupplier column, Immutab @Override public GenericColumn get() { - if (nullValueBitmap.isEmpty()) { - return new LongsColumn(column.get()); - } else { - return new LongsColumnWithNulls(column.get(), nullValueBitmap); - } + return LongsColumn.create(column.get(), nullValueBitmap); } } From 54b91f4cd35a3b917b68c82ce47a52ed27f4eb58 Mon Sep 17 00:00:00 2001 From: Nishant Date: Sat, 10 Feb 2018 00:01:29 +0530 Subject: [PATCH 15/19] fix checkstyle --- .../src/main/java/io/druid/segment/data/GenericIndexed.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 882cb0f7ae94..9612f435f2ed 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -412,7 +412,7 @@ public int size() T bufferedIndexedGet(ByteBuffer copyValueBuffer, int startOffset, int endOffset) { int size = endOffset - startOffset; - if (size == 0 && copyValueBuffer.get(startOffset - Ints.BYTES) == NULL_VALUE_SIZE_MARKER) { + if (size == 0 && copyValueBuffer.get(startOffset - Integer.BYTES) == NULL_VALUE_SIZE_MARKER) { return null; } lastReadSize = size; @@ -562,7 +562,7 @@ public T get(final int index) final int endOffset; if (index == 0) { - startOffset = Ints.BYTES; + startOffset = Integer.BYTES; endOffset = headerBuffer.getInt(0); } else { int headerPosition = (index - 1) * Integer.BYTES; From 63252abcc28ec925b908f4570b01f7da01f01a62 Mon Sep 17 00:00:00 2001 From: Nishant Date: Wed, 14 Feb 2018 00:42:11 +0530 Subject: [PATCH 16/19] more review comments --- .../java/io/druid/math/expr/ExprEval.java | 5 ++--- .../io/druid/segment/MapVirtualColumn.java | 4 ++++ .../query/aggregation/BufferAggregator.java | 2 +- .../segment/ConstantColumnValueSelector.java | 1 + .../druid/segment/DimensionSelectorUtils.java | 4 ++-- .../segment/DoubleColumnSerializerV2.java | 3 ++- .../segment/FloatColumnSerializerV2.java | 3 ++- .../druid/segment/LongColumnSerializerV2.java | 3 ++- .../druid/segment/NilColumnValueSelector.java | 10 ++++++++- .../io/druid/segment/column/FloatsColumn.java | 2 +- .../io/druid/segment/column/LongsColumn.java | 2 +- .../io/druid/segment/data/GenericIndexed.java | 22 +++++++++++++------ .../segment/data/GenericIndexedWriter.java | 5 +++-- .../segment/virtual/ExpressionSelectors.java | 7 ++++++ 14 files changed, 52 insertions(+), 21 deletions(-) diff --git a/common/src/main/java/io/druid/math/expr/ExprEval.java b/common/src/main/java/io/druid/math/expr/ExprEval.java index 100a1adcc632..c5f0f6a1c379 100644 --- a/common/src/main/java/io/druid/math/expr/ExprEval.java +++ b/common/src/main/java/io/druid/math/expr/ExprEval.java @@ -133,21 +133,18 @@ private NumericExprEval(Number value) @Override public final int asInt() { - assert NullHandling.replaceWithDefault() || !isNull(); return value.intValue(); } @Override public final long asLong() { - assert NullHandling.replaceWithDefault() || !isNull(); return value.longValue(); } @Override public final double asDouble() { - assert NullHandling.replaceWithDefault() || !isNull(); return value.doubleValue(); } } @@ -254,6 +251,7 @@ public final int asInt() } final Integer theInt = Ints.tryParse(value); + assert NullHandling.replaceWithDefault() || theInt != null; return theInt == null ? 0 : theInt; } @@ -275,6 +273,7 @@ public final double asDouble() } final Double theDouble = Doubles.tryParse(value); + assert NullHandling.replaceWithDefault() || theDouble != null; return theDouble == null ? 0.0 : theDouble; } diff --git a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java index 64889315792c..4598e07dcaba 100644 --- a/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java +++ b/extensions-contrib/virtual-columns/src/main/java/io/druid/segment/MapVirtualColumn.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.StringUtils; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; @@ -177,18 +178,21 @@ private MapVirtualColumnValueSelector(DimensionSelector keySelector, DimensionSe @Override public double getDouble() { + assert NullHandling.replaceWithDefault(); return 0.0; } @Override public float getFloat() { + assert NullHandling.replaceWithDefault(); return 0.0f; } @Override public long getLong() { + assert NullHandling.replaceWithDefault(); return 0L; } 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 60e8d841d63e..4e97cc79c8b3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/BufferAggregator.java @@ -181,7 +181,7 @@ default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, By /** * returns true if aggregator's output type is primitive long/double/float and aggregated value is null, * but when aggregated output type is Object, this method always returns false, - * and users are advised to check nullability for the object returned by {@link #get(ByteBuffer, int)} ()} + * and users are advised to check nullability for the object returned by {@link BufferAggregator#get(ByteBuffer, int)} * method. * The default implementation always return false to enable smooth backward compatibility, * re-implement if your aggregator is nullable. diff --git a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java index b28755cedbe0..aa089e36e0ed 100644 --- a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java @@ -90,6 +90,7 @@ public void inspectRuntimeShape(final RuntimeShapeInspector inspector) @Override public boolean isNull() { + // return false always as the primitive values for this selector can never be null. return false; } } diff --git a/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java b/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java index f83be70241b7..1fbaf72c76ac 100644 --- a/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java +++ b/processing/src/main/java/io/druid/segment/DimensionSelectorUtils.java @@ -21,7 +21,7 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; -import com.google.common.base.Strings; +import io.druid.common.config.NullHandling; import io.druid.java.util.common.IAE; import io.druid.query.extraction.ExtractionFn; import io.druid.query.filter.ValueMatcher; @@ -252,7 +252,7 @@ public static BitSet makePredicateMatchingSet(DimensionSelector selector, Predic public static DimensionSelector constantSelector(@Nullable final String value) { - if (Strings.isNullOrEmpty(value)) { + if (NullHandling.isNullOrEquivalent(value)) { return NullDimensionSelector.instance(); } else { return new ConstantDimensionSelector(value); diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java index b0a95edcfba1..07828f996cba 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializerV2.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; import io.druid.common.utils.SerializerUtils; @@ -129,7 +130,7 @@ public long getSerializedSize() throws IOException @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - SerializerUtils.writeInt(channel, (int) writer.getSerializedSize()); + SerializerUtils.writeInt(channel, Ints.checkedCast(writer.getSerializedSize())); writer.writeTo(channel, smoosher); if (!nullRowsBitmap.isEmpty()) { nullValueBitmapWriter.writeTo(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java index 9b872a89f9c4..1ec6f377e713 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializerV2.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; import io.druid.common.utils.SerializerUtils; @@ -130,7 +131,7 @@ public long getSerializedSize() throws IOException @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - SerializerUtils.writeInt(channel, (int) writer.getSerializedSize()); + SerializerUtils.writeInt(channel, Ints.checkedCast(writer.getSerializedSize())); writer.writeTo(channel, smoosher); if (!nullRowsBitmap.isEmpty()) { nullValueBitmapWriter.writeTo(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java index 37004a7e2cdf..d9a71e015e4a 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializerV2.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; import io.druid.common.utils.SerializerUtils; @@ -135,7 +136,7 @@ public long getSerializedSize() throws IOException @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - SerializerUtils.writeInt(channel, (int) writer.getSerializedSize()); + SerializerUtils.writeInt(channel, Ints.checkedCast(writer.getSerializedSize())); writer.writeTo(channel, smoosher); if (!nullRowsBitmap.isEmpty()) { nullValueBitmapWriter.writeTo(channel, smoosher); diff --git a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java index a687d61cea95..b389c9bd985d 100644 --- a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java @@ -102,19 +102,27 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) private static class SqlCompatibleNilColumnValueSelector extends NilColumnValueSelector { - + /** + * Always throws IllegalStateException. + */ @Override public double getDouble() { throw new IllegalStateException("Cannot return null value as double"); } + /** + * Always throws IllegalStateException. + */ @Override public float getFloat() { throw new IllegalStateException("Cannot return null value as float"); } + /** + * Always throws IllegalStateException. + */ @Override public long getLong() { diff --git a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java index dff6c3385f6c..cdf18d38a4b3 100644 --- a/processing/src/main/java/io/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/FloatsColumn.java @@ -44,7 +44,7 @@ public static FloatsColumn create(ColumnarFloats column, ImmutableBitmap nullVal final ColumnarFloats column; - protected FloatsColumn(final ColumnarFloats column) + FloatsColumn(final ColumnarFloats column) { this.column = column; } diff --git a/processing/src/main/java/io/druid/segment/column/LongsColumn.java b/processing/src/main/java/io/druid/segment/column/LongsColumn.java index 936d242cac47..0329c8aae54c 100644 --- a/processing/src/main/java/io/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/io/druid/segment/column/LongsColumn.java @@ -44,7 +44,7 @@ public static LongsColumn create(ColumnarLongs column, ImmutableBitmap nullValue final ColumnarLongs column; - protected LongsColumn(final ColumnarLongs column) + LongsColumn(final ColumnarLongs column) { this.column = column; } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index 9612f435f2ed..6dbe07f30f29 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -35,6 +35,7 @@ import io.druid.segment.serde.MetaSerdeHelper; import io.druid.segment.serde.Serializer; import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; + import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; @@ -56,7 +57,7 @@ * bytes 7-10 =>; numElements * bytes 10-((numElements * 4) + 10): integers representing *end* offsets of byte serialized values * bytes ((numElements * 4) + 10)-(numBytesUsed + 2): 4-byte integer representing length of value, followed by bytes - * for value. This field has no meaning, if next offset is strictly greater than the current offset, + * for value. Length of value stored has no meaning, if next offset is strictly greater than the current offset, * and if they are the same, -1 at this field means null, and 0 at this field means some object * (potentially non-null - e. g. in the string case, that is serialized as an empty sequence of bytes). *

@@ -101,13 +102,13 @@ public Class getClazz() } @Override - @Nullable public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) { - return NullHandling.emptyToNullIfNeeded(StringUtils.fromUtf8Nullable(buffer, numBytes)); + return StringUtils.fromUtf8(buffer, numBytes); } @Override + @Nullable public byte[] toBytes(String val) { return StringUtils.toUtf8Nullable(NullHandling.nullToEmptyIfNeeded(val)); @@ -370,9 +371,13 @@ public GenericIndexed.BufferIndexed singleThreaded() private T copyBufferAndGet(ByteBuffer valueBuffer, int startOffset, int endOffset) { ByteBuffer copyValueBuffer = valueBuffer.asReadOnlyBuffer(); - final int size = endOffset > startOffset - ? endOffset - startOffset - : copyValueBuffer.get(startOffset - Integer.BYTES); + int size = endOffset - startOffset; + // When size is 0 and SQL compatibility is enabled also check for null marker before returning null. + // When SQL compatibility is not enabled return null for both null as well as empty string case. + if (size == 0 && (NullHandling.replaceWithDefault() + || copyValueBuffer.get(startOffset - Integer.BYTES) == NULL_VALUE_SIZE_MARKER)) { + return null; + } copyValueBuffer.position(startOffset); // fromByteBuffer must not modify the buffer limit return strategy.fromByteBuffer(copyValueBuffer, size); @@ -412,7 +417,10 @@ public int size() T bufferedIndexedGet(ByteBuffer copyValueBuffer, int startOffset, int endOffset) { int size = endOffset - startOffset; - if (size == 0 && copyValueBuffer.get(startOffset - Integer.BYTES) == NULL_VALUE_SIZE_MARKER) { + // When size is 0 and SQL compatibility is enabled also check for null marker before returning null. + // When SQL compatibility is not enabled return null for both null as well as empty string case. + if (size == 0 && (NullHandling.replaceWithDefault() + || copyValueBuffer.get(startOffset - Integer.BYTES) == NULL_VALUE_SIZE_MARKER)) { return null; } lastReadSize = size; diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index 20e6374a328e..1cdc4b38111d 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -214,8 +214,9 @@ public void write(T objectToWrite) throws IOException } ++numWritten; - - valuesOut.writeInt(objectToWrite == null ? -1 : 0); + // for compatibility with the format (see GenericIndexed javadoc for description of the format), + // this field is used to store nullness marker, but in a better format this info can take 1 bit. + valuesOut.writeInt(objectToWrite == null ? GenericIndexed.NULL_VALUE_SIZE_MARKER : 0); if (objectToWrite != null) { strategy.writeTo(objectToWrite, valuesOut); } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java index 5f8652643aa5..f43e77a634d9 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java @@ -39,6 +39,7 @@ import io.druid.segment.DimensionSelector; import io.druid.segment.DimensionSelectorUtils; import io.druid.segment.NilColumnValueSelector; +import io.druid.segment.NullDimensionSelector; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ValueType; @@ -157,6 +158,9 @@ public static ColumnValueSelector makeExprEvalSelector( if (bindings.equals(ExprUtils.nilBindings())) { // Optimization for constant expressions. final ExprEval eval = expression.eval(bindings); + if (NullHandling.sqlCompatible() && eval.isNull()) { + return NilColumnValueSelector.instance(); + } return new ConstantColumnValueSelector<>( eval.asLong(), (float) eval.asDouble(), @@ -198,6 +202,9 @@ public static DimensionSelector makeDimensionSelector( if (baseSelector instanceof ConstantColumnValueSelector) { // Optimization for dimension selectors on constants. return DimensionSelectorUtils.constantSelector(baseSelector.getObject().asString(), extractionFn); + } else if (baseSelector instanceof NilColumnValueSelector) { + // Optimization for null dimension selector. + return NullDimensionSelector.instance(); } else if (extractionFn == null) { class DefaultExpressionDimensionSelector extends BaseSingleValueDimensionSelector { From c75ff2d07e928e0dc78295c88df03b249e3a2bf8 Mon Sep 17 00:00:00 2001 From: Nishant Date: Mon, 19 Feb 2018 19:56:24 +0530 Subject: [PATCH 17/19] review comments. fix javadoc links remove Nullable from ConstantColumnValueSelector --- .../io/druid/common/config/NullHandling.java | 2 +- .../segment/ConstantColumnValueSelector.java | 6 +- .../druid/segment/NilColumnValueSelector.java | 23 +++-- .../serde/DoubleGenericColumnPartSerdeV2.java | 89 ++++++++++--------- .../serde/FloatGenericColumnPartSerdeV2.java | 4 +- .../serde/LongGenericColumnPartSerdeV2.java | 3 +- .../ExpressionColumnValueSelector.java | 2 +- 7 files changed, 61 insertions(+), 68 deletions(-) diff --git a/common/src/main/java/io/druid/common/config/NullHandling.java b/common/src/main/java/io/druid/common/config/NullHandling.java index 20a5d7cf52ef..762f8213214f 100644 --- a/common/src/main/java/io/druid/common/config/NullHandling.java +++ b/common/src/main/java/io/druid/common/config/NullHandling.java @@ -31,7 +31,7 @@ */ public class NullHandling { - private static String NULL_HANDLING_CONFIG_STRING = "druid.generic.useDefaultValueForNull"; + public static String NULL_HANDLING_CONFIG_STRING = "druid.generic.useDefaultValueForNull"; /** * use these values to ensure that {@link NullHandling#defaultDoubleValue()}, diff --git a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java index aa089e36e0ed..88519008f1cf 100644 --- a/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/ConstantColumnValueSelector.java @@ -22,15 +22,12 @@ import com.google.common.base.Preconditions; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import javax.annotation.Nullable; - public class ConstantColumnValueSelector implements ColumnValueSelector { private long longValue; private float floatValue; private double doubleValue; - @Nullable private T objectValue; private Class objectClass; @@ -39,7 +36,7 @@ public ConstantColumnValueSelector( final long longValue, final float floatValue, final double doubleValue, - @Nullable final T objectValue, + final T objectValue, final Class objectClass ) { @@ -68,7 +65,6 @@ public long getLong() return longValue; } - @Nullable @Override public T getObject() { diff --git a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java index b389c9bd985d..73dacbbbeb40 100644 --- a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java @@ -43,7 +43,9 @@ public static NilColumnValueSelector instance() private NilColumnValueSelector() {} /** - * Always returns 0.0. + * always returns 0, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, + * or always throws an exception, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is + * set to false. */ @Override public double getDouble() @@ -52,7 +54,9 @@ public double getDouble() } /** - * Always returns 0.0f. + * always returns 0.0f, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, + * or always throws an exception, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is + * set to false. */ @Override public float getFloat() @@ -61,7 +65,9 @@ public float getFloat() } /** - * Always returns 0L. + * always returns 0L, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, + * or always throws an exception, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is + * set to false. */ @Override public long getLong() @@ -102,27 +108,18 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) private static class SqlCompatibleNilColumnValueSelector extends NilColumnValueSelector { - /** - * Always throws IllegalStateException. - */ @Override public double getDouble() { throw new IllegalStateException("Cannot return null value as double"); } - /** - * Always throws IllegalStateException. - */ @Override public float getFloat() { throw new IllegalStateException("Cannot return null value as float"); } - - /** - * Always throws IllegalStateException. - */ + @Override public long getLong() { diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java index b371dd0dee33..20aa30266b09 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java @@ -36,36 +36,27 @@ import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; +/** + */ public class DoubleGenericColumnPartSerdeV2 implements ColumnPartSerde { - private final ByteOrder byteOrder; - private Serializer serialize; - private final BitmapSerdeFactory bitmapSerdeFactory; - @JsonCreator public static DoubleGenericColumnPartSerdeV2 getDoubleGenericColumnPartSerde( @JsonProperty("byteOrder") ByteOrder byteOrder, @Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory ) { - return new DoubleGenericColumnPartSerdeV2(byteOrder, - bitmapSerdeFactory != null - ? bitmapSerdeFactory - : new BitmapSerde.LegacyBitmapSerdeFactory(), null + return new DoubleGenericColumnPartSerdeV2( + byteOrder, + bitmapSerdeFactory != null + ? bitmapSerdeFactory + : new BitmapSerde.LegacyBitmapSerdeFactory(), null ); } - @JsonProperty - public ByteOrder getByteOrder() - { - return byteOrder; - } - - @JsonProperty - public BitmapSerdeFactory getBitmapSerdeFactory() - { - return bitmapSerdeFactory; - } + private final ByteOrder byteOrder; + private Serializer serialize; + private final BitmapSerdeFactory bitmapSerdeFactory; public DoubleGenericColumnPartSerdeV2( ByteOrder byteOrder, @@ -78,34 +69,16 @@ public DoubleGenericColumnPartSerdeV2( this.serialize = serialize; } - @Override - public Serializer getSerializer() + @JsonProperty + public ByteOrder getByteOrder() { - return serialize; + return byteOrder; } - @Override - public Deserializer getDeserializer() + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() { - return (buffer, builder, columnConfig) -> { - int offset = buffer.getInt(); - int initialPos = buffer.position(); - final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( - buffer, - byteOrder - ); - - buffer.position(initialPos + offset); - final ImmutableBitmap bitmap; - if (buffer.hasRemaining()) { - bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); - } else { - bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); - } - builder.setType(ValueType.DOUBLE) - .setHasMultipleValues(false) - .setGenericColumn(new DoubleGenericColumnSupplier(column, bitmap)); - }; + return bitmapSerdeFactory; } public static SerializerBuilder serializerBuilder() @@ -159,4 +132,34 @@ public void writeTo(WritableByteChannel channel, FileSmoosher fileSmoosher) thro ); } } + + @Override + public Serializer getSerializer() + { + return serialize; + } + + @Override + public Deserializer getDeserializer() + { + return (buffer, builder, columnConfig) -> { + int offset = buffer.getInt(); + int initialPos = buffer.position(); + final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( + buffer, + byteOrder + ); + + buffer.position(initialPos + offset); + final ImmutableBitmap bitmap; + if (buffer.hasRemaining()) { + bitmap = bitmapSerdeFactory.getObjectStrategy().fromByteBufferWithSize(buffer); + } else { + bitmap = bitmapSerdeFactory.getBitmapFactory().makeEmptyImmutableBitmap(); + } + builder.setType(ValueType.DOUBLE) + .setHasMultipleValues(false) + .setGenericColumn(new DoubleGenericColumnSupplier(column, bitmap)); + }; + } } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java index 604c5abb5038..d1fc5a496963 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerdeV2.java @@ -51,8 +51,8 @@ public static FloatGenericColumnPartSerdeV2 createDeserializer( } private final ByteOrder byteOrder; - private final BitmapSerdeFactory bitmapSerdeFactory; private Serializer serializer; + private final BitmapSerdeFactory bitmapSerdeFactory; private FloatGenericColumnPartSerdeV2( ByteOrder byteOrder, @@ -126,7 +126,6 @@ public void writeTo(WritableByteChannel channel, FileSmoosher fileSmoosher) thro } ); } - } @Override @@ -155,7 +154,6 @@ public Deserializer getDeserializer() builder.setType(ValueType.FLOAT) .setHasMultipleValues(false) .setGenericColumn(new FloatGenericColumnSupplier(column, bitmap)); - }; } } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java index 0d95a86ded72..a49c6e56b966 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerdeV2.java @@ -51,8 +51,8 @@ public static LongGenericColumnPartSerdeV2 createDeserializer( } private final ByteOrder byteOrder; - private final BitmapSerdeFactory bitmapSerdeFactory; private Serializer serializer; + private final BitmapSerdeFactory bitmapSerdeFactory; private LongGenericColumnPartSerdeV2( ByteOrder byteOrder, @@ -153,7 +153,6 @@ public Deserializer getDeserializer() builder.setType(ValueType.LONG) .setHasMultipleValues(false) .setGenericColumn(new LongGenericColumnSupplier(column, bitmap)); - }; } } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java index 0fd23ebffcdc..3c29b45573de 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java @@ -79,6 +79,6 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public boolean isNull() { - return getObject() == null; + return getObject().isNull(); } } From 4def13dad1d93fa74b65327784b607e05e4ed19d Mon Sep 17 00:00:00 2001 From: Nishant Date: Tue, 20 Feb 2018 22:27:44 +0530 Subject: [PATCH 18/19] review comments. --- .../serde/DoubleGenericColumnPartSerdeV2.java | 5 ++--- .../virtual/ExpressionColumnValueSelector.java | 3 +++ .../segment/virtual/ExpressionSelectors.java | 15 +++++++++------ ...InputCachingExpressionColumnValueSelector.java | 3 +++ ...InputCachingExpressionColumnValueSelector.java | 3 +++ 5 files changed, 20 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java index 20aa30266b09..f68bec1f108b 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerdeV2.java @@ -48,9 +48,8 @@ public static DoubleGenericColumnPartSerdeV2 getDoubleGenericColumnPartSerde( { return new DoubleGenericColumnPartSerdeV2( byteOrder, - bitmapSerdeFactory != null - ? bitmapSerdeFactory - : new BitmapSerde.LegacyBitmapSerdeFactory(), null + bitmapSerdeFactory != null ? bitmapSerdeFactory : new BitmapSerde.LegacyBitmapSerdeFactory(), + null ); } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java index 3c29b45573de..18c64d1bfcdd 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionColumnValueSelector.java @@ -41,18 +41,21 @@ public ExpressionColumnValueSelector(Expr expression, Expr.ObjectBinding binding @Override public double getDouble() { + // No Assert for null handling as ExprEval already have it. return getObject().asDouble(); } @Override public float getFloat() { + // No Assert for null handling as ExprEval already have it. return (float) getObject().asDouble(); } @Override public long getLong() { + // No Assert for null handling as ExprEval already have it. return getObject().asLong(); } diff --git a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java index f43e77a634d9..3aae3b797d7c 100644 --- a/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/io/druid/segment/virtual/ExpressionSelectors.java @@ -75,27 +75,30 @@ public static ColumnValueSelector makeColumnValueSelector( @Override public double getDouble() { + // No Assert for null handling as baseSelector already have it. return baseSelector.getDouble(); } - @Override - public boolean isNull() - { - return baseSelector.getObject().isNull(); - } - @Override public float getFloat() { + // No Assert for null handling as baseSelector already have it. return baseSelector.getFloat(); } @Override public long getLong() { + // No Assert for null handling as baseSelector already have it. return baseSelector.getLong(); } + @Override + public boolean isNull() + { + return baseSelector.isNull(); + } + @Nullable @Override public Object getObject() diff --git a/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java index f2051ae9f64b..2a0be1ccf767 100644 --- a/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/SingleLongInputCachingExpressionColumnValueSelector.java @@ -80,6 +80,7 @@ public void inspectRuntimeShape(final RuntimeShapeInspector inspector) @Override public double getDouble() { + // No Assert for null handling as delegate selector already have it. final long currentInput = selector.getLong(); if (lastInput == currentInput && validity == Validity.DOUBLE) { @@ -96,12 +97,14 @@ public double getDouble() @Override public float getFloat() { + // No Assert for null handling as delegate selector already have it. return (float) getDouble(); } @Override public long getLong() { + // No Assert for null handling as delegate selector already have it. final long currentInput = selector.getLong(); if (lastInput == currentInput && validity == Validity.LONG) { diff --git a/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java b/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java index 71557231cfe8..2b2db902d9ea 100644 --- a/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/virtual/SingleStringInputCachingExpressionColumnValueSelector.java @@ -84,18 +84,21 @@ public void inspectRuntimeShape(final RuntimeShapeInspector inspector) @Override public double getDouble() { + // No Assert for null handling as ExprEval already have it. return eval().asDouble(); } @Override public float getFloat() { + // No Assert for null handling as ExprEval already have it. return (float) eval().asDouble(); } @Override public long getLong() { + // No Assert for null handling as ExprEval already have it. return eval().asLong(); } From 03b97571e23bd0e38e0f69493877a1b986184a02 Mon Sep 17 00:00:00 2001 From: Nishant Date: Wed, 21 Feb 2018 02:14:58 +0530 Subject: [PATCH 19/19] satisfy teamcity inspections --- .../io/druid/segment/NilColumnValueSelector.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java index 73dacbbbeb40..1d219c6dcdc3 100644 --- a/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java +++ b/processing/src/main/java/io/druid/segment/NilColumnValueSelector.java @@ -43,8 +43,8 @@ public static NilColumnValueSelector instance() private NilColumnValueSelector() {} /** - * always returns 0, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, - * or always throws an exception, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is + * always returns 0, if {@link NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, + * or always throws an exception, if {@link NullHandling#NULL_HANDLING_CONFIG_STRING} is * set to false. */ @Override @@ -54,8 +54,8 @@ public double getDouble() } /** - * always returns 0.0f, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, - * or always throws an exception, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is + * always returns 0.0f, if {@link NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, + * or always throws an exception, if {@link NullHandling#NULL_HANDLING_CONFIG_STRING} is * set to false. */ @Override @@ -65,8 +65,8 @@ public float getFloat() } /** - * always returns 0L, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, - * or always throws an exception, if {@link io.druid.common.config.NullHandling#NULL_HANDLING_CONFIG_STRING} is + * always returns 0L, if {@link NullHandling#NULL_HANDLING_CONFIG_STRING} is set to true, + * or always throws an exception, if {@link NullHandling#NULL_HANDLING_CONFIG_STRING} is * set to false. */ @Override