mirror of https://github.com/apache/druid.git
Part 2 of changes for SQL Compatible Null Handling (#5958)
* Part 2 of changes for SQL Compatible Null Handling * Review comments - break lines longer than 120 characters * review comments * review comments * fix license * fix test failure * fix CalciteQueryTest failure * Null Handling - Review comments * review comments * review comments * fix checkstyle * fix checkstyle * remove unrelated change * fix test failure * fix failing test * fix travis failures * Make StringLast and StringFirst aggregators nullable and fix travis failures
This commit is contained in:
parent
b9c445c780
commit
75c8a87ce1
30
.travis.yml
30
.travis.yml
|
@ -33,6 +33,15 @@ matrix:
|
|||
- unset _JAVA_OPTIONS
|
||||
script: echo "MAVEN_OPTS='-Xmx512m'" > ~/.mavenrc && mvn test -B -Pparallel-test -Dmaven.fork.count=2 -pl processing
|
||||
|
||||
# processing module tests with SQL Compatibility enabled
|
||||
- sudo: false
|
||||
env:
|
||||
- NAME="processing module test with SQL Compatibility"
|
||||
install: echo "MAVEN_OPTS='-Xmx3000m'" > ~/.mavenrc && mvn install -q -ff -DskipTests -B
|
||||
before_script:
|
||||
- unset _JAVA_OPTIONS
|
||||
script: echo "MAVEN_OPTS='-Xmx512m'" > ~/.mavenrc && mvn test -B -Pparallel-test -Dmaven.fork.count=2 -Ddruid.generic.useDefaultValueForNull=false -pl processing
|
||||
|
||||
# server module test
|
||||
- sudo: false
|
||||
env:
|
||||
|
@ -43,6 +52,17 @@ matrix:
|
|||
# Server module test is run without the parallel-test option because it's memory sensitive and often fails with that option.
|
||||
script: echo "MAVEN_OPTS='-Xmx512m'" > ~/.mavenrc && mvn test -B -pl server
|
||||
|
||||
# server module test with SQL Compatibility enabled
|
||||
- sudo: false
|
||||
env:
|
||||
- NAME="server module test with SQL Compatibility enabled"
|
||||
install: echo "MAVEN_OPTS='-Xmx3000m'" > ~/.mavenrc && mvn install -q -ff -DskipTests -B
|
||||
before_script:
|
||||
- unset _JAVA_OPTIONS
|
||||
# Server module test is run without the parallel-test option because it's memory sensitive and often fails with that option.
|
||||
script: echo "MAVEN_OPTS='-Xmx512m'" > ~/.mavenrc && mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false
|
||||
|
||||
|
||||
# other modules test
|
||||
- sudo: false
|
||||
env:
|
||||
|
@ -53,6 +73,16 @@ matrix:
|
|||
- unset _JAVA_OPTIONS
|
||||
script: echo "MAVEN_OPTS='-Xmx512m'" > ~/.mavenrc && mvn test -B -Pparallel-test -Dmaven.fork.count=2 -pl '!processing,!server'
|
||||
|
||||
# other modules test with SQL Compatibility enabled
|
||||
- sudo: false
|
||||
env:
|
||||
- NAME="other modules test with SQL Compatibility"
|
||||
- AWS_REGION=us-east-1 # set a aws region for unit tests
|
||||
install: echo "MAVEN_OPTS='-Xmx3000m'" > ~/.mavenrc && mvn install -q -ff -DskipTests -B
|
||||
before_script:
|
||||
- unset _JAVA_OPTIONS
|
||||
script: echo "MAVEN_OPTS='-Xmx512m'" > ~/.mavenrc && mvn test -B -Pparallel-test -Dmaven.fork.count=2 -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server'
|
||||
|
||||
# run integration tests
|
||||
- sudo: required
|
||||
services:
|
||||
|
|
|
@ -23,9 +23,11 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.parsers.ParseException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -92,10 +94,11 @@ public class Rows
|
|||
* @throws NullPointerException if the string is null
|
||||
* @throws ParseException if the column cannot be converted to a number
|
||||
*/
|
||||
@Nullable
|
||||
public static Number objectToNumber(final String name, final Object inputValue)
|
||||
{
|
||||
if (inputValue == null) {
|
||||
return Rows.LONG_ZERO;
|
||||
return NullHandling.defaultLongValue();
|
||||
}
|
||||
|
||||
if (inputValue instanceof Number) {
|
||||
|
|
|
@ -21,12 +21,12 @@ package io.druid.benchmark;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.io.Files;
|
||||
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
import io.druid.benchmark.datagen.BenchmarkSchemas;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -557,7 +557,7 @@ public class FilterPartitionBenchmark
|
|||
if (extractionFn == null) {
|
||||
return new NoBitmapSelectorFilter(dimension, value);
|
||||
} else {
|
||||
final String valueOrNull = Strings.emptyToNull(value);
|
||||
final String valueOrNull = NullHandling.emptyToNullIfNeeded(value);
|
||||
|
||||
final DruidPredicateFactory predicateFactory = new DruidPredicateFactory()
|
||||
{
|
||||
|
|
|
@ -171,7 +171,6 @@
|
|||
<property name="illegalPattern" value="true"/>
|
||||
<property name="message" value="Use java.lang.Primitive.BYTES instead."/>
|
||||
</module>
|
||||
|
||||
<!-- This regex should be replaced with an IntelliJ inspection when teamcity.jetbrains.com updates to at least IntelliJ 2018.1 (currently it uses 2017.2) -->
|
||||
<module name="Regexp">
|
||||
<property name="format" value='[a-z][a-zA-Z0-9_]*\.equals\((\"|[A-Z_]+\))'/>
|
||||
|
@ -184,5 +183,15 @@
|
|||
<property name="illegalPattern" value="true"/>
|
||||
<property name="message" value='Use toArray(new Object[0]) instead'/>
|
||||
</module>
|
||||
<module name="Regexp">
|
||||
<property name="format" value="Strings.emptyToNull"/>
|
||||
<property name="illegalPattern" value="true"/>
|
||||
<property name="message" value="Use io.druid.common.config.NullHandling.emptyToNullIfNeeded instead"/>
|
||||
</module>
|
||||
<module name="Regexp">
|
||||
<property name="format" value="Strings.nullToEmpty"/>
|
||||
<property name="illegalPattern" value="true"/>
|
||||
<property name="message" value="Use io.druid.common.config.NullHandling.nullToEmptyIfNeeded instead"/>
|
||||
</module>
|
||||
</module>
|
||||
</module>
|
||||
|
|
|
@ -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.logger.Logger;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
@ -83,6 +83,6 @@ public class Evals
|
|||
|
||||
public static boolean asBoolean(String x)
|
||||
{
|
||||
return !Strings.isNullOrEmpty(x) && Boolean.valueOf(x);
|
||||
return !NullHandling.isNullOrEquivalent(x) && Boolean.valueOf(x);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -272,6 +272,9 @@ class UnaryMinusExpr extends UnaryExpr
|
|||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
ExprEval ret = expr.eval(bindings);
|
||||
if (NullHandling.sqlCompatible() && (ret.value() == null)) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
if (ret.type() == ExprType.LONG) {
|
||||
return ExprEval.of(-ret.asLong());
|
||||
}
|
||||
|
@ -307,6 +310,9 @@ class UnaryNotExpr extends UnaryExpr
|
|||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
ExprEval ret = expr.eval(bindings);
|
||||
if (NullHandling.sqlCompatible() && (ret.value() == null)) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
// conforming to other boolean-returning binary operators
|
||||
ExprType retType = ret.type() == ExprType.DOUBLE ? ExprType.DOUBLE : ExprType.LONG;
|
||||
return ExprEval.of(!ret.asBoolean(), retType);
|
||||
|
@ -365,15 +371,21 @@ abstract class BinaryEvalOpExprBase extends BinaryOpExprBase
|
|||
|
||||
// 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.sqlCompatible() && (leftVal.isNull() || rightVal.isNull())) {
|
||||
if (NullHandling.sqlCompatible() && (leftVal.value() == null || rightVal.value() == null)) {
|
||||
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) {
|
||||
if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(evalLong(leftVal.asLong(), rightVal.asLong()));
|
||||
} else {
|
||||
if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(evalDouble(leftVal.asDouble(), rightVal.asDouble()));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,9 +19,7 @@
|
|||
|
||||
package io.druid.math.expr;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
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;
|
||||
|
@ -32,7 +30,7 @@ import javax.annotation.Nullable;
|
|||
*/
|
||||
public abstract class ExprEval<T>
|
||||
{
|
||||
public static ExprEval ofLong(Number longValue)
|
||||
public static ExprEval ofLong(@Nullable Number longValue)
|
||||
{
|
||||
return new LongExprEval(longValue);
|
||||
}
|
||||
|
@ -42,7 +40,7 @@ public abstract class ExprEval<T>
|
|||
return new LongExprEval(longValue);
|
||||
}
|
||||
|
||||
public static ExprEval ofDouble(Number doubleValue)
|
||||
public static ExprEval ofDouble(@Nullable Number doubleValue)
|
||||
{
|
||||
return new DoubleExprEval(doubleValue);
|
||||
}
|
||||
|
@ -71,7 +69,7 @@ public abstract class ExprEval<T>
|
|||
}
|
||||
}
|
||||
|
||||
public static ExprEval bestEffortOf(Object val)
|
||||
public static ExprEval bestEffortOf(@Nullable Object val)
|
||||
{
|
||||
if (val instanceof ExprEval) {
|
||||
return (ExprEval) val;
|
||||
|
@ -85,6 +83,7 @@ public abstract class ExprEval<T>
|
|||
return new StringExprEval(val == null ? null : String.valueOf(val));
|
||||
}
|
||||
|
||||
@Nullable
|
||||
final T value;
|
||||
|
||||
private ExprEval(T value)
|
||||
|
@ -99,10 +98,10 @@ public abstract class ExprEval<T>
|
|||
return value;
|
||||
}
|
||||
|
||||
public boolean isNull()
|
||||
{
|
||||
return value == null;
|
||||
}
|
||||
/**
|
||||
* returns true if numeric primitive value for this ExprEval is null, otherwise false.
|
||||
*/
|
||||
public abstract boolean isNumericNull();
|
||||
|
||||
public abstract int asInt();
|
||||
|
||||
|
@ -125,7 +124,7 @@ public abstract class ExprEval<T>
|
|||
private abstract static class NumericExprEval extends ExprEval<Number>
|
||||
{
|
||||
|
||||
private NumericExprEval(Number value)
|
||||
private NumericExprEval(@Nullable Number value)
|
||||
{
|
||||
super(value);
|
||||
}
|
||||
|
@ -147,13 +146,19 @@ public abstract class ExprEval<T>
|
|||
{
|
||||
return value.doubleValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNumericNull()
|
||||
{
|
||||
return value == null;
|
||||
}
|
||||
}
|
||||
|
||||
private static class DoubleExprEval extends NumericExprEval
|
||||
{
|
||||
private DoubleExprEval(Number value)
|
||||
private DoubleExprEval(@Nullable Number value)
|
||||
{
|
||||
super(Preconditions.checkNotNull(value, "value"));
|
||||
super(value == null ? NullHandling.defaultDoubleValue() : value);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -175,7 +180,7 @@ public abstract class ExprEval<T>
|
|||
case DOUBLE:
|
||||
return this;
|
||||
case LONG:
|
||||
return ExprEval.of(asLong());
|
||||
return ExprEval.of(value == null ? null : asLong());
|
||||
case STRING:
|
||||
return ExprEval.of(asString());
|
||||
}
|
||||
|
@ -191,9 +196,9 @@ public abstract class ExprEval<T>
|
|||
|
||||
private static class LongExprEval extends NumericExprEval
|
||||
{
|
||||
private LongExprEval(Number value)
|
||||
private LongExprEval(@Nullable Number value)
|
||||
{
|
||||
super(Preconditions.checkNotNull(value, "value"));
|
||||
super(value == null ? NullHandling.defaultLongValue() : value);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -213,7 +218,7 @@ public abstract class ExprEval<T>
|
|||
{
|
||||
switch (castTo) {
|
||||
case DOUBLE:
|
||||
return ExprEval.of(asDouble());
|
||||
return ExprEval.of(value == null ? null : asDouble());
|
||||
case LONG:
|
||||
return this;
|
||||
case STRING:
|
||||
|
@ -231,6 +236,8 @@ public abstract class ExprEval<T>
|
|||
|
||||
private static class StringExprEval extends ExprEval<String>
|
||||
{
|
||||
private Number numericVal;
|
||||
|
||||
private StringExprEval(@Nullable String value)
|
||||
{
|
||||
super(NullHandling.emptyToNullIfNeeded(value));
|
||||
|
@ -245,36 +252,63 @@ public abstract class ExprEval<T>
|
|||
@Override
|
||||
public final int asInt()
|
||||
{
|
||||
if (value == null) {
|
||||
Number number = asNumber();
|
||||
if (number == null) {
|
||||
assert NullHandling.replaceWithDefault();
|
||||
return 0;
|
||||
}
|
||||
|
||||
final Integer theInt = Ints.tryParse(value);
|
||||
assert NullHandling.replaceWithDefault() || theInt != null;
|
||||
return theInt == null ? 0 : theInt;
|
||||
return number.intValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
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;
|
||||
Number number = asNumber();
|
||||
if (number == null) {
|
||||
assert NullHandling.replaceWithDefault();
|
||||
return 0L;
|
||||
}
|
||||
return number.longValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final double asDouble()
|
||||
{
|
||||
if (value == null) {
|
||||
Number number = asNumber();
|
||||
if (number == null) {
|
||||
assert NullHandling.replaceWithDefault();
|
||||
return 0.0;
|
||||
return 0.0d;
|
||||
}
|
||||
return number.doubleValue();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private Number asNumber()
|
||||
{
|
||||
if (value == null) {
|
||||
return null;
|
||||
}
|
||||
if (numericVal != null) {
|
||||
// Optimization for non-null case.
|
||||
return numericVal;
|
||||
}
|
||||
Number rv;
|
||||
Long v = GuavaUtils.tryParseLong(value);
|
||||
// Do NOT use ternary operator here, because it makes Java to convert Long to Double
|
||||
if (v != null) {
|
||||
rv = v;
|
||||
} else {
|
||||
rv = Doubles.tryParse(value);
|
||||
}
|
||||
|
||||
final Double theDouble = Doubles.tryParse(value);
|
||||
assert NullHandling.replaceWithDefault() || theDouble != null;
|
||||
return theDouble == null ? 0.0 : theDouble;
|
||||
numericVal = rv;
|
||||
return rv;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNumericNull()
|
||||
{
|
||||
return asNumber() == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -288,9 +322,9 @@ public abstract class ExprEval<T>
|
|||
{
|
||||
switch (castTo) {
|
||||
case DOUBLE:
|
||||
return ExprEval.of(asDouble());
|
||||
return ExprEval.ofDouble(asNumber());
|
||||
case LONG:
|
||||
return ExprEval.of(asLong());
|
||||
return ExprEval.ofLong(asNumber());
|
||||
case STRING:
|
||||
return this;
|
||||
}
|
||||
|
|
|
@ -74,7 +74,7 @@ interface Function
|
|||
@Override
|
||||
protected final ExprEval eval(ExprEval param)
|
||||
{
|
||||
if (NullHandling.sqlCompatible() && param.isNull()) {
|
||||
if (NullHandling.sqlCompatible() && param.isNumericNull()) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
if (param.type() == ExprType.LONG) {
|
||||
|
@ -796,6 +796,9 @@ interface Function
|
|||
@Override
|
||||
protected ExprEval eval(ExprEval x, ExprEval y)
|
||||
{
|
||||
if (NullHandling.sqlCompatible() && x.value() == null) {
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
ExprType castTo;
|
||||
try {
|
||||
castTo = ExprType.valueOf(StringUtils.toUpperCase(y.asString()));
|
||||
|
@ -880,7 +883,7 @@ interface Function
|
|||
throw new IAE("Function[%s] needs 2 arguments", name());
|
||||
}
|
||||
final ExprEval eval = args.get(0).eval(bindings);
|
||||
return eval.isNull() ? args.get(1).eval(bindings) : eval;
|
||||
return eval.value() == null ? args.get(1).eval(bindings) : eval;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -937,7 +940,7 @@ interface Function
|
|||
}
|
||||
|
||||
final String arg = args.get(0).eval(bindings).asString();
|
||||
return arg == null ? ExprEval.of(0) : ExprEval.of(arg.length());
|
||||
return arg == null ? ExprEval.ofLong(NullHandling.defaultLongValue()) : ExprEval.of(arg.length());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1094,7 +1097,7 @@ interface Function
|
|||
}
|
||||
|
||||
final ExprEval expr = args.get(0).eval(bindings);
|
||||
return ExprEval.of(expr.isNull(), ExprType.LONG);
|
||||
return ExprEval.of(expr.value() == null, ExprType.LONG);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1114,7 +1117,7 @@ interface Function
|
|||
}
|
||||
|
||||
final ExprEval expr = args.get(0).eval(bindings);
|
||||
return ExprEval.of(!expr.isNull(), ExprType.LONG);
|
||||
return ExprEval.of(expr.value() != null, ExprType.LONG);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -140,11 +140,10 @@ public class EvalTest
|
|||
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.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());
|
||||
}
|
||||
Assert.assertEquals(
|
||||
NullHandling.replaceWithDefault() ? "NULL" : "",
|
||||
eval("nvl(if(x == 9223372036854775807, '', 'x'), 'NULL')", bindings).asString()
|
||||
);
|
||||
Assert.assertEquals("x", eval("nvl(if(x == 9223372036854775806, '', 'x'), 'NULL')", bindings).asString());
|
||||
}
|
||||
|
||||
|
|
|
@ -88,7 +88,7 @@ public class FunctionTest
|
|||
public void testStrlen()
|
||||
{
|
||||
assertExpr("strlen(x)", 3L);
|
||||
assertExpr("strlen(nonexistent)", 0L);
|
||||
assertExpr("strlen(nonexistent)", NullHandling.defaultLongValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -12,7 +12,7 @@ layout: doc_page
|
|||
|`druid.auth.unsecuredPaths`| List of Strings|List of paths for which security checks will not be performed. All requests to these paths will be allowed.|[]|no|
|
||||
|`druid.auth.allowUnauthenticatedHttpOptions`|Boolean|If true, skip authentication checks for HTTP OPTIONS requests. This is needed for certain use cases, such as supporting CORS pre-flight requests. Note that disabling authentication checks for OPTIONS requests will allow unauthenticated users to determine what Druid endpoints are valid (by checking if the OPTIONS request returns a 200 instead of 404), so enabling this option may reveal information about server configuration, including information about what extensions are loaded (if those extensions add endpoints).|false|no|
|
||||
|
||||
## Enabling Authentication/Authorization
|
||||
## Enabling Authentication/AuthorizationLoadingLookupTest
|
||||
|
||||
## Authenticator Chain
|
||||
Authentication decisions are handled by a chain of Authenticator instances. A request will be checked by Authenticators in the sequence defined by the `druid.auth.authenticatorChain`.
|
||||
|
|
|
@ -106,13 +106,13 @@ public class DoublesSketchAggregatorFactory extends AggregatorFactory
|
|||
if (selector instanceof NilColumnValueSelector) {
|
||||
return new DoublesSketchNoOpBufferAggregator();
|
||||
}
|
||||
return new DoublesSketchBuildBufferAggregator(selector, k, getMaxIntermediateSize());
|
||||
return new DoublesSketchBuildBufferAggregator(selector, k, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
final ColumnValueSelector<DoublesSketch> selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
if (selector instanceof NilColumnValueSelector) {
|
||||
return new DoublesSketchNoOpBufferAggregator();
|
||||
}
|
||||
return new DoublesSketchMergeBufferAggregator(selector, k, getMaxIntermediateSize());
|
||||
return new DoublesSketchMergeBufferAggregator(selector, k, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -57,7 +57,7 @@ public class DoublesSketchMergeAggregatorFactory extends DoublesSketchAggregator
|
|||
if (selector instanceof NilColumnValueSelector) {
|
||||
return new DoublesSketchNoOpBufferAggregator();
|
||||
}
|
||||
return new DoublesSketchMergeBufferAggregator(selector, getK(), getMaxIntermediateSize());
|
||||
return new DoublesSketchMergeBufferAggregator(selector, getK(), getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -74,7 +74,7 @@ public abstract class SketchAggregatorFactory extends AggregatorFactory
|
|||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new SketchBufferAggregator(selector, size, getMaxIntermediateSize());
|
||||
return new SketchBufferAggregator(selector, size, getMaxIntermediateSizeWithNulls());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -125,7 +125,7 @@ public class ArrayOfDoublesSketchAggregatorFactory extends AggregatorFactory
|
|||
selector,
|
||||
nominalEntries,
|
||||
numberOfValues,
|
||||
getMaxIntermediateSize()
|
||||
getMaxIntermediateSizeWithNulls()
|
||||
);
|
||||
}
|
||||
// input is raw data (key and array of values), use build aggregator
|
||||
|
@ -143,7 +143,7 @@ public class ArrayOfDoublesSketchAggregatorFactory extends AggregatorFactory
|
|||
keySelector,
|
||||
valueSelectors,
|
||||
nominalEntries,
|
||||
getMaxIntermediateSize()
|
||||
getMaxIntermediateSizeWithNulls()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.query.aggregation.histogram;
|
||||
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.segment.BaseFloatColumnValueSelector;
|
||||
|
||||
|
@ -59,7 +60,12 @@ public class ApproximateHistogramAggregator implements Aggregator
|
|||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
histogram.offer(selector.getFloat());
|
||||
// In case of ExpressionColumnValueSelector isNull will compute the expression and then give the result,
|
||||
// the check for is NullHandling.replaceWithDefault is there to not have any performance impact of calling
|
||||
// isNull for default case.
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
histogram.offer(selector.getFloat());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.query.aggregation.histogram;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -79,10 +80,14 @@ public class ApproximateHistogramAggregationTest
|
|||
@Test
|
||||
public void testIngestWithNullsToZeroAndQuery() throws Exception
|
||||
{
|
||||
MapBasedRow row = ingestAndQuery(false);
|
||||
Assert.assertEquals(0.0, row.getMetric("index_min").floatValue(), 0.0001);
|
||||
Assert.assertEquals(135.109191, row.getMetric("index_max").floatValue(), 0.0001);
|
||||
Assert.assertEquals(131.428176, row.getMetric("index_quantile").floatValue(), 0.0001);
|
||||
// Nulls are ignored and not replaced with default for SQL compatible null handling.
|
||||
// This is already tested in testIngestWithNullsIgnoredAndQuery()
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
MapBasedRow row = ingestAndQuery(false);
|
||||
Assert.assertEquals(0.0F, row.getMetric("index_min"));
|
||||
Assert.assertEquals(135.109191, row.getMetric("index_max").floatValue(), 0.0001);
|
||||
Assert.assertEquals(131.428176, row.getMetric("index_quantile").floatValue(), 0.0001);
|
||||
}
|
||||
}
|
||||
|
||||
private MapBasedRow ingestAndQuery(boolean ignoreNulls) throws Exception
|
||||
|
|
|
@ -48,7 +48,7 @@ public class ApproximateHistogramAggregatorTest
|
|||
);
|
||||
ApproximateHistogramBufferAggregator agg = new ApproximateHistogramBufferAggregator(selector, resolution);
|
||||
|
||||
ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize());
|
||||
ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSizeWithNulls());
|
||||
int position = 0;
|
||||
|
||||
agg.init(buf, position);
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryDataSource;
|
||||
|
@ -312,9 +313,12 @@ public class QuantileSqlAggregatorTest extends CalciteTestBase
|
|||
|
||||
// Verify results
|
||||
final List<Object[]> results = plannerResult.run().toList();
|
||||
final List<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[]{7.0, 8.26386833190918}
|
||||
);
|
||||
final List<Object[]> expectedResults;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
expectedResults = ImmutableList.of(new Object[]{7.0, 8.26386833190918});
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(new Object[]{5.25, 6.59091854095459});
|
||||
}
|
||||
Assert.assertEquals(expectedResults.size(), results.size());
|
||||
for (int i = 0; i < expectedResults.size(); i++) {
|
||||
Assert.assertArrayEquals(expectedResults.get(i), results.get(i));
|
||||
|
|
|
@ -115,6 +115,7 @@ import io.druid.query.timeseries.TimeseriesQueryEngine;
|
|||
import io.druid.query.timeseries.TimeseriesQueryQueryToolChest;
|
||||
import io.druid.query.timeseries.TimeseriesQueryRunnerFactory;
|
||||
import io.druid.query.timeseries.TimeseriesResultValue;
|
||||
import io.druid.segment.DimensionHandlerUtils;
|
||||
import io.druid.segment.IndexIO;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.TestHelper;
|
||||
|
@ -2244,7 +2245,7 @@ public class KafkaIndexTaskTest
|
|||
List<Result<TimeseriesResultValue>> results =
|
||||
task.getQueryRunner(query).run(wrap(query), ImmutableMap.of()).toList();
|
||||
|
||||
return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric("rows");
|
||||
return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows"));
|
||||
}
|
||||
|
||||
private static byte[] JB(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1)
|
||||
|
|
|
@ -35,6 +35,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.UOE;
|
||||
import io.druid.java.util.common.jackson.JacksonUtils;
|
||||
import io.druid.java.util.common.parsers.CSVParser;
|
||||
|
@ -396,8 +397,8 @@ public class UriExtractionNamespace implements ExtractionNamespace
|
|||
"Must specify more than one column to have a key value pair"
|
||||
);
|
||||
final DelimitedParser delegate = new DelimitedParser(
|
||||
Strings.emptyToNull(delimiter),
|
||||
Strings.emptyToNull(listDelimiter),
|
||||
StringUtils.emptyToNullNonDruidDataString(delimiter),
|
||||
StringUtils.emptyToNullNonDruidDataString(listDelimiter),
|
||||
hasHeaderRow,
|
||||
skipHeaderRows
|
||||
);
|
||||
|
|
|
@ -19,15 +19,15 @@
|
|||
|
||||
package io.druid.server.lookup.namespace.cache;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.concurrent.Execs;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.io.Closer;
|
||||
import io.druid.java.util.common.lifecycle.Lifecycle;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -382,7 +382,7 @@ public class JdbcExtractionNamespaceTest
|
|||
String key = e.getKey();
|
||||
String[] val = e.getValue();
|
||||
String field = val[0];
|
||||
Assert.assertEquals("non-null check", Strings.emptyToNull(field), Strings.emptyToNull(map.get(key)));
|
||||
Assert.assertEquals("non-null check", NullHandling.emptyToNullIfNeeded(field), NullHandling.emptyToNullIfNeeded(map.get(key)));
|
||||
}
|
||||
Assert.assertEquals("null check", null, map.get("baz"));
|
||||
}
|
||||
|
@ -412,9 +412,9 @@ public class JdbcExtractionNamespaceTest
|
|||
String filterVal = val[1];
|
||||
|
||||
if ("1".equals(filterVal)) {
|
||||
Assert.assertEquals("non-null check", Strings.emptyToNull(field), Strings.emptyToNull(map.get(key)));
|
||||
Assert.assertEquals("non-null check", NullHandling.emptyToNullIfNeeded(field), NullHandling.emptyToNullIfNeeded(map.get(key)));
|
||||
} else {
|
||||
Assert.assertEquals("non-null check", null, Strings.emptyToNull(map.get(key)));
|
||||
Assert.assertEquals("non-null check", null, NullHandling.emptyToNullIfNeeded(map.get(key)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,11 +21,12 @@ package io.druid.server.lookup;
|
|||
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.lookup.LookupExtractor;
|
||||
import io.druid.server.lookup.cache.loading.LoadingCache;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Callable;
|
||||
|
@ -62,15 +63,19 @@ public class LoadingLookup extends LookupExtractor
|
|||
|
||||
|
||||
@Override
|
||||
public String apply(final String key)
|
||||
public String apply(@Nullable final String key)
|
||||
{
|
||||
if (key == null) {
|
||||
String keyEquivalent = NullHandling.nullToEmptyIfNeeded(key);
|
||||
if (keyEquivalent == null) {
|
||||
// valueEquivalent is null only for SQL Compatible Null Behavior
|
||||
// otherwise null will be replaced with empty string in nullToEmptyIfNeeded above.
|
||||
return null;
|
||||
}
|
||||
|
||||
final String presentVal;
|
||||
try {
|
||||
presentVal = loadingCache.get(key, new ApplyCallable(key));
|
||||
return Strings.emptyToNull(presentVal);
|
||||
presentVal = loadingCache.get(keyEquivalent, new ApplyCallable(keyEquivalent));
|
||||
return NullHandling.emptyToNullIfNeeded(presentVal);
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
LOGGER.debug("value not found for key [%s]", key);
|
||||
|
@ -79,15 +84,18 @@ public class LoadingLookup extends LookupExtractor
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<String> unapply(final String value)
|
||||
public List<String> unapply(@Nullable final String value)
|
||||
{
|
||||
// null value maps to empty list
|
||||
if (value == null) {
|
||||
String valueEquivalent = NullHandling.nullToEmptyIfNeeded(value);
|
||||
if (valueEquivalent == null) {
|
||||
// valueEquivalent is null only for SQL Compatible Null Behavior
|
||||
// otherwise null will be replaced with empty string in nullToEmptyIfNeeded above.
|
||||
// null value maps to empty list when SQL Compatible
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
final List<String> retList;
|
||||
try {
|
||||
retList = reverseLoadingCache.get(value, new UnapplyCallable(value));
|
||||
retList = reverseLoadingCache.get(valueEquivalent, new UnapplyCallable(valueEquivalent));
|
||||
return retList;
|
||||
}
|
||||
catch (ExecutionException e) {
|
||||
|
@ -131,8 +139,9 @@ public class LoadingLookup extends LookupExtractor
|
|||
@Override
|
||||
public String call()
|
||||
{
|
||||
// When SQL compatible null handling is disabled,
|
||||
// avoid returning null and return an empty string to cache it.
|
||||
return Strings.nullToEmpty(dataFetcher.fetch(key));
|
||||
return NullHandling.nullToEmptyIfNeeded(dataFetcher.fetch(key));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,11 +20,11 @@
|
|||
package io.druid.server.lookup;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.concurrent.Execs;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -33,7 +33,8 @@ import io.druid.server.lookup.cache.polling.OnHeapPollingCache;
|
|||
import io.druid.server.lookup.cache.polling.PollingCache;
|
||||
import io.druid.server.lookup.cache.polling.PollingCacheFactory;
|
||||
|
||||
import javax.validation.constraints.NotNull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -107,8 +108,15 @@ public class PollingLookup extends LookupExtractor
|
|||
}
|
||||
|
||||
@Override
|
||||
public String apply(@NotNull String key)
|
||||
@Nullable
|
||||
public String apply(@Nullable String key)
|
||||
{
|
||||
String keyEquivalent = NullHandling.nullToEmptyIfNeeded(key);
|
||||
if (keyEquivalent == null) {
|
||||
// valueEquivalent is null only for SQL Compatible Null Behavior
|
||||
// otherwise null will be replaced with empty string in nullToEmptyIfNeeded above.
|
||||
return null;
|
||||
}
|
||||
final CacheRefKeeper cacheRefKeeper = refOfCacheKeeper.get();
|
||||
if (cacheRefKeeper == null) {
|
||||
throw new ISE("Cache reference is null WTF");
|
||||
|
@ -117,9 +125,9 @@ public class PollingLookup extends LookupExtractor
|
|||
try {
|
||||
if (cache == null) {
|
||||
// it must've been closed after swapping while I was getting it. Try again.
|
||||
return this.apply(key);
|
||||
return this.apply(keyEquivalent);
|
||||
}
|
||||
return Strings.emptyToNull((String) cache.get(key));
|
||||
return NullHandling.emptyToNullIfNeeded((String) cache.get(keyEquivalent));
|
||||
}
|
||||
finally {
|
||||
if (cacheRefKeeper != null && cache != null) {
|
||||
|
@ -129,8 +137,16 @@ public class PollingLookup extends LookupExtractor
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<String> unapply(final String value)
|
||||
public List<String> unapply(@Nullable final String value)
|
||||
{
|
||||
String valueEquivalent = NullHandling.nullToEmptyIfNeeded(value);
|
||||
if (valueEquivalent == null) {
|
||||
// valueEquivalent is null only for SQL Compatible Null Behavior
|
||||
// otherwise null will be replaced with empty string in nullToEmptyIfNeeded above.
|
||||
// null value maps to empty list when SQL Compatible
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
CacheRefKeeper cacheRefKeeper = refOfCacheKeeper.get();
|
||||
if (cacheRefKeeper == null) {
|
||||
throw new ISE("pollingLookup id [%s] is closed", id);
|
||||
|
@ -139,9 +155,9 @@ public class PollingLookup extends LookupExtractor
|
|||
try {
|
||||
if (cache == null) {
|
||||
// it must've been closed after swapping while I was getting it. Try again.
|
||||
return this.unapply(value);
|
||||
return this.unapply(valueEquivalent);
|
||||
}
|
||||
return cache.getKeys(value);
|
||||
return cache.getKeys(valueEquivalent);
|
||||
}
|
||||
finally {
|
||||
if (cacheRefKeeper != null && cache != null) {
|
||||
|
|
|
@ -21,8 +21,9 @@ package io.druid.server.lookup.jdbc;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
|
@ -131,7 +132,7 @@ public class JdbcDataFetcher implements DataFetcher<String, String>
|
|||
if (pairs.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
return Strings.nullToEmpty(pairs.get(0));
|
||||
return NullHandling.nullToEmptyIfNeeded(pairs.get(0));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.server.lookup;
|
|||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.server.lookup.cache.loading.LoadingCache;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
|
@ -39,16 +40,29 @@ public class LoadingLookupTest
|
|||
LoadingLookup loadingLookup = new LoadingLookup(dataFetcher, lookupCache, reverseLookupCache);
|
||||
|
||||
@Test
|
||||
public void testApplyEmptyOrNull()
|
||||
public void testApplyEmptyOrNull() throws ExecutionException
|
||||
{
|
||||
Assert.assertEquals(null, loadingLookup.apply(null));
|
||||
Assert.assertEquals(null, loadingLookup.apply(""));
|
||||
EasyMock.expect(lookupCache.get(EasyMock.eq(""), EasyMock.anyObject(Callable.class)))
|
||||
.andReturn("empty").atLeastOnce();
|
||||
EasyMock.replay(lookupCache);
|
||||
Assert.assertEquals("empty", loadingLookup.apply(""));
|
||||
if (!NullHandling.sqlCompatible()) {
|
||||
// Nulls and empty strings should have same behavior
|
||||
Assert.assertEquals("empty", loadingLookup.apply(null));
|
||||
} else {
|
||||
Assert.assertNull(loadingLookup.apply(null));
|
||||
}
|
||||
EasyMock.verify(lookupCache);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnapplyNull()
|
||||
{
|
||||
Assert.assertEquals(Collections.EMPTY_LIST, loadingLookup.unapply(null));
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertEquals(Collections.emptyList(), loadingLookup.unapply(null));
|
||||
} else {
|
||||
Assert.assertNull(loadingLookup.unapply(null));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -88,7 +102,7 @@ public class LoadingLookupTest
|
|||
.andThrow(new ExecutionException(null))
|
||||
.once();
|
||||
EasyMock.replay(lookupCache);
|
||||
Assert.assertEquals(null, loadingLookup.apply("key"));
|
||||
Assert.assertNull(loadingLookup.apply("key"));
|
||||
EasyMock.verify(lookupCache);
|
||||
}
|
||||
|
||||
|
@ -99,7 +113,7 @@ public class LoadingLookupTest
|
|||
.andThrow(new ExecutionException(null))
|
||||
.once();
|
||||
EasyMock.replay(reverseLookupCache);
|
||||
Assert.assertEquals(Collections.EMPTY_LIST, loadingLookup.unapply("value"));
|
||||
Assert.assertEquals(Collections.emptyList(), loadingLookup.unapply("value"));
|
||||
EasyMock.verify(reverseLookupCache);
|
||||
}
|
||||
|
||||
|
|
|
@ -21,10 +21,11 @@ package io.druid.server.lookup;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.query.lookup.LookupExtractor;
|
||||
import io.druid.server.lookup.cache.polling.OffHeapPollingCache;
|
||||
|
@ -190,7 +191,7 @@ public class PollingLookupTest
|
|||
public String apply(String input)
|
||||
{
|
||||
//make sure to rewrite null strings as empty.
|
||||
return Strings.nullToEmpty(input);
|
||||
return NullHandling.nullToEmptyIfNeeded(input);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
@ -207,7 +208,7 @@ public class PollingLookupTest
|
|||
for (Map.Entry<String, String> entry : map.entrySet()) {
|
||||
String key = entry.getKey();
|
||||
String val = entry.getValue();
|
||||
Assert.assertEquals("non-null check", Strings.emptyToNull(val), lookup.apply(key));
|
||||
Assert.assertEquals("non-null check", NullHandling.emptyToNullIfNeeded(val), lookup.apply(key));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -111,7 +111,7 @@ public class VarianceAggregatorTest
|
|||
colSelectorFactory
|
||||
);
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.wrap(new byte[aggFactory.getMaxIntermediateSize()]);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(new byte[aggFactory.getMaxIntermediateSizeWithNulls()]);
|
||||
agg.init(buffer, 0);
|
||||
|
||||
assertValues((VarianceAggregatorCollector) agg.get(buffer, 0), 0, 0d, 0d);
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.io.ByteArrayDataInput;
|
|||
import com.google.common.io.ByteArrayDataOutput;
|
||||
import com.google.common.io.ByteStreams;
|
||||
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.Rows;
|
||||
|
@ -45,6 +46,7 @@ import io.druid.segment.serde.ComplexMetricSerde;
|
|||
import io.druid.segment.serde.ComplexMetrics;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.DataInput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -330,18 +332,22 @@ public class InputRowSerde
|
|||
}
|
||||
|
||||
String t = aggFactory.getTypeName();
|
||||
|
||||
if ("float".equals(t)) {
|
||||
out.writeFloat(agg.getFloat());
|
||||
} else if ("long".equals(t)) {
|
||||
WritableUtils.writeVLong(out, agg.getLong());
|
||||
} else if ("double".equals(t)) {
|
||||
out.writeDouble(agg.getDouble());
|
||||
if (agg.isNull()) {
|
||||
out.writeByte(NullHandling.IS_NULL_BYTE);
|
||||
} else {
|
||||
//its a complex metric
|
||||
Object val = agg.get();
|
||||
ComplexMetricSerde serde = getComplexMetricSerde(t);
|
||||
writeBytes(serde.toBytes(val), out);
|
||||
out.writeByte(NullHandling.IS_NOT_NULL_BYTE);
|
||||
if ("float".equals(t)) {
|
||||
out.writeFloat(agg.getFloat());
|
||||
} else if ("long".equals(t)) {
|
||||
WritableUtils.writeVLong(out, agg.getLong());
|
||||
} else if ("double".equals(t)) {
|
||||
out.writeDouble(agg.getDouble());
|
||||
} else {
|
||||
//its a complex metric
|
||||
Object val = agg.get();
|
||||
ComplexMetricSerde serde = getComplexMetricSerde(t);
|
||||
writeBytes(serde.toBytes(val), out);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -353,10 +359,13 @@ public class InputRowSerde
|
|||
}
|
||||
}
|
||||
|
||||
private static void writeBytes(byte[] value, ByteArrayDataOutput out) throws IOException
|
||||
private static void writeBytes(@Nullable byte[] value, ByteArrayDataOutput out) throws IOException
|
||||
{
|
||||
WritableUtils.writeVInt(out, value.length);
|
||||
out.write(value, 0, value.length);
|
||||
int length = value == null ? -1 : value.length;
|
||||
WritableUtils.writeVInt(out, length);
|
||||
if (value != null) {
|
||||
out.write(value, 0, value.length);
|
||||
}
|
||||
}
|
||||
|
||||
private static void writeString(String value, ByteArrayDataOutput out) throws IOException
|
||||
|
@ -450,6 +459,11 @@ public class InputRowSerde
|
|||
for (int i = 0; i < metricSize; i++) {
|
||||
String metric = readString(in);
|
||||
String type = getType(metric, aggs, i);
|
||||
byte metricNullability = in.readByte();
|
||||
if (metricNullability == NullHandling.IS_NULL_BYTE) {
|
||||
// metric value is null.
|
||||
continue;
|
||||
}
|
||||
if ("float".equals(type)) {
|
||||
event.put(metric, in.readFloat());
|
||||
} else if ("long".equals(type)) {
|
||||
|
|
|
@ -295,8 +295,8 @@ public class JobHelper
|
|||
|
||||
public static void injectDruidProperties(Configuration configuration, List<String> listOfAllowedPrefix)
|
||||
{
|
||||
String mapJavaOpts = Strings.nullToEmpty(configuration.get(MRJobConfig.MAP_JAVA_OPTS));
|
||||
String reduceJavaOpts = Strings.nullToEmpty(configuration.get(MRJobConfig.REDUCE_JAVA_OPTS));
|
||||
String mapJavaOpts = StringUtils.nullToEmptyNonDruidDataString(configuration.get(MRJobConfig.MAP_JAVA_OPTS));
|
||||
String reduceJavaOpts = StringUtils.nullToEmptyNonDruidDataString(configuration.get(MRJobConfig.REDUCE_JAVA_OPTS));
|
||||
|
||||
for (String propName : System.getProperties().stringPropertyNames()) {
|
||||
for (String prefix : listOfAllowedPrefix) {
|
||||
|
|
|
@ -21,11 +21,11 @@ package io.druid.indexer.path;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -126,7 +126,9 @@ public class StaticPathSpec implements PathSpec
|
|||
private static void addInputPath(Job job, Iterable<String> pathStrings, Class<? extends InputFormat> inputFormatClass)
|
||||
{
|
||||
Configuration conf = job.getConfiguration();
|
||||
StringBuilder inputFormats = new StringBuilder(Strings.nullToEmpty(conf.get(MultipleInputs.DIR_FORMATS)));
|
||||
StringBuilder inputFormats = new StringBuilder(
|
||||
StringUtils.nullToEmptyNonDruidDataString(conf.get(MultipleInputs.DIR_FORMATS))
|
||||
);
|
||||
|
||||
String[] paths = Iterables.toArray(pathStrings, String.class);
|
||||
for (int i = 0; i < paths.length - 1; i++) {
|
||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.indexer;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
|
@ -83,6 +84,7 @@ public class InputRowSerdeTest
|
|||
{
|
||||
// Prepare the mocks & set close() call count expectation to 1
|
||||
final Aggregator mockedAggregator = EasyMock.createMock(DoubleSumAggregator.class);
|
||||
EasyMock.expect(mockedAggregator.isNull()).andReturn(false).times(1);
|
||||
EasyMock.expect(mockedAggregator.getDouble()).andReturn(0d).times(1);
|
||||
mockedAggregator.aggregate();
|
||||
EasyMock.expectLastCall().times(1);
|
||||
|
@ -90,6 +92,26 @@ public class InputRowSerdeTest
|
|||
EasyMock.expectLastCall().times(1);
|
||||
EasyMock.replay(mockedAggregator);
|
||||
|
||||
final Aggregator mockedNullAggregator = EasyMock.createMock(DoubleSumAggregator.class);
|
||||
EasyMock.expect(mockedNullAggregator.isNull()).andReturn(true).times(1);
|
||||
mockedNullAggregator.aggregate();
|
||||
EasyMock.expectLastCall().times(1);
|
||||
mockedNullAggregator.close();
|
||||
EasyMock.expectLastCall().times(1);
|
||||
EasyMock.replay(mockedNullAggregator);
|
||||
|
||||
final AggregatorFactory mockedAggregatorFactory = EasyMock.createMock(AggregatorFactory.class);
|
||||
EasyMock.expect(mockedAggregatorFactory.factorize(EasyMock.anyObject(ColumnSelectorFactory.class))).andReturn(mockedAggregator);
|
||||
EasyMock.expect(mockedAggregatorFactory.getTypeName()).andReturn("double").anyTimes();
|
||||
EasyMock.expect(mockedAggregatorFactory.getName()).andReturn("mockedAggregator").anyTimes();
|
||||
|
||||
final AggregatorFactory mockedNullAggregatorFactory = EasyMock.createMock(AggregatorFactory.class);
|
||||
EasyMock.expect(mockedNullAggregatorFactory.factorize(EasyMock.anyObject(ColumnSelectorFactory.class))).andReturn(mockedNullAggregator);
|
||||
EasyMock.expect(mockedNullAggregatorFactory.getName()).andReturn("mockedNullAggregator").anyTimes();
|
||||
EasyMock.expect(mockedNullAggregatorFactory.getTypeName()).andReturn("double").anyTimes();
|
||||
|
||||
EasyMock.replay(mockedAggregatorFactory, mockedNullAggregatorFactory);
|
||||
|
||||
InputRow in = new MapBasedInputRow(
|
||||
timestamp,
|
||||
dims,
|
||||
|
@ -102,13 +124,8 @@ public class InputRowSerdeTest
|
|||
new LongSumAggregatorFactory("m2out", "m2"),
|
||||
new HyperUniquesAggregatorFactory("m3out", "m3"),
|
||||
new LongSumAggregatorFactory("unparseable", "m3"), // Unparseable from String to Long
|
||||
new DoubleSumAggregatorFactory("mockedAggregator", "m4") {
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return mockedAggregator;
|
||||
}
|
||||
}
|
||||
mockedAggregatorFactory,
|
||||
mockedNullAggregatorFactory
|
||||
};
|
||||
|
||||
DimensionsSpec dimensionsSpec = new DimensionsSpec(
|
||||
|
@ -136,13 +153,14 @@ public class InputRowSerdeTest
|
|||
Assert.assertEquals(300.1f, out.getRaw("d4"));
|
||||
Assert.assertEquals(400.5d, out.getRaw("d5"));
|
||||
|
||||
Assert.assertEquals(0.0f, out.getMetric("agg_non_existing").floatValue(), 0.00001);
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), out.getMetric("agg_non_existing"));
|
||||
Assert.assertEquals(5.0f, out.getMetric("m1out").floatValue(), 0.00001);
|
||||
Assert.assertEquals(100L, out.getMetric("m2out"));
|
||||
Assert.assertEquals(1, ((HyperLogLogCollector) out.getRaw("m3out")).estimateCardinality(), 0.001);
|
||||
Assert.assertEquals(0L, out.getMetric("unparseable"));
|
||||
|
||||
EasyMock.verify(mockedAggregator);
|
||||
EasyMock.verify(mockedNullAggregator);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -31,6 +31,7 @@ import com.google.common.util.concurrent.ListeningExecutorService;
|
|||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.client.cache.CacheConfig;
|
||||
import io.druid.client.cache.MapCache;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
|
@ -130,6 +131,7 @@ import io.druid.server.security.AuthTestUtils;
|
|||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.LinearShardSpec;
|
||||
import io.druid.timeline.partition.NumberedShardSpec;
|
||||
import io.druid.utils.Runnables;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -142,6 +144,7 @@ import org.junit.Test;
|
|||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
|
@ -223,7 +226,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return () -> {};
|
||||
return Runnables.getNoopRunnable();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -331,7 +334,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
|
||||
// handoff would timeout, resulting in exception
|
||||
TaskStatus status = statusFuture.get();
|
||||
Assert.assertTrue(status.getErrorMsg().contains("java.util.concurrent.TimeoutException: Timeout waiting for task."));
|
||||
Assert.assertTrue(status.getErrorMsg()
|
||||
.contains("java.util.concurrent.TimeoutException: Timeout waiting for task."));
|
||||
}
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
|
@ -367,8 +371,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -429,8 +433,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -494,8 +498,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(2000, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(2000, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(2000, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(2000, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -562,10 +566,14 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
Assert.assertEquals(0, task.getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(2, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows"));
|
||||
Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(2, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows").longValue());
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "metric1").longValue());
|
||||
} else {
|
||||
Assert.assertNull(sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "metric1"));
|
||||
}
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -620,7 +628,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
|
||||
// Wait for the task to finish.
|
||||
TaskStatus status = statusFuture.get();
|
||||
Assert.assertTrue(status.getErrorMsg().contains("java.lang.RuntimeException: Max parse exceptions exceeded, terminating task..."));
|
||||
Assert.assertTrue(status.getErrorMsg()
|
||||
.contains("java.lang.RuntimeException: Max parse exceptions exceeded, terminating task..."));
|
||||
|
||||
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
|
||||
|
||||
|
@ -639,7 +648,15 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
{
|
||||
expectPublishedSegments(1);
|
||||
|
||||
final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, false, 0, true, null, 1);
|
||||
final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(
|
||||
null,
|
||||
TransformSpec.NONE,
|
||||
false,
|
||||
0,
|
||||
true,
|
||||
null,
|
||||
1
|
||||
);
|
||||
final ListenableFuture<TaskStatus> statusFuture = runTask(task);
|
||||
|
||||
// Wait for firehose to show up, it starts off null.
|
||||
|
@ -683,8 +700,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
Assert.assertEquals(2, task.getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(3, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -750,7 +767,18 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "foo"),
|
||||
|
||||
// Bad long dim- will count as processed, but bad dims will get default values
|
||||
ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "dimLong", "notnumber", "dimFloat", "notnumber", "met1", "foo"),
|
||||
ImmutableMap.of(
|
||||
"t",
|
||||
1521251960729L,
|
||||
"dim1",
|
||||
"foo",
|
||||
"dimLong",
|
||||
"notnumber",
|
||||
"dimFloat",
|
||||
"notnumber",
|
||||
"met1",
|
||||
"foo"
|
||||
),
|
||||
|
||||
// Bad row- will be unparseable.
|
||||
ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"),
|
||||
|
@ -775,8 +803,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
Assert.assertEquals(2, task.getRowIngestionMeters().getUnparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(4, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(4, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -852,7 +880,18 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "met1", "foo"),
|
||||
|
||||
// Bad long dim- will count as processed, but bad dims will get default values
|
||||
ImmutableMap.of("t", 1521251960729L, "dim1", "foo", "dimLong", "notnumber", "dimFloat", "notnumber", "met1", "foo"),
|
||||
ImmutableMap.of(
|
||||
"t",
|
||||
1521251960729L,
|
||||
"dim1",
|
||||
"foo",
|
||||
"dimLong",
|
||||
"notnumber",
|
||||
"dimFloat",
|
||||
"notnumber",
|
||||
"met1",
|
||||
"foo"
|
||||
),
|
||||
|
||||
// Bad row- will be unparseable.
|
||||
ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"),
|
||||
|
@ -943,7 +982,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
}
|
||||
|
||||
// Do a query, at this point the previous data should be loaded.
|
||||
Assert.assertEquals(1, sumMetric(task2, null, "rows"));
|
||||
Assert.assertEquals(1, sumMetric(task2, null, "rows").longValue());
|
||||
|
||||
final TestFirehose firehose = (TestFirehose) task2.getFirehose();
|
||||
|
||||
|
@ -961,7 +1000,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
publishedSegment = Iterables.getOnlyElement(publishedSegments);
|
||||
|
||||
// Do a query.
|
||||
Assert.assertEquals(2, sumMetric(task2, null, "rows"));
|
||||
Assert.assertEquals(2, sumMetric(task2, null, "rows").longValue());
|
||||
|
||||
awaitHandoffs();
|
||||
|
||||
|
@ -1018,7 +1057,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
publishedSegment = Iterables.getOnlyElement(publishedSegments);
|
||||
|
||||
// Do a query.
|
||||
Assert.assertEquals(1, sumMetric(task1, null, "rows"));
|
||||
Assert.assertEquals(1, sumMetric(task1, null, "rows").longValue());
|
||||
|
||||
// Trigger graceful shutdown.
|
||||
task1.stopGracefully();
|
||||
|
@ -1137,7 +1176,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
|
||||
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
|
||||
Assert.assertEquals(expectedMetrics, reportData.getRowStats());
|
||||
Assert.assertTrue(status.getErrorMsg().contains("java.lang.IllegalArgumentException\n\tat java.nio.Buffer.position"));
|
||||
Assert.assertTrue(status.getErrorMsg()
|
||||
.contains("java.lang.IllegalArgumentException\n\tat java.nio.Buffer.position"));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1466,7 +1506,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
);
|
||||
}
|
||||
|
||||
public long sumMetric(final Task task, final DimFilter filter, final String metric)
|
||||
@Nullable
|
||||
public Long sumMetric(final Task task, final DimFilter filter, final String metric)
|
||||
{
|
||||
// Do a query.
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
|
@ -1482,7 +1523,12 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
|
||||
List<Result<TimeseriesResultValue>> results =
|
||||
task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
|
||||
return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric(metric);
|
||||
|
||||
if (results.isEmpty()) {
|
||||
return 0L;
|
||||
} else {
|
||||
return results.get(0).getValue().getLongMetric(metric);
|
||||
}
|
||||
}
|
||||
|
||||
private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException
|
||||
|
|
|
@ -32,6 +32,7 @@ import com.google.common.util.concurrent.ListeningExecutorService;
|
|||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.client.cache.CacheConfig;
|
||||
import io.druid.client.cache.MapCache;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.data.input.Firehose;
|
||||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
|
@ -118,6 +119,7 @@ import io.druid.server.DruidNode;
|
|||
import io.druid.server.coordination.DataSegmentServerAnnouncer;
|
||||
import io.druid.server.coordination.ServerType;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.utils.Runnables;
|
||||
import org.easymock.EasyMock;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -131,6 +133,7 @@ import org.junit.internal.matchers.ThrowableMessageMatcher;
|
|||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.nio.file.Files;
|
||||
import java.util.Arrays;
|
||||
|
@ -204,7 +207,7 @@ public class RealtimeIndexTaskTest
|
|||
@Override
|
||||
public Runnable commit()
|
||||
{
|
||||
return () -> {};
|
||||
return Runnables.getNoopRunnable();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -350,8 +353,8 @@ public class RealtimeIndexTaskTest
|
|||
Assert.assertEquals(0, task.getMetrics().unparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(2, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
// Simulate handoff.
|
||||
for (Map.Entry<SegmentDescriptor, Pair<Executor, Runnable>> entry : handOffCallbacks.entrySet()) {
|
||||
|
@ -419,10 +422,15 @@ public class RealtimeIndexTaskTest
|
|||
Assert.assertEquals(0, task.getMetrics().unparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(1, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(1, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows"));
|
||||
Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows"));
|
||||
Assert.assertEquals(1, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(1, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(1, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows").longValue());
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows").longValue());
|
||||
} else {
|
||||
Assert.assertNull(sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows"));
|
||||
|
||||
}
|
||||
Assert.assertEquals(1, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
// Simulate handoff.
|
||||
for (Map.Entry<SegmentDescriptor, Pair<Executor, Runnable>> entry : handOffCallbacks.entrySet()) {
|
||||
|
@ -537,8 +545,8 @@ public class RealtimeIndexTaskTest
|
|||
Assert.assertEquals(2, task.getMetrics().unparseable());
|
||||
|
||||
// Do some queries.
|
||||
Assert.assertEquals(3, sumMetric(task, null, "rows"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1"));
|
||||
Assert.assertEquals(3, sumMetric(task, null, "rows").longValue());
|
||||
Assert.assertEquals(3, sumMetric(task, null, "met1").longValue());
|
||||
|
||||
// Simulate handoff.
|
||||
for (Map.Entry<SegmentDescriptor, Pair<Executor, Runnable>> entry : handOffCallbacks.entrySet()) {
|
||||
|
@ -610,7 +618,7 @@ public class RealtimeIndexTaskTest
|
|||
}
|
||||
|
||||
// Do a query, at this point the previous data should be loaded.
|
||||
Assert.assertEquals(1, sumMetric(task2, null, "rows"));
|
||||
Assert.assertEquals(1, sumMetric(task2, null, "rows").longValue());
|
||||
|
||||
final TestFirehose firehose = (TestFirehose) task2.getFirehose();
|
||||
|
||||
|
@ -631,7 +639,7 @@ public class RealtimeIndexTaskTest
|
|||
publishedSegment = Iterables.getOnlyElement(mdc.getPublished());
|
||||
|
||||
// Do a query.
|
||||
Assert.assertEquals(2, sumMetric(task2, null, "rows"));
|
||||
Assert.assertEquals(2, sumMetric(task2, null, "rows").longValue());
|
||||
|
||||
// Simulate handoff.
|
||||
for (Map.Entry<SegmentDescriptor, Pair<Executor, Runnable>> entry : handOffCallbacks.entrySet()) {
|
||||
|
@ -692,7 +700,7 @@ public class RealtimeIndexTaskTest
|
|||
publishedSegment = Iterables.getOnlyElement(mdc.getPublished());
|
||||
|
||||
// Do a query.
|
||||
Assert.assertEquals(1, sumMetric(task1, null, "rows"));
|
||||
Assert.assertEquals(1, sumMetric(task1, null, "rows").longValue());
|
||||
|
||||
// Trigger graceful shutdown.
|
||||
task1.stopGracefully();
|
||||
|
@ -1080,7 +1088,8 @@ public class RealtimeIndexTaskTest
|
|||
return toolboxFactory.build(task);
|
||||
}
|
||||
|
||||
public long sumMetric(final Task task, final DimFilter filter, final String metric)
|
||||
@Nullable
|
||||
public Long sumMetric(final Task task, final DimFilter filter, final String metric)
|
||||
{
|
||||
// Do a query.
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
|
@ -1096,6 +1105,10 @@ public class RealtimeIndexTaskTest
|
|||
|
||||
List<Result<TimeseriesResultValue>> results =
|
||||
task.getQueryRunner(query).run(QueryPlus.wrap(query), ImmutableMap.of()).toList();
|
||||
return results.isEmpty() ? 0 : results.get(0).getValue().getLongMetric(metric);
|
||||
if (results.isEmpty()) {
|
||||
return 0L;
|
||||
} else {
|
||||
return results.get(0).getValue().getLongMetric(metric);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,6 +53,10 @@
|
|||
<groupId>org.skife.config</groupId>
|
||||
<artifactId>config-magic</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
|
|
|
@ -41,18 +41,22 @@ public class NullHandling
|
|||
public static final Double ZERO_DOUBLE = 0.0d;
|
||||
public static final Float ZERO_FLOAT = 0.0f;
|
||||
public static final Long ZERO_LONG = 0L;
|
||||
public static final byte IS_NULL_BYTE = (byte) 1;
|
||||
public static final byte IS_NOT_NULL_BYTE = (byte) 0;
|
||||
|
||||
/**
|
||||
* INSTANCE is injected using static injection to avoid adding JacksonInject annotations all over the code.
|
||||
* 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
|
||||
*/
|
||||
@Inject
|
||||
private static NullValueHandlingConfig INSTANCE = new NullValueHandlingConfig(
|
||||
Boolean.valueOf(System.getProperty(NULL_HANDLING_CONFIG_STRING, "true"))
|
||||
);
|
||||
|
||||
/**
|
||||
* whether nulls should be replaced with default value.
|
||||
*/
|
||||
public static boolean replaceWithDefault()
|
||||
{
|
||||
return INSTANCE.isUseDefaultValuesForNull();
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.java.util.common;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Throwables;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -172,4 +173,36 @@ public class StringUtils
|
|||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the given string if it is non-null; the empty string otherwise.
|
||||
* This method should only be used at places where null to empty conversion is
|
||||
* irrelevant to null handling of the data.
|
||||
*
|
||||
* @param string the string to test and possibly return
|
||||
* @return {@code string} itself if it is non-null; {@code ""} if it is null
|
||||
*/
|
||||
public static String nullToEmptyNonDruidDataString(@Nullable String string)
|
||||
{
|
||||
//CHECKSTYLE.OFF: Regexp
|
||||
return Strings.nullToEmpty(string);
|
||||
//CHECKSTYLE.ON: Regexp
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the given string if it is nonempty; {@code null} otherwise.
|
||||
* This method should only be used at places where null to empty conversion is
|
||||
* irrelevant to null handling of the data.
|
||||
*
|
||||
* @param string the string to test and possibly return
|
||||
* @return {@code string} itself if it is nonempty; {@code null} if it is
|
||||
* empty or null
|
||||
*/
|
||||
@Nullable
|
||||
public static String emptyToNullNonDruidDataString(@Nullable String string)
|
||||
{
|
||||
//CHECKSTYLE.OFF: Regexp
|
||||
return Strings.emptyToNull(string);
|
||||
//CHECKSTYLE.ON: Regexp
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,8 +21,8 @@ package io.druid.java.util.common.parsers;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
|
@ -60,10 +60,10 @@ public class ParserUtils
|
|||
return (input) -> {
|
||||
if (input != null && input.contains(listDelimiter)) {
|
||||
return StreamSupport.stream(listSplitter.split(input).spliterator(), false)
|
||||
.map(Strings::emptyToNull)
|
||||
.map(NullHandling::emptyToNullIfNeeded)
|
||||
.collect(Collectors.toList());
|
||||
} else {
|
||||
return Strings.emptyToNull(input);
|
||||
return NullHandling.emptyToNullIfNeeded(input);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.java.util.http.client;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -140,8 +139,7 @@ public class NettyHttpClient extends AbstractHttpClient
|
|||
} else {
|
||||
channel = channelFuture.getChannel();
|
||||
}
|
||||
|
||||
final String urlFile = Strings.nullToEmpty(url.getFile());
|
||||
final String urlFile = StringUtils.nullToEmptyNonDruidDataString(url.getFile());
|
||||
final HttpRequest httpRequest = new DefaultHttpRequest(
|
||||
HttpVersion.HTTP_1_1,
|
||||
method,
|
||||
|
|
|
@ -41,6 +41,7 @@ public class GuiceInjectors
|
|||
new JacksonModule(),
|
||||
new PropertiesModule(Arrays.asList("common.runtime.properties", "runtime.properties")),
|
||||
new ConfigModule(),
|
||||
new NullHandlingModule(),
|
||||
binder -> {
|
||||
binder.bind(DruidSecondaryModule.class);
|
||||
JsonConfigProvider.bind(binder, "druid.extensions", ExtensionsConfig.class);
|
||||
|
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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.guice;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.common.config.NullValueHandlingConfig;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class NullHandlingModule implements Module
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
JsonConfigProvider.bind(binder, "druid.generic", NullValueHandlingConfig.class);
|
||||
binder.requestStaticInjection(NullHandling.class);
|
||||
binder.requestStaticInjection(NullHandling.class);
|
||||
}
|
||||
}
|
|
@ -21,11 +21,11 @@ package io.druid.query;
|
|||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import io.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.query.filter.Filter;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -118,7 +118,7 @@ public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMet
|
|||
@Override
|
||||
public void queryId(QueryType query)
|
||||
{
|
||||
setDimension(DruidMetrics.ID, Strings.nullToEmpty(query.getId()));
|
||||
setDimension(DruidMetrics.ID, StringUtils.nullToEmptyNonDruidDataString(query.getId()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
import io.druid.guice.annotations.ExtensionPoint;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
|
||||
|
@ -39,6 +40,7 @@ import io.druid.segment.ColumnValueSelector;
|
|||
* @see DoubleAggregateCombiner
|
||||
* @see ObjectAggregateCombiner
|
||||
*/
|
||||
@ExtensionPoint
|
||||
public interface AggregateCombiner<T> extends ColumnValueSelector<T>
|
||||
{
|
||||
/**
|
||||
|
|
|
@ -37,6 +37,8 @@ import java.util.Map;
|
|||
* AggregatorFactory is a strategy (in the terms of Design Patterns) that represents column aggregation, e. g. min,
|
||||
* max, sum of metric columns, or cardinality of dimension columns (see {@link
|
||||
* io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory}).
|
||||
* Implementations of {@link AggregatorFactory} which need to Support Nullable Aggregations are encouraged
|
||||
* to extend {@link NullableAggregatorFactory}.
|
||||
*/
|
||||
@ExtensionPoint
|
||||
public abstract class AggregatorFactory implements Cacheable
|
||||
|
@ -60,7 +62,8 @@ public abstract class AggregatorFactory implements Cacheable
|
|||
*
|
||||
* @return an object representing the combination of lhs and rhs, this can be a new object or a mutation of the inputs
|
||||
*/
|
||||
public abstract Object combine(Object lhs, Object rhs);
|
||||
@Nullable
|
||||
public abstract Object combine(@Nullable Object lhs, @Nullable Object rhs);
|
||||
|
||||
/**
|
||||
* Creates an AggregateCombiner to fold rollup aggregation results from serveral "rows" of different indexes during
|
||||
|
@ -76,6 +79,20 @@ public abstract class AggregatorFactory implements Cacheable
|
|||
throw new UOE("[%s] does not implement makeAggregateCombiner()", this.getClass().getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an {@link AggregateCombiner} which supports nullability.
|
||||
* Implementations of {@link AggregatorFactory} which need to Support Nullable Aggregations are encouraged
|
||||
* to extend {@link NullableAggregatorFactory} instead of overriding this method.
|
||||
* Default implementation calls {@link #makeAggregateCombiner()} for backwards compatibility.
|
||||
*
|
||||
* @see AggregateCombiner
|
||||
* @see NullableAggregatorFactory
|
||||
*/
|
||||
public AggregateCombiner makeNullableAggregateCombiner()
|
||||
{
|
||||
return makeAggregateCombiner();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an AggregatorFactory that can be used to combine the output of aggregators from this factory. This
|
||||
* generally amounts to simply creating a new factory that is the same as the current except with its input
|
||||
|
@ -127,7 +144,8 @@ public abstract class AggregatorFactory implements Cacheable
|
|||
*
|
||||
* @return the finalized value that should be returned for the initial query
|
||||
*/
|
||||
public abstract Object finalizeComputation(Object object);
|
||||
@Nullable
|
||||
public abstract Object finalizeComputation(@Nullable Object object);
|
||||
|
||||
public abstract String getName();
|
||||
|
||||
|
@ -142,6 +160,19 @@ public abstract class AggregatorFactory implements Cacheable
|
|||
*/
|
||||
public abstract int getMaxIntermediateSize();
|
||||
|
||||
/**
|
||||
* Returns the maximum size that this aggregator will require in bytes for intermediate storage of results.
|
||||
* Implementations of {@link AggregatorFactory} which need to Support Nullable Aggregations are encouraged
|
||||
* to extend {@link NullableAggregatorFactory} instead of overriding this method.
|
||||
* Default implementation calls {@link #makeAggregateCombiner()} for backwards compatibility.
|
||||
*
|
||||
* @return the maximum number of bytes that an aggregator of this type will require for intermediate result storage.
|
||||
*/
|
||||
public int getMaxIntermediateSizeWithNulls()
|
||||
{
|
||||
return getMaxIntermediateSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a potentially optimized form of this AggregatorFactory for per-segment queries.
|
||||
*/
|
||||
|
|
|
@ -174,8 +174,10 @@ public class AggregatorUtil
|
|||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
// Although baseSelector.getObject is nullable
|
||||
// exprEval returned from Expression selectors is never null.
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull() ? nullValue : (float) exprEval.asDouble();
|
||||
return exprEval.isNumericNull() ? nullValue : (float) exprEval.asDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -188,7 +190,7 @@ public class AggregatorUtil
|
|||
public boolean isNull()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull();
|
||||
return exprEval == null || exprEval.isNumericNull();
|
||||
}
|
||||
}
|
||||
return new ExpressionFloatColumnSelector();
|
||||
|
@ -216,7 +218,7 @@ public class AggregatorUtil
|
|||
public long getLong()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull() ? nullValue : exprEval.asLong();
|
||||
return exprEval.isNumericNull() ? nullValue : exprEval.asLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -229,7 +231,7 @@ public class AggregatorUtil
|
|||
public boolean isNull()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull();
|
||||
return exprEval == null || exprEval.isNumericNull();
|
||||
}
|
||||
}
|
||||
return new ExpressionLongColumnSelector();
|
||||
|
@ -257,7 +259,7 @@ public class AggregatorUtil
|
|||
public double getDouble()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull() ? nullValue : exprEval.asDouble();
|
||||
return exprEval.isNumericNull() ? nullValue : exprEval.asDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -270,7 +272,7 @@ public class AggregatorUtil
|
|||
public boolean isNull()
|
||||
{
|
||||
final ExprEval exprEval = baseSelector.getObject();
|
||||
return exprEval.isNull();
|
||||
return exprEval == null || exprEval.isNumericNull();
|
||||
}
|
||||
}
|
||||
return new ExpressionDoubleColumnSelector();
|
||||
|
|
|
@ -44,7 +44,7 @@ public interface BufferAggregator extends HotLoopCallee
|
|||
*
|
||||
* <b>Implementations must not change the position, limit or mark of the given buffer</b>
|
||||
*
|
||||
* This method must not exceed the number of bytes returned by {@link AggregatorFactory#getMaxIntermediateSize()}
|
||||
* This method must not exceed the number of bytes returned by {@link AggregatorFactory#getMaxIntermediateSizeWithNulls}
|
||||
* in the corresponding {@link AggregatorFactory}
|
||||
*
|
||||
* @param buf byte buffer to initialize
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class DoubleMaxAggregatorFactory extends SimpleDoubleAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseDoubleColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new DoubleMaxAggregator(getDoubleColumnSelector(metricFactory, Double.NEGATIVE_INFINITY));
|
||||
return getDoubleColumnSelector(
|
||||
metricFactory,
|
||||
Double.NEGATIVE_INFINITY
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleMaxBufferAggregator(getDoubleColumnSelector(metricFactory, Double.NEGATIVE_INFINITY));
|
||||
return new DoubleMaxAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseDoubleColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new DoubleMaxBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return DoubleMaxAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class DoubleMinAggregatorFactory extends SimpleDoubleAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseDoubleColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new DoubleMinAggregator(getDoubleColumnSelector(metricFactory, Double.POSITIVE_INFINITY));
|
||||
return getDoubleColumnSelector(
|
||||
metricFactory,
|
||||
Double.POSITIVE_INFINITY
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleMinBufferAggregator(getDoubleColumnSelector(metricFactory, Double.POSITIVE_INFINITY));
|
||||
return new DoubleMinAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseDoubleColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new DoubleMinBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return DoubleMinAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseDoubleColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new DoubleSumAggregator(getDoubleColumnSelector(metricFactory, 0.0));
|
||||
return getDoubleColumnSelector(
|
||||
metricFactory,
|
||||
0.0d
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseDoubleColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleSumBufferAggregator(getDoubleColumnSelector(metricFactory, 0.0));
|
||||
return new DoubleSumAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseDoubleColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new DoubleSumBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return DoubleSumAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -142,7 +142,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
|
|||
@Override
|
||||
public int getMaxIntermediateSize()
|
||||
{
|
||||
return delegate.getMaxIntermediateSize();
|
||||
return delegate.getMaxIntermediateSizeWithNulls();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseFloatColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class FloatMaxAggregatorFactory extends SimpleFloatAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseFloatColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new FloatMaxAggregator(getFloatColumnSelector(metricFactory, Float.NEGATIVE_INFINITY));
|
||||
return getFloatColumnSelector(
|
||||
metricFactory,
|
||||
Float.NEGATIVE_INFINITY
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return new FloatMaxBufferAggregator(getFloatColumnSelector(metricFactory, Float.NEGATIVE_INFINITY));
|
||||
return new FloatMaxAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseFloatColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new FloatMaxBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return FloatMaxAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseFloatColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class FloatMinAggregatorFactory extends SimpleFloatAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseFloatColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new FloatMinAggregator(getFloatColumnSelector(metricFactory, Float.POSITIVE_INFINITY));
|
||||
return getFloatColumnSelector(
|
||||
metricFactory,
|
||||
Float.POSITIVE_INFINITY
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return new FloatMinBufferAggregator(getFloatColumnSelector(metricFactory, Float.POSITIVE_INFINITY));
|
||||
return new FloatMinAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseFloatColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new FloatMinBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return FloatMinAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseFloatColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseFloatColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new FloatSumAggregator(getFloatColumnSelector(metricFactory, 0.0f));
|
||||
return getFloatColumnSelector(
|
||||
metricFactory,
|
||||
0.0f
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseFloatColumnValueSelector selector)
|
||||
{
|
||||
return new FloatSumBufferAggregator(getFloatColumnSelector(metricFactory, 0.0f));
|
||||
return new FloatSumAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseFloatColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new FloatSumBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return FloatSumAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseLongColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class LongMaxAggregatorFactory extends SimpleLongAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseLongColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new LongMaxAggregator(getLongColumnSelector(metricFactory, Long.MIN_VALUE));
|
||||
return getLongColumnSelector(
|
||||
metricFactory,
|
||||
Long.MIN_VALUE
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return new LongMaxBufferAggregator(getLongColumnSelector(metricFactory, Long.MIN_VALUE));
|
||||
return new LongMaxAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseLongColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new LongMaxBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return LongMaxAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseLongColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,36 @@ public class LongMinAggregatorFactory extends SimpleLongAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseLongColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new LongMinAggregator(getLongColumnSelector(metricFactory, Long.MAX_VALUE));
|
||||
return getLongColumnSelector(
|
||||
metricFactory,
|
||||
Long.MAX_VALUE
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return new LongMinBufferAggregator(getLongColumnSelector(metricFactory, Long.MAX_VALUE));
|
||||
return new LongMinAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return new LongMinBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return LongMinAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,8 +24,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.segment.BaseLongColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -51,20 +53,39 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseLongColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return new LongSumAggregator(getLongColumnSelector(metricFactory, 0L));
|
||||
return getLongColumnSelector(
|
||||
metricFactory,
|
||||
0L
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
|
||||
{
|
||||
return new LongSumBufferAggregator(getLongColumnSelector(metricFactory, 0L));
|
||||
return new LongSumAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
protected BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
BaseLongColumnValueSelector selector
|
||||
)
|
||||
{
|
||||
return new LongSumBufferAggregator(selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return LongSumAggregator.combineValues(lhs, rhs);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
import io.druid.guice.annotations.PublicApi;
|
||||
import io.druid.segment.BaseNullableColumnValueSelector;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* The result of a NullableAggregateCombiner will be null if all the values to be combined are null values or no values
|
||||
* are combined at all. If any of the value is non-null, the result would be the value of the delegate combiner.
|
||||
* Note that the delegate combiner is not required to perform check for {@link BaseNullableColumnValueSelector#isNull()}
|
||||
* on the selector as only non-null values will be passed to the delegate combiner.
|
||||
* This class is only used when SQL compatible null handling is enabled.
|
||||
*/
|
||||
@PublicApi
|
||||
public final class NullableAggregateCombiner<T> implements AggregateCombiner<T>
|
||||
{
|
||||
private boolean isNullResult = true;
|
||||
|
||||
private final AggregateCombiner<T> delegate;
|
||||
|
||||
public NullableAggregateCombiner(AggregateCombiner<T> delegate)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset(ColumnValueSelector selector)
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
isNullResult = true;
|
||||
} else {
|
||||
isNullResult = false;
|
||||
delegate.reset(selector);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fold(ColumnValueSelector selector)
|
||||
{
|
||||
boolean isNotNull = !selector.isNull();
|
||||
if (isNotNull) {
|
||||
if (isNullResult) {
|
||||
isNullResult = false;
|
||||
delegate.reset(selector);
|
||||
} else {
|
||||
delegate.fold(selector);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
if (isNullResult) {
|
||||
throw new IllegalStateException("Cannot return primitive float for Null Value");
|
||||
}
|
||||
return delegate.getFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
if (isNullResult) {
|
||||
throw new IllegalStateException("Cannot return double for Null Value");
|
||||
}
|
||||
return delegate.getDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
if (isNullResult) {
|
||||
throw new IllegalStateException("Cannot return long for Null Value");
|
||||
}
|
||||
return delegate.getLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return isNullResult || delegate.isNull();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public T getObject()
|
||||
{
|
||||
return isNullResult ? null : delegate.getObject();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return delegate.classOfObject();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
import io.druid.guice.annotations.PublicApi;
|
||||
import io.druid.segment.BaseNullableColumnValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* The result of a NullableAggregator will be null if all the values to be aggregated are null values
|
||||
* or no values are aggregated at all. If any of the value is non-null, the result would be the aggregated
|
||||
* value of the delegate aggregator. Note that the delegate aggregator is not required to perform check for
|
||||
* {@link BaseNullableColumnValueSelector#isNull()} on the selector as only non-null values will be passed
|
||||
* to the delegate aggregator. This class is only used when SQL compatible null handling is enabled.
|
||||
*/
|
||||
@PublicApi
|
||||
public final class NullableAggregator implements Aggregator
|
||||
{
|
||||
private final Aggregator delegate;
|
||||
private final BaseNullableColumnValueSelector selector;
|
||||
private boolean isNullResult = true;
|
||||
|
||||
public NullableAggregator(Aggregator delegate, BaseNullableColumnValueSelector selector)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
this.selector = selector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
boolean isNotNull = !selector.isNull();
|
||||
if (isNotNull) {
|
||||
if (isNullResult) {
|
||||
isNullResult = false;
|
||||
}
|
||||
delegate.aggregate();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object get()
|
||||
{
|
||||
if (isNullResult) {
|
||||
return null;
|
||||
}
|
||||
return delegate.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
if (isNullResult) {
|
||||
throw new IllegalStateException("Cannot return float for Null Value");
|
||||
}
|
||||
return delegate.getFloat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
if (isNullResult) {
|
||||
throw new IllegalStateException("Cannot return long for Null Value");
|
||||
}
|
||||
return delegate.getLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
if (isNullResult) {
|
||||
throw new IllegalStateException("Cannot return double for Null Value");
|
||||
}
|
||||
return delegate.getDouble();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
return isNullResult || delegate.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
delegate.close();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,95 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.guice.annotations.ExtensionPoint;
|
||||
import io.druid.segment.BaseNullableColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
|
||||
/**
|
||||
* Abstract class with functionality to wrap {@link Aggregator}, {@link BufferAggregator} and {@link AggregateCombiner}
|
||||
* to support nullable aggregations for SQL compatibility. Implementations of {@link AggregatorFactory} which need to
|
||||
* Support Nullable Aggregations are encouraged to extend this class.
|
||||
*/
|
||||
@ExtensionPoint
|
||||
public abstract class NullableAggregatorFactory<T extends BaseNullableColumnValueSelector> extends AggregatorFactory
|
||||
{
|
||||
@Override
|
||||
public final Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
T selector = selector(metricFactory);
|
||||
Aggregator aggregator = factorize(metricFactory, selector);
|
||||
return NullHandling.replaceWithDefault() ? aggregator : new NullableAggregator(aggregator, selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
T selector = selector(metricFactory);
|
||||
BufferAggregator aggregator = factorizeBuffered(metricFactory, selector);
|
||||
return NullHandling.replaceWithDefault() ? aggregator : new NullableBufferAggregator(aggregator, selector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final AggregateCombiner makeNullableAggregateCombiner()
|
||||
{
|
||||
AggregateCombiner combiner = makeAggregateCombiner();
|
||||
return NullHandling.replaceWithDefault() ? combiner : new NullableAggregateCombiner(combiner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final int getMaxIntermediateSizeWithNulls()
|
||||
{
|
||||
return getMaxIntermediateSize() + (NullHandling.replaceWithDefault() ? 0 : Byte.BYTES);
|
||||
}
|
||||
|
||||
// ---- ABSTRACT METHODS BELOW ------
|
||||
|
||||
/**
|
||||
* Creates a {@link ColumnValueSelector} for the aggregated column.
|
||||
*
|
||||
* @see ColumnValueSelector
|
||||
*/
|
||||
protected abstract T selector(ColumnSelectorFactory metricFactory);
|
||||
|
||||
/**
|
||||
* Creates an {@link Aggregator} to aggregate values from several rows, by using the provided selector.
|
||||
* @param metricFactory metricFactory
|
||||
* @param selector {@link ColumnValueSelector} for the column to aggregate.
|
||||
*
|
||||
* @see Aggregator
|
||||
*/
|
||||
protected abstract Aggregator factorize(ColumnSelectorFactory metricFactory, T selector);
|
||||
|
||||
/**
|
||||
* Creates an {@link BufferAggregator} to aggregate values from several rows into a ByteBuffer.
|
||||
* @param metricFactory metricFactory
|
||||
* @param selector {@link ColumnValueSelector} for the column to aggregate.
|
||||
*
|
||||
* @see BufferAggregator
|
||||
*/
|
||||
protected abstract BufferAggregator factorizeBuffered(
|
||||
ColumnSelectorFactory metricFactory,
|
||||
T selector
|
||||
);
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation;
|
||||
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.guice.annotations.PublicApi;
|
||||
import io.druid.segment.BaseNullableColumnValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
* The result of a NullableBufferAggregator will be null if all the values to be aggregated are null values or no values
|
||||
* are aggregated at all. If any of the value is non-null, the result would be the aggregated value of the delegate
|
||||
* aggregator. Note that the delegate aggregator is not required to perform check for
|
||||
* {@link BaseNullableColumnValueSelector#isNull()} on the selector as only non-null values will be passed to the
|
||||
* delegate aggregator. This class is only used when SQL compatible null handling is enabled.
|
||||
* When writing aggregated result to buffer, it will write an additional byte to store the nullability of the
|
||||
* aggregated result.
|
||||
* Buffer Layout - 1 byte for storing nullability + delegate storage bytes.
|
||||
*/
|
||||
@PublicApi
|
||||
public final class NullableBufferAggregator implements BufferAggregator
|
||||
{
|
||||
|
||||
private final BufferAggregator delegate;
|
||||
private final BaseNullableColumnValueSelector selector;
|
||||
|
||||
public NullableBufferAggregator(BufferAggregator delegate, BaseNullableColumnValueSelector selector)
|
||||
{
|
||||
this.delegate = delegate;
|
||||
this.selector = selector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(ByteBuffer buf, int position)
|
||||
{
|
||||
buf.put(position, NullHandling.IS_NULL_BYTE);
|
||||
delegate.init(buf, position + Byte.BYTES);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
boolean isNotNull = !selector.isNull();
|
||||
if (isNotNull) {
|
||||
if (buf.get(position) == NullHandling.IS_NULL_BYTE) {
|
||||
buf.put(position, NullHandling.IS_NOT_NULL_BYTE);
|
||||
}
|
||||
delegate.aggregate(buf, position + Byte.BYTES);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public Object get(ByteBuffer buf, int position)
|
||||
{
|
||||
if (buf.get(position) == NullHandling.IS_NULL_BYTE) {
|
||||
return null;
|
||||
}
|
||||
return delegate.get(buf, position + Byte.BYTES);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(ByteBuffer buf, int position)
|
||||
{
|
||||
if (buf.get(position) == NullHandling.IS_NULL_BYTE) {
|
||||
throw new IllegalStateException("Cannot return float for Null Value");
|
||||
}
|
||||
return delegate.getFloat(buf, position + Byte.BYTES);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(ByteBuffer buf, int position)
|
||||
{
|
||||
if (buf.get(position) == NullHandling.IS_NULL_BYTE) {
|
||||
throw new IllegalStateException("Cannot return long for Null Value");
|
||||
}
|
||||
return delegate.getLong(buf, position + Byte.BYTES);
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble(ByteBuffer buf, int position)
|
||||
{
|
||||
if (buf.get(position) == NullHandling.IS_NULL_BYTE) {
|
||||
throw new IllegalStateException("Cannot return double for Null Value");
|
||||
}
|
||||
return delegate.getDouble(buf, position + Byte.BYTES);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull(ByteBuffer buf, int position)
|
||||
{
|
||||
return buf.get(position) == NullHandling.IS_NULL_BYTE || delegate.isNull(buf, position + Byte.BYTES);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
delegate.close();
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ package io.druid.query.aggregation;
|
|||
import io.druid.guice.annotations.ExtensionPoint;
|
||||
import io.druid.java.util.common.Cacheable;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -36,6 +37,7 @@ public interface PostAggregator extends Cacheable
|
|||
|
||||
Comparator getComparator();
|
||||
|
||||
@Nullable
|
||||
Object compute(Map<String, Object> combinedAggregators);
|
||||
|
||||
String getName();
|
||||
|
|
|
@ -28,12 +28,13 @@ import io.druid.segment.BaseDoubleColumnValueSelector;
|
|||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class SimpleDoubleAggregatorFactory extends AggregatorFactory
|
||||
public abstract class SimpleDoubleAggregatorFactory extends NullableAggregatorFactory<BaseDoubleColumnValueSelector>
|
||||
{
|
||||
protected final String name;
|
||||
protected final String fieldName;
|
||||
|
@ -103,7 +104,8 @@ public abstract class SimpleDoubleAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return object;
|
||||
}
|
||||
|
|
|
@ -27,12 +27,13 @@ import io.druid.math.expr.Parser;
|
|||
import io.druid.segment.BaseFloatColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class SimpleFloatAggregatorFactory extends AggregatorFactory
|
||||
public abstract class SimpleFloatAggregatorFactory extends NullableAggregatorFactory<BaseFloatColumnValueSelector>
|
||||
{
|
||||
protected final String name;
|
||||
protected final String fieldName;
|
||||
|
@ -97,7 +98,8 @@ public abstract class SimpleFloatAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return object;
|
||||
}
|
||||
|
|
|
@ -27,12 +27,13 @@ import io.druid.math.expr.Parser;
|
|||
import io.druid.segment.BaseLongColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class SimpleLongAggregatorFactory extends AggregatorFactory
|
||||
public abstract class SimpleLongAggregatorFactory extends NullableAggregatorFactory<BaseLongColumnValueSelector>
|
||||
{
|
||||
protected final String name;
|
||||
protected final String fieldName;
|
||||
|
@ -93,7 +94,8 @@ public abstract class SimpleLongAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return object;
|
||||
}
|
||||
|
|
|
@ -20,23 +20,33 @@
|
|||
package io.druid.query.aggregation.cardinality.types;
|
||||
|
||||
import com.google.common.hash.Hasher;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.hll.HyperLogLogCollector;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
|
||||
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||
|
||||
|
||||
/**
|
||||
* If performance of this class appears to be a bottleneck for somebody,
|
||||
* one simple way to improve it is to split it into two different classes,
|
||||
* one that is used when {@link NullHandling#replaceWithDefault()} is false,
|
||||
* and one - when it's true, moving this computation out of the tight loop
|
||||
*/
|
||||
public class DoubleCardinalityAggregatorColumnSelectorStrategy
|
||||
implements CardinalityAggregatorColumnSelectorStrategy<BaseDoubleColumnValueSelector>
|
||||
{
|
||||
@Override
|
||||
public void hashRow(BaseDoubleColumnValueSelector dimSelector, Hasher hasher)
|
||||
public void hashRow(BaseDoubleColumnValueSelector selector, Hasher hasher)
|
||||
{
|
||||
hasher.putDouble(dimSelector.getDouble());
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
hasher.putDouble(selector.getDouble());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void hashValues(BaseDoubleColumnValueSelector dimSelector, HyperLogLogCollector collector)
|
||||
public void hashValues(BaseDoubleColumnValueSelector selector, HyperLogLogCollector collector)
|
||||
{
|
||||
collector.add(CardinalityAggregator.hashFn.hashLong(Double.doubleToLongBits(dimSelector.getDouble())).asBytes());
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
collector.add(CardinalityAggregator.hashFn.hashLong(Double.doubleToLongBits(selector.getDouble())).asBytes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,22 +20,33 @@
|
|||
package io.druid.query.aggregation.cardinality.types;
|
||||
|
||||
import com.google.common.hash.Hasher;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.hll.HyperLogLogCollector;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
|
||||
import io.druid.segment.BaseFloatColumnValueSelector;
|
||||
|
||||
/**
|
||||
* If performance of this class appears to be a bottleneck for somebody,
|
||||
* one simple way to improve it is to split it into two different classes,
|
||||
* one that is used when {@link NullHandling#replaceWithDefault()} is false,
|
||||
* and one - when it's true, moving this computation out of the tight loop
|
||||
*/
|
||||
public class FloatCardinalityAggregatorColumnSelectorStrategy
|
||||
implements CardinalityAggregatorColumnSelectorStrategy<BaseFloatColumnValueSelector>
|
||||
{
|
||||
@Override
|
||||
public void hashRow(BaseFloatColumnValueSelector selector, Hasher hasher)
|
||||
{
|
||||
hasher.putFloat(selector.getFloat());
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
hasher.putFloat(selector.getFloat());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void hashValues(BaseFloatColumnValueSelector selector, HyperLogLogCollector collector)
|
||||
{
|
||||
collector.add(CardinalityAggregator.hashFn.hashInt(Float.floatToIntBits(selector.getFloat())).asBytes());
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
collector.add(CardinalityAggregator.hashFn.hashInt(Float.floatToIntBits(selector.getFloat())).asBytes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,22 +20,33 @@
|
|||
package io.druid.query.aggregation.cardinality.types;
|
||||
|
||||
import com.google.common.hash.Hasher;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.hll.HyperLogLogCollector;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
|
||||
import io.druid.segment.BaseLongColumnValueSelector;
|
||||
|
||||
/**
|
||||
* If performance of this class appears to be a bottleneck for somebody,
|
||||
* one simple way to improve it is to split it into two different classes,
|
||||
* one that is used when {@link NullHandling#replaceWithDefault()} is false,
|
||||
* and one - when it's true, moving this computation out of the tight loop
|
||||
*/
|
||||
public class LongCardinalityAggregatorColumnSelectorStrategy
|
||||
implements CardinalityAggregatorColumnSelectorStrategy<BaseLongColumnValueSelector>
|
||||
{
|
||||
@Override
|
||||
public void hashRow(BaseLongColumnValueSelector dimSelector, Hasher hasher)
|
||||
public void hashRow(BaseLongColumnValueSelector selector, Hasher hasher)
|
||||
{
|
||||
hasher.putLong(dimSelector.getLong());
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
hasher.putLong(selector.getLong());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void hashValues(BaseLongColumnValueSelector dimSelector, HyperLogLogCollector collector)
|
||||
public void hashValues(BaseLongColumnValueSelector selector, HyperLogLogCollector collector)
|
||||
{
|
||||
collector.add(CardinalityAggregator.hashFn.hashLong(dimSelector.getLong()).asBytes());
|
||||
if (NullHandling.replaceWithDefault() || !selector.isNull()) {
|
||||
collector.add(CardinalityAggregator.hashFn.hashLong(selector.getLong()).asBytes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.query.aggregation.cardinality.types;
|
||||
|
||||
import com.google.common.hash.Hasher;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.hll.HyperLogLogCollector;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
|
@ -40,20 +41,34 @@ public class StringCardinalityAggregatorColumnSelectorStrategy implements Cardin
|
|||
// nothing to add to hasher if size == 0, only handle size == 1 and size != 0 cases.
|
||||
if (size == 1) {
|
||||
final String value = dimSelector.lookupName(row.get(0));
|
||||
hasher.putUnencodedChars(nullToSpecial(value));
|
||||
if (NullHandling.replaceWithDefault() || value != null) {
|
||||
hasher.putUnencodedChars(nullToSpecial(value));
|
||||
}
|
||||
} else if (size != 0) {
|
||||
boolean hasNonNullValue = false;
|
||||
final String[] values = new String[size];
|
||||
for (int i = 0; i < size; ++i) {
|
||||
final String value = dimSelector.lookupName(row.get(i));
|
||||
// SQL standard spec does not count null values,
|
||||
// Skip counting null values when we are not replacing null with default value.
|
||||
// A special value for null in case null handling is configured to use empty string for null.
|
||||
if (NullHandling.sqlCompatible() && !hasNonNullValue && value != null) {
|
||||
hasNonNullValue = true;
|
||||
}
|
||||
values[i] = nullToSpecial(value);
|
||||
}
|
||||
// Values need to be sorted to ensure consistent multi-value ordering across different segments
|
||||
Arrays.sort(values);
|
||||
for (int i = 0; i < size; ++i) {
|
||||
if (i != 0) {
|
||||
hasher.putChar(CARDINALITY_AGG_SEPARATOR);
|
||||
// SQL standard spec does not count null values,
|
||||
// Skip counting null values when we are not replacing null with default value.
|
||||
// A special value for null in case null handling is configured to use empty string for null.
|
||||
if (NullHandling.replaceWithDefault() || hasNonNullValue) {
|
||||
// Values need to be sorted to ensure consistent multi-value ordering across different segments
|
||||
Arrays.sort(values);
|
||||
for (int i = 0; i < size; ++i) {
|
||||
if (i != 0) {
|
||||
hasher.putChar(CARDINALITY_AGG_SEPARATOR);
|
||||
}
|
||||
hasher.putUnencodedChars(values[i]);
|
||||
}
|
||||
hasher.putUnencodedChars(values[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -65,7 +80,12 @@ public class StringCardinalityAggregatorColumnSelectorStrategy implements Cardin
|
|||
for (int i = 0, rowSize = row.size(); i < rowSize; i++) {
|
||||
int index = row.get(i);
|
||||
final String value = dimSelector.lookupName(index);
|
||||
collector.add(CardinalityAggregator.hashFn.hashUnencodedChars(nullToSpecial(value)).asBytes());
|
||||
// SQL standard spec does not count null values,
|
||||
// Skip counting null values when we are not replacing null with default value.
|
||||
// A special value for null in case null handling is configured to use empty string for null.
|
||||
if (NullHandling.replaceWithDefault() || value != null) {
|
||||
collector.add(CardinalityAggregator.hashFn.hashUnencodedChars(nullToSpecial(value)).asBytes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -32,11 +32,13 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -45,7 +47,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
||||
public class DoubleFirstAggregatorFactory extends NullableAggregatorFactory<ColumnValueSelector>
|
||||
{
|
||||
public static final Comparator VALUE_COMPARATOR = (o1, o2) -> Doubles.compare(
|
||||
((SerializablePair<Long, Double>) o1).rhs,
|
||||
|
@ -76,20 +78,26 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleFirstAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleFirstBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -100,8 +108,15 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return TIME_COMPARATOR.compare(lhs, rhs) <= 0 ? lhs : rhs;
|
||||
}
|
||||
|
||||
|
@ -117,9 +132,8 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
|||
return new DoubleFirstAggregatorFactory(name, name)
|
||||
{
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new DoubleFirstAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -135,9 +149,8 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new DoubleFirstBufferAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -175,9 +188,10 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return ((SerializablePair<Long, Double>) object).rhs;
|
||||
return object == null ? null : ((SerializablePair<Long, Double>) object).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -32,11 +32,13 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -45,7 +47,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class FloatFirstAggregatorFactory extends AggregatorFactory
|
||||
public class FloatFirstAggregatorFactory extends NullableAggregatorFactory<ColumnValueSelector>
|
||||
{
|
||||
public static final Comparator VALUE_COMPARATOR = (o1, o2) -> Doubles.compare(
|
||||
((SerializablePair<Long, Float>) o1).rhs,
|
||||
|
@ -74,20 +76,26 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new FloatFirstAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new FloatFirstBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -98,8 +106,15 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return TIME_COMPARATOR.compare(lhs, rhs) <= 0 ? lhs : rhs;
|
||||
}
|
||||
|
||||
|
@ -115,9 +130,8 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
|||
return new FloatFirstAggregatorFactory(name, name)
|
||||
{
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new FloatFirstAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -133,9 +147,8 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new FloatFirstBufferAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -173,9 +186,10 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return ((SerializablePair<Long, Float>) object).rhs;
|
||||
return object == null ? null : ((SerializablePair<Long, Float>) object).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -31,11 +31,13 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -43,7 +45,7 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class LongFirstAggregatorFactory extends AggregatorFactory
|
||||
public class LongFirstAggregatorFactory extends NullableAggregatorFactory<ColumnValueSelector>
|
||||
{
|
||||
public static final Comparator VALUE_COMPARATOR = (o1, o2) -> Longs.compare(
|
||||
((SerializablePair<Long, Long>) o1).rhs,
|
||||
|
@ -67,20 +69,26 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new LongFirstAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new LongFirstBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -91,8 +99,15 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) <= 0 ? lhs : rhs;
|
||||
}
|
||||
|
||||
|
@ -108,9 +123,8 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
|||
return new LongFirstAggregatorFactory(name, name)
|
||||
{
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new LongFirstAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -126,9 +140,8 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new LongFirstBufferAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -166,9 +179,10 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return ((SerializablePair<Long, Long>) object).rhs;
|
||||
return object == null ? null : ((SerializablePair<Long, Long>) object).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -29,8 +29,10 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.aggregation.SerializablePairLongString;
|
||||
import io.druid.query.cache.CacheKeyBuilder;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
|
@ -42,7 +44,7 @@ import java.util.Map;
|
|||
import java.util.Objects;
|
||||
|
||||
@JsonTypeName("stringFirst")
|
||||
public class StringFirstAggregatorFactory extends AggregatorFactory
|
||||
public class StringFirstAggregatorFactory extends NullableAggregatorFactory<BaseObjectColumnValueSelector>
|
||||
{
|
||||
public static final int DEFAULT_MAX_STRING_SIZE = 1024;
|
||||
|
||||
|
@ -107,21 +109,27 @@ public class StringFirstAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseObjectColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
return new StringFirstAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName),
|
||||
selector,
|
||||
maxStringBytes
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
return new StringFirstBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName),
|
||||
selector,
|
||||
maxStringBytes
|
||||
);
|
||||
}
|
||||
|
|
|
@ -44,9 +44,8 @@ public class StringFirstFoldingAggregatorFactory extends StringFirstAggregatorFa
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName());
|
||||
return new StringFirstAggregator(null, null, maxStringBytes)
|
||||
{
|
||||
@Override
|
||||
|
@ -62,9 +61,8 @@ public class StringFirstFoldingAggregatorFactory extends StringFirstAggregatorFa
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName());
|
||||
return new StringFirstBufferAggregator(null, null, maxStringBytes)
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -30,13 +30,15 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory;
|
||||
import io.druid.query.aggregation.first.LongFirstAggregatorFactory;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -45,7 +47,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DoubleLastAggregatorFactory extends AggregatorFactory
|
||||
public class DoubleLastAggregatorFactory extends NullableAggregatorFactory<ColumnValueSelector>
|
||||
{
|
||||
|
||||
private final String fieldName;
|
||||
|
@ -66,20 +68,26 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleLastAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new DoubleLastBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -90,8 +98,15 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs;
|
||||
}
|
||||
|
||||
|
@ -107,9 +122,8 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
|||
return new DoubleLastAggregatorFactory(name, name)
|
||||
{
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new DoubleLastAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -125,9 +139,8 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new DoubleLastBufferAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -165,9 +178,10 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return ((SerializablePair<Long, Double>) object).rhs;
|
||||
return object == null ? null : ((SerializablePair<Long, Double>) object).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -30,13 +30,15 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.aggregation.first.FloatFirstAggregatorFactory;
|
||||
import io.druid.query.aggregation.first.LongFirstAggregatorFactory;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -45,7 +47,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class FloatLastAggregatorFactory extends AggregatorFactory
|
||||
public class FloatLastAggregatorFactory extends NullableAggregatorFactory<ColumnValueSelector>
|
||||
{
|
||||
|
||||
private final String fieldName;
|
||||
|
@ -64,20 +66,26 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new FloatLastAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new FloatLastBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -88,8 +96,15 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return FloatFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs;
|
||||
}
|
||||
|
||||
|
@ -105,9 +120,8 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
|||
return new FloatLastAggregatorFactory(name, name)
|
||||
{
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new FloatLastAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -123,9 +137,8 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new FloatLastBufferAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -163,9 +176,10 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return ((SerializablePair<Long, Float>) object).rhs;
|
||||
return object == null ? null : ((SerializablePair<Long, Float>) object).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -30,13 +30,15 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory;
|
||||
import io.druid.query.aggregation.first.LongFirstAggregatorFactory;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -45,7 +47,7 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public class LongLastAggregatorFactory extends AggregatorFactory
|
||||
public class LongLastAggregatorFactory extends NullableAggregatorFactory<ColumnValueSelector>
|
||||
{
|
||||
private final String fieldName;
|
||||
private final String name;
|
||||
|
@ -63,20 +65,26 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new LongLastAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
return new LongLastBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName)
|
||||
selector
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -87,8 +95,15 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object combine(Object lhs, Object rhs)
|
||||
@Nullable
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs;
|
||||
}
|
||||
|
||||
|
@ -104,9 +119,8 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
|||
return new LongLastAggregatorFactory(name, name)
|
||||
{
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new LongLastAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -122,9 +136,8 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name);
|
||||
return new LongLastBufferAggregator(null, null)
|
||||
{
|
||||
@Override
|
||||
|
@ -162,9 +175,10 @@ public class LongLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
@Nullable
|
||||
public Object finalizeComputation(@Nullable Object object)
|
||||
{
|
||||
return ((SerializablePair<Long, Long>) object).rhs;
|
||||
return object == null ? null : ((SerializablePair<Long, Long>) object).rhs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -28,9 +28,11 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.AggregatorUtil;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.query.aggregation.NullableAggregatorFactory;
|
||||
import io.druid.query.aggregation.SerializablePairLongString;
|
||||
import io.druid.query.aggregation.first.StringFirstAggregatorFactory;
|
||||
import io.druid.query.cache.CacheKeyBuilder;
|
||||
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
|
@ -43,7 +45,7 @@ import java.util.Objects;
|
|||
|
||||
|
||||
@JsonTypeName("stringLast")
|
||||
public class StringLastAggregatorFactory extends AggregatorFactory
|
||||
public class StringLastAggregatorFactory extends NullableAggregatorFactory<BaseObjectColumnValueSelector>
|
||||
{
|
||||
private final String fieldName;
|
||||
private final String name;
|
||||
|
@ -66,21 +68,27 @@ public class StringLastAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
protected BaseObjectColumnValueSelector selector(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
return metricFactory.makeColumnValueSelector(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
return new StringLastAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName),
|
||||
selector,
|
||||
maxStringBytes
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
return new StringLastBufferAggregator(
|
||||
metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME),
|
||||
metricFactory.makeColumnValueSelector(fieldName),
|
||||
selector,
|
||||
maxStringBytes
|
||||
);
|
||||
}
|
||||
|
|
|
@ -44,9 +44,8 @@ public class StringLastFoldingAggregatorFactory extends StringLastAggregatorFact
|
|||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName());
|
||||
return new StringLastAggregator(null, null, maxStringBytes)
|
||||
{
|
||||
@Override
|
||||
|
@ -62,9 +61,8 @@ public class StringLastFoldingAggregatorFactory extends StringLastAggregatorFact
|
|||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, BaseObjectColumnValueSelector selector)
|
||||
{
|
||||
final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName());
|
||||
return new StringLastBufferAggregator(null, null, maxStringBytes)
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -109,11 +110,21 @@ public class ArithmeticPostAggregator implements PostAggregator
|
|||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
double retVal = 0.0;
|
||||
Double retVal = NullHandling.defaultDoubleValue();
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
Number nextVal = (Number) fieldsIter.next().compute(values);
|
||||
if (nextVal == null) {
|
||||
// As per SQL standard if any of the value is null, arithmetic operators will return null.
|
||||
return null;
|
||||
}
|
||||
retVal = nextVal.doubleValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
retVal = op.compute(retVal, ((Number) fieldsIter.next().compute(values)).doubleValue());
|
||||
nextVal = (Number) fieldsIter.next().compute(values);
|
||||
if (nextVal == null) {
|
||||
// As per SQL standard if any of the value is null, arithmetic operators will return null.
|
||||
return null;
|
||||
}
|
||||
retVal = op.compute(retVal, (nextVal).doubleValue());
|
||||
}
|
||||
}
|
||||
return retVal;
|
||||
|
@ -268,7 +279,7 @@ public class ArithmeticPostAggregator implements PostAggregator
|
|||
/**
|
||||
* Ensures the following order: numeric > NaN > Infinite.
|
||||
*
|
||||
* The name may be referenced via Ordering.valueOf(String) in the constructor {@link
|
||||
* The name may be referenced via {@link #valueOf(String)} in the constructor {@link
|
||||
* ArithmeticPostAggregator#ArithmeticPostAggregator(String, String, List, String)}.
|
||||
*/
|
||||
@SuppressWarnings("unused")
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -37,14 +38,9 @@ import java.util.Set;
|
|||
|
||||
public class DoubleGreatestPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return ((Double) o).compareTo((Double) o1);
|
||||
}
|
||||
};
|
||||
private static final Comparator<Number> COMPARATOR = Comparator.nullsFirst(
|
||||
Comparator.comparingDouble(Number::doubleValue)
|
||||
);
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
@ -81,13 +77,15 @@ public class DoubleGreatestPostAggregator implements PostAggregator
|
|||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
double retVal = Double.NEGATIVE_INFINITY;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
double other = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
if (other > retVal) {
|
||||
retVal = other;
|
||||
Double retVal = NullHandling.replaceWithDefault() ? Double.NEGATIVE_INFINITY : null;
|
||||
while (fieldsIter.hasNext()) {
|
||||
Number nextVal = ((Number) fieldsIter.next().compute(values));
|
||||
// Ignore NULL values and return the greatest out of non-null values.
|
||||
if (nextVal != null && (retVal == null || COMPARATOR.compare(nextVal, retVal) > 0)) {
|
||||
if (nextVal instanceof Double) {
|
||||
retVal = (Double) nextVal;
|
||||
} else {
|
||||
retVal = nextVal.doubleValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -37,14 +38,9 @@ import java.util.Set;
|
|||
|
||||
public class DoubleLeastPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return ((Double) o).compareTo((Double) o1);
|
||||
}
|
||||
};
|
||||
private static final Comparator<Number> COMPARATOR = Comparator.nullsLast(
|
||||
Comparator.comparingDouble(Number::doubleValue)
|
||||
);
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
@ -81,13 +77,15 @@ public class DoubleLeastPostAggregator implements PostAggregator
|
|||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
double retVal = Double.POSITIVE_INFINITY;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
double other = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
if (other < retVal) {
|
||||
retVal = other;
|
||||
Double retVal = NullHandling.replaceWithDefault() ? Double.POSITIVE_INFINITY : null;
|
||||
while (fieldsIter.hasNext()) {
|
||||
Number nextVal = ((Number) fieldsIter.next().compute(values));
|
||||
// Ignore NULL values and return the greatest out of non-null values.
|
||||
if (nextVal != null && (retVal == null || COMPARATOR.compare(nextVal, retVal) < 0)) {
|
||||
if (nextVal instanceof Double) {
|
||||
retVal = (Double) nextVal;
|
||||
} else {
|
||||
retVal = nextVal.doubleValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -38,14 +38,9 @@ import java.util.Set;
|
|||
|
||||
public class LongGreatestPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return Longs.compare(((Number) o).longValue(), ((Number) o1).longValue());
|
||||
}
|
||||
};
|
||||
private static final Comparator<Number> COMPARATOR = Comparator.nullsFirst(
|
||||
Comparator.comparingLong(Number::longValue)
|
||||
);
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
@ -82,13 +77,15 @@ public class LongGreatestPostAggregator implements PostAggregator
|
|||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
long retVal = Long.MIN_VALUE;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
long other = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
if (other > retVal) {
|
||||
retVal = other;
|
||||
Long retVal = NullHandling.replaceWithDefault() ? Long.MIN_VALUE : null;
|
||||
while (fieldsIter.hasNext()) {
|
||||
Number nextVal = ((Number) fieldsIter.next().compute(values));
|
||||
// Ignore NULL values and return the greatest out of non-null values.
|
||||
if (nextVal != null && (retVal == null || COMPARATOR.compare(nextVal, retVal) > 0)) {
|
||||
if (nextVal instanceof Long) {
|
||||
retVal = (Long) nextVal;
|
||||
} else {
|
||||
retVal = nextVal.longValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.query.Queries;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -38,14 +38,9 @@ import java.util.Set;
|
|||
|
||||
public class LongLeastPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return Longs.compare(((Number) o).longValue(), ((Number) o1).longValue());
|
||||
}
|
||||
};
|
||||
private static final Comparator<Number> COMPARATOR = Comparator.nullsLast(
|
||||
Comparator.comparingLong(Number::longValue)
|
||||
);
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
@ -82,13 +77,15 @@ public class LongLeastPostAggregator implements PostAggregator
|
|||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
long retVal = Long.MAX_VALUE;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
long other = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
if (other < retVal) {
|
||||
retVal = other;
|
||||
Long retVal = NullHandling.replaceWithDefault() ? Long.MAX_VALUE : null;
|
||||
while (fieldsIter.hasNext()) {
|
||||
Number nextVal = ((Number) fieldsIter.next().compute(values));
|
||||
// Ignore NULL values and return the greatest out of non-null values.
|
||||
if (nextVal != null && (retVal == null || COMPARATOR.compare(nextVal, retVal) < 0)) {
|
||||
if (nextVal instanceof Long) {
|
||||
retVal = (Long) nextVal;
|
||||
} else {
|
||||
retVal = nextVal.longValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
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.StringUtils;
|
||||
import io.druid.query.filter.DimFilterUtils;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
|
@ -106,7 +106,7 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
}
|
||||
} else {
|
||||
for (int i = 0; i < selectorCardinality; i++) {
|
||||
if (values.contains(Strings.nullToEmpty(selector.lookupName(i)))) {
|
||||
if (values.contains(NullHandling.nullToEmptyIfNeeded(selector.lookupName(i)))) {
|
||||
forwardMapping.put(i, count);
|
||||
reverseMapping[count++] = i;
|
||||
}
|
||||
|
@ -137,7 +137,7 @@ public class ListFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
forwardMapping.defaultReturnValue(-1);
|
||||
final int[] reverseMapping = new int[maxPossibleFilteredCardinality];
|
||||
for (int i = 0; i < selectorCardinality; i++) {
|
||||
if (!values.contains(Strings.nullToEmpty(selector.lookupName(i)))) {
|
||||
if (!values.contains(NullHandling.nullToEmptyIfNeeded(selector.lookupName(i)))) {
|
||||
forwardMapping.put(i, count);
|
||||
reverseMapping[count++] = i;
|
||||
}
|
||||
|
|
|
@ -22,7 +22,7 @@ package io.druid.query.dimension;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.query.filter.DimFilterUtils;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
|
@ -76,7 +76,7 @@ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
@Override
|
||||
public boolean apply(@Nullable String input)
|
||||
{
|
||||
return compiledRegex.matcher(Strings.nullToEmpty(input)).matches();
|
||||
return compiledRegex.matcher(NullHandling.nullToEmptyIfNeeded(input)).matches();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -86,7 +86,8 @@ public class RegexFilteredDimensionSpec extends BaseFilteredDimensionSpec
|
|||
final Int2IntOpenHashMap forwardMapping = new Int2IntOpenHashMap();
|
||||
forwardMapping.defaultReturnValue(-1);
|
||||
for (int i = 0; i < selectorCardinality; i++) {
|
||||
if (compiledRegex.matcher(Strings.nullToEmpty(selector.lookupName(i))).matches()) {
|
||||
String val = NullHandling.nullToEmptyIfNeeded(selector.lookupName(i));
|
||||
if (val != null && compiledRegex.matcher(val).matches()) {
|
||||
forwardMapping.put(i, count++);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.query.expression;
|
||||
|
||||
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.granularity.PeriodGranularity;
|
||||
|
@ -73,7 +74,14 @@ public class ExprUtils
|
|||
} else {
|
||||
Chronology chronology = timeZone == null ? ISOChronology.getInstanceUTC() : ISOChronology.getInstance(timeZone);
|
||||
final Object value = originArg.eval(bindings).value();
|
||||
origin = value != null ? new DateTime(value, chronology) : null;
|
||||
if (value instanceof String && NullHandling.isNullOrEquivalent((String) value)) {
|
||||
// We get a blank string here, when sql compatible null handling is enabled
|
||||
// and expression contains empty string for for origin
|
||||
// e.g timestamp_floor(\"__time\",'PT1M','','UTC')
|
||||
origin = null;
|
||||
} else {
|
||||
origin = value != null ? new DateTime(value, chronology) : null;
|
||||
}
|
||||
}
|
||||
|
||||
return new PeriodGranularity(period, origin, timeZone);
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package io.druid.query.expression;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.math.expr.Expr;
|
||||
import io.druid.math.expr.ExprEval;
|
||||
|
@ -63,7 +63,7 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
|
|||
}
|
||||
|
||||
final LikeDimFilter.LikeMatcher likeMatcher = LikeDimFilter.LikeMatcher.from(
|
||||
Strings.nullToEmpty((String) patternExpr.getLiteralValue()),
|
||||
NullHandling.nullToEmptyIfNeeded((String) patternExpr.getLiteralValue()),
|
||||
escapeChar
|
||||
);
|
||||
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package io.druid.query.expression;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.math.expr.Expr;
|
||||
import io.druid.math.expr.ExprEval;
|
||||
|
@ -63,9 +63,10 @@ public class RegexpExtractExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
final Matcher matcher = pattern.matcher(Strings.nullToEmpty(arg.eval(bindings).asString()));
|
||||
String s = arg.eval(bindings).asString();
|
||||
final Matcher matcher = pattern.matcher(NullHandling.nullToEmptyIfNeeded(s));
|
||||
final String retVal = matcher.find() ? matcher.group(index) : null;
|
||||
return ExprEval.of(Strings.emptyToNull(retVal));
|
||||
return ExprEval.of(NullHandling.emptyToNullIfNeeded(retVal));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -67,6 +67,11 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
ExprEval eval = arg.eval(bindings);
|
||||
if (eval.isNumericNull()) {
|
||||
// Return null if the argument if null.
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(granularity.bucketEnd(DateTimes.utc(arg.eval(bindings).asLong())).getMillis());
|
||||
}
|
||||
|
||||
|
|
|
@ -88,7 +88,12 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
final DateTime dateTime = new DateTime(arg.eval(bindings).asLong(), chronology);
|
||||
Object val = arg.eval(bindings).value();
|
||||
if (val == null) {
|
||||
// Return null if the argument if null.
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
final DateTime dateTime = new DateTime(val, chronology);
|
||||
switch (unit) {
|
||||
case EPOCH:
|
||||
return ExprEval.of(dateTime.getMillis() / 1000);
|
||||
|
|
|
@ -92,7 +92,12 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
return ExprEval.of(granularity.bucketStart(DateTimes.utc(arg.eval(bindings).asLong())).getMillis());
|
||||
ExprEval eval = arg.eval(bindings);
|
||||
if (eval.isNumericNull()) {
|
||||
// Return null if the argument if null.
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(granularity.bucketStart(DateTimes.utc(eval.asLong())).getMillis());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -74,6 +74,11 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
ExprEval eval = arg.eval(bindings);
|
||||
if (eval.isNumericNull()) {
|
||||
// Return null if the argument if null.
|
||||
return ExprEval.of(null);
|
||||
}
|
||||
return ExprEval.of(formatter.print(arg.eval(bindings).asLong()));
|
||||
}
|
||||
|
||||
|
|
|
@ -113,7 +113,7 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
|||
{
|
||||
final ExprEval stringEval = stringExpr.eval(bindings);
|
||||
|
||||
if (chars.length == 0 || stringEval.isNull()) {
|
||||
if (chars.length == 0 || stringEval.value() == null) {
|
||||
return stringEval;
|
||||
}
|
||||
|
||||
|
@ -176,13 +176,13 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
|
|||
{
|
||||
final ExprEval stringEval = stringExpr.eval(bindings);
|
||||
|
||||
if (stringEval.isNull()) {
|
||||
if (stringEval.value() == null) {
|
||||
return stringEval;
|
||||
}
|
||||
|
||||
final ExprEval charsEval = charsExpr.eval(bindings);
|
||||
|
||||
if (charsEval.isNull()) {
|
||||
if (charsEval.value() == null) {
|
||||
return stringEval;
|
||||
}
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ package io.druid.query.extraction;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
@ -52,9 +52,9 @@ public abstract class FunctionalExtraction extends DimExtractionFn
|
|||
)
|
||||
{
|
||||
this.retainMissingValue = retainMissingValue;
|
||||
this.replaceMissingValueWith = Strings.emptyToNull(replaceMissingValueWith);
|
||||
this.replaceMissingValueWith = NullHandling.emptyToNullIfNeeded(replaceMissingValueWith);
|
||||
Preconditions.checkArgument(
|
||||
!(this.retainMissingValue && !Strings.isNullOrEmpty(this.replaceMissingValueWith)),
|
||||
!(this.retainMissingValue && !(this.replaceMissingValueWith == null)),
|
||||
"Cannot specify a [replaceMissingValueWith] and set [retainMissingValue] to true"
|
||||
);
|
||||
|
||||
|
@ -69,7 +69,7 @@ public abstract class FunctionalExtraction extends DimExtractionFn
|
|||
public String apply(@Nullable String dimValue)
|
||||
{
|
||||
final String retval = extractionFunction.apply(dimValue);
|
||||
return Strings.isNullOrEmpty(retval) ? Strings.emptyToNull(dimValue) : retval;
|
||||
return NullHandling.isNullOrEquivalent(retval) ? NullHandling.emptyToNullIfNeeded(dimValue) : retval;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
|
@ -79,8 +79,10 @@ public abstract class FunctionalExtraction extends DimExtractionFn
|
|||
@Override
|
||||
public String apply(@Nullable String dimValue)
|
||||
{
|
||||
final String retval = extractionFunction.apply(dimValue);
|
||||
return Strings.isNullOrEmpty(retval) ? FunctionalExtraction.this.replaceMissingValueWith : retval;
|
||||
final String retval = NullHandling.emptyToNullIfNeeded(extractionFunction.apply(dimValue));
|
||||
return retval == null
|
||||
? FunctionalExtraction.this.replaceMissingValueWith
|
||||
: retval;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package io.druid.query.extraction;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
|
@ -42,14 +42,14 @@ public class IdentityExtractionFn implements ExtractionFn
|
|||
@Nullable
|
||||
public String apply(@Nullable Object value)
|
||||
{
|
||||
return value == null ? null : Strings.emptyToNull(value.toString());
|
||||
return value == null ? null : NullHandling.emptyToNullIfNeeded(value.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
@Nullable
|
||||
public String apply(@Nullable String value)
|
||||
{
|
||||
return Strings.emptyToNull(value);
|
||||
return NullHandling.emptyToNullIfNeeded(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -24,7 +24,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import org.mozilla.javascript.Context;
|
||||
|
@ -113,7 +113,7 @@ public class JavaScriptExtractionFn implements ExtractionFn
|
|||
public String apply(@Nullable Object value)
|
||||
{
|
||||
checkAndCompileScript();
|
||||
return Strings.emptyToNull(fn.apply(value));
|
||||
return NullHandling.emptyToNullIfNeeded(fn.apply(value));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -139,7 +139,7 @@ public class JavaScriptExtractionFn implements ExtractionFn
|
|||
@Nullable
|
||||
public String apply(@Nullable String value)
|
||||
{
|
||||
return this.apply((Object) Strings.emptyToNull(value));
|
||||
return this.apply((Object) NullHandling.emptyToNullIfNeeded(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,7 @@ package io.druid.query.extraction;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -52,7 +52,7 @@ public class LowerExtractionFn extends DimExtractionFn
|
|||
@Override
|
||||
public String apply(@Nullable String key)
|
||||
{
|
||||
if (Strings.isNullOrEmpty(key)) {
|
||||
if (NullHandling.isNullOrEquivalent(key)) {
|
||||
return null;
|
||||
}
|
||||
return key.toLowerCase(locale);
|
||||
|
@ -73,7 +73,7 @@ public class LowerExtractionFn extends DimExtractionFn
|
|||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
byte[] localeBytes = StringUtils.toUtf8(Strings.nullToEmpty(localeString));
|
||||
byte[] localeBytes = StringUtils.toUtf8(StringUtils.nullToEmptyNonDruidDataString(localeString));
|
||||
return ByteBuffer.allocate(2 + localeBytes.length)
|
||||
.put(ExtractionCacheHelper.CACHE_TYPE_ID_LOWER)
|
||||
.put((byte) 0XFF)
|
||||
|
|
|
@ -23,21 +23,20 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.query.lookup.LookupExtractor;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.validation.constraints.NotNull;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@JsonTypeName("map")
|
||||
public class MapLookupExtractor extends LookupExtractor
|
||||
|
@ -64,22 +63,32 @@ public class MapLookupExtractor extends LookupExtractor
|
|||
|
||||
@Nullable
|
||||
@Override
|
||||
public String apply(@NotNull String val)
|
||||
public String apply(@Nullable String key)
|
||||
{
|
||||
return map.get(val);
|
||||
String keyEquivalent = NullHandling.nullToEmptyIfNeeded(key);
|
||||
if (keyEquivalent == null) {
|
||||
// keyEquivalent is null only for SQL Compatible Null Behavior
|
||||
// otherwise null will be replaced with empty string in nullToEmptyIfNeeded above.
|
||||
return null;
|
||||
}
|
||||
return NullHandling.emptyToNullIfNeeded(map.get(keyEquivalent));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> unapply(final String value)
|
||||
public List<String> unapply(@Nullable final String value)
|
||||
{
|
||||
return Lists.newArrayList(Maps.filterKeys(map, new Predicate<String>()
|
||||
{
|
||||
@Override public boolean apply(@Nullable String key)
|
||||
{
|
||||
return map.get(key).equals(Strings.nullToEmpty(value));
|
||||
}
|
||||
}).keySet());
|
||||
|
||||
String valueToLookup = NullHandling.nullToEmptyIfNeeded(value);
|
||||
if (valueToLookup == null) {
|
||||
// valueToLookup is null only for SQL Compatible Null Behavior
|
||||
// otherwise null will be replaced with empty string in nullToEmptyIfNeeded above.
|
||||
// null value maps to empty list when SQL Compatible
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return map.entrySet()
|
||||
.stream()
|
||||
.filter(entry -> entry.getValue().equals(valueToLookup))
|
||||
.map(entry -> entry.getKey())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -22,7 +22,7 @@ package io.druid.query.extraction;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.config.NullHandling;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -62,7 +62,7 @@ public class MatchingDimExtractionFn extends DimExtractionFn
|
|||
@Override
|
||||
public String apply(@Nullable String dimValue)
|
||||
{
|
||||
if (Strings.isNullOrEmpty(dimValue)) {
|
||||
if (NullHandling.isNullOrEquivalent(dimValue)) {
|
||||
// We'd return null whether or not the pattern matched
|
||||
return null;
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue