mirror of https://github.com/apache/druid.git
bug fixes and add support for boolean inputs to classic long dimension indexer (#14069)
changes: * adds support for boolean inputs to the classic long dimension indexer, which plays nice with LONG being the semi official boolean type in Druid, and even nicer when druid.expressions.useStrictBooleans is set to true, since the sampler when using the new 'auto' schema when 'useSchemaDiscovery' is specified on the dimensions spec will call the type out as LONG * fix bugs with sampler response and new schema discovery stuff incorrectly using classic 'json' type for the logical schema instead of the new 'auto' type
This commit is contained in:
parent
966cae1c94
commit
9ed8beca5e
|
@ -0,0 +1,186 @@
|
|||
/*
|
||||
* 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 org.apache.druid.indexing.overlord.sampler;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.client.indexing.SamplerResponse;
|
||||
import org.apache.druid.data.input.InputSource;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.DoubleDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.InlineInputSource;
|
||||
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||
import org.apache.druid.data.input.impl.LongDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.math.expr.ExpressionProcessing;
|
||||
import org.apache.druid.segment.AutoTypeColumnSchema;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.indexing.DataSchema;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.apache.logging.log4j.util.Strings;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
||||
{
|
||||
private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper();
|
||||
private static final List<String> STR_JSON_ROWS = ImmutableList.of(
|
||||
"{ \"t\": \"2019-04-22T12:00\", \"string\": \"a\", \"long\": 1, \"double\":1.1, \"bool\":true, \"variant\":\"a\", \"array\":[1, 2, 3], \"nested\": {\"x\":1, \"y\": 2.0}}",
|
||||
"{ \"t\": \"2019-04-22T12:00\", \"string\": \"b\", \"long\": 2, \"double\":2.2, \"bool\":false, \"variant\": 1.0, \"array\":[4, 5, 6], \"nested\": {\"x\":2, \"y\": 4.0} }",
|
||||
"{ \"t\": \"2019-04-22T12:01\", \"string\": null, \"long\": null, \"double\":3.3, \"bool\":null, \"variant\":2, \"array\":[7, 8, 9], \"nested\": {\"x\":3, \"y\": 6.0} }",
|
||||
"{ \"t\": \"2019-04-22T12:00\", \"string\": \"c\", \"long\": 4, \"double\":4.4, \"bool\":true, \"variant\":\"3\", \"array\":[10, 11, 12], \"nested\": {\"x\":4, \"y\": 8.0} }",
|
||||
"{ \"t\": \"2019-04-22T12:00\", \"string\": \"d\", \"long\": 5, \"double\":null, \"bool\":false, \"variant\":null, \"array\":[13, 14, 15], \"nested\": {\"x\":5, \"y\": 10.0} }",
|
||||
"{ \"t\": \"bad_timestamp\", \"string\": \"e\", \"long\": 6, \"double\":6.6, \"bool\":true, \"variant\":\"4\", \"array\":[16, 17, 18], \"nested\": {\"x\":6, \"y\": 12.0} }"
|
||||
);
|
||||
private InputSourceSampler inputSourceSampler = new InputSourceSampler(OBJECT_MAPPER);
|
||||
|
||||
@Test
|
||||
public void testDiscoveredTypes()
|
||||
{
|
||||
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
|
||||
final SamplerResponse response = inputSourceSampler.sample(
|
||||
inputSource,
|
||||
new JsonInputFormat(null, null, null, null, null),
|
||||
new DataSchema(
|
||||
"test",
|
||||
new TimestampSpec("t", null, null),
|
||||
DimensionsSpec.builder().useSchemaDiscovery(true).build(),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
Assert.assertEquals(6, response.getNumRowsRead());
|
||||
Assert.assertEquals(5, response.getNumRowsIndexed());
|
||||
Assert.assertEquals(6, response.getData().size());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("string"),
|
||||
new LongDimensionSchema("long"),
|
||||
new DoubleDimensionSchema("double"),
|
||||
new StringDimensionSchema("bool"),
|
||||
new AutoTypeColumnSchema("variant"),
|
||||
new AutoTypeColumnSchema("array"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
),
|
||||
response.getLogicalDimensions()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
new AutoTypeColumnSchema("string"),
|
||||
new AutoTypeColumnSchema("long"),
|
||||
new AutoTypeColumnSchema("double"),
|
||||
new AutoTypeColumnSchema("bool"),
|
||||
new AutoTypeColumnSchema("variant"),
|
||||
new AutoTypeColumnSchema("array"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
),
|
||||
response.getPhysicalDimensions()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
RowSignature.builder()
|
||||
.addTimeColumn()
|
||||
.add("string", ColumnType.STRING)
|
||||
.add("long", ColumnType.LONG)
|
||||
.add("double", ColumnType.DOUBLE)
|
||||
.add("bool", ColumnType.STRING)
|
||||
.add("variant", ColumnType.NESTED_DATA)
|
||||
.add("array", ColumnType.LONG_ARRAY)
|
||||
.add("nested", ColumnType.NESTED_DATA)
|
||||
.build(),
|
||||
response.getLogicalSegmentSchema()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDiscoveredTypesStrictBooleans()
|
||||
{
|
||||
try {
|
||||
ExpressionProcessing.initializeForStrictBooleansTests(true);
|
||||
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
|
||||
final SamplerResponse response = inputSourceSampler.sample(
|
||||
inputSource,
|
||||
new JsonInputFormat(null, null, null, null, null),
|
||||
new DataSchema(
|
||||
"test",
|
||||
new TimestampSpec("t", null, null),
|
||||
DimensionsSpec.builder().useSchemaDiscovery(true).build(),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
),
|
||||
null
|
||||
);
|
||||
|
||||
Assert.assertEquals(6, response.getNumRowsRead());
|
||||
Assert.assertEquals(5, response.getNumRowsIndexed());
|
||||
Assert.assertEquals(6, response.getData().size());
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("string"),
|
||||
new LongDimensionSchema("long"),
|
||||
new DoubleDimensionSchema("double"),
|
||||
new LongDimensionSchema("bool"),
|
||||
new AutoTypeColumnSchema("variant"),
|
||||
new AutoTypeColumnSchema("array"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
),
|
||||
response.getLogicalDimensions()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
new AutoTypeColumnSchema("string"),
|
||||
new AutoTypeColumnSchema("long"),
|
||||
new AutoTypeColumnSchema("double"),
|
||||
new AutoTypeColumnSchema("bool"),
|
||||
new AutoTypeColumnSchema("variant"),
|
||||
new AutoTypeColumnSchema("array"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
),
|
||||
response.getPhysicalDimensions()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
RowSignature.builder()
|
||||
.addTimeColumn()
|
||||
.add("string", ColumnType.STRING)
|
||||
.add("long", ColumnType.LONG)
|
||||
.add("double", ColumnType.DOUBLE)
|
||||
.add("bool", ColumnType.LONG)
|
||||
.add("variant", ColumnType.NESTED_DATA)
|
||||
.add("array", ColumnType.LONG_ARRAY)
|
||||
.add("nested", ColumnType.NESTED_DATA)
|
||||
.build(),
|
||||
response.getLogicalSegmentSchema()
|
||||
);
|
||||
}
|
||||
finally {
|
||||
ExpressionProcessing.initializeForTests();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -68,8 +68,8 @@ public abstract class DimensionSchema
|
|||
case DOUBLE:
|
||||
return new DoubleDimensionSchema(name);
|
||||
default:
|
||||
// the nested column indexer can handle any type
|
||||
return new NestedDataDimensionSchema(name);
|
||||
// the auto column indexer can handle any type
|
||||
return new AutoTypeColumnSchema(name);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -300,7 +300,6 @@ public class DimensionsSpec
|
|||
private List<String> dimensionExclusions;
|
||||
private List<SpatialDimensionSchema> spatialDimensions;
|
||||
private boolean includeAllDimensions;
|
||||
|
||||
private boolean useSchemaDiscovery;
|
||||
|
||||
public Builder setDimensions(List<DimensionSchema> dimensions)
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.IAE;
|
|||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.apache.druid.java.util.common.parsers.ParseException;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.query.ColumnSelectorPlus;
|
||||
import org.apache.druid.query.dimension.ColumnSelectorStrategy;
|
||||
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
|
||||
|
@ -321,6 +322,8 @@ public final class DimensionHandlerUtils
|
|||
return (Long) valObj;
|
||||
} else if (valObj instanceof Number) {
|
||||
return ((Number) valObj).longValue();
|
||||
} else if (valObj instanceof Boolean) {
|
||||
return Evals.asLong((Boolean) valObj);
|
||||
} else if (valObj instanceof String) {
|
||||
Long ret = DimensionHandlerUtils.getExactLongFromDecimalString((String) valObj);
|
||||
if (reportParseExceptions && ret == null) {
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.DoubleDimensionSchema;
|
||||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||
|
@ -36,7 +37,10 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.segment.AutoTypeColumnSchema;
|
||||
import org.apache.druid.segment.CloserRule;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
|
@ -46,6 +50,8 @@ import org.junit.runners.Parameterized;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -77,7 +83,14 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest
|
|||
new StringDimensionSchema("string"),
|
||||
new FloatDimensionSchema("float"),
|
||||
new LongDimensionSchema("long"),
|
||||
new DoubleDimensionSchema("double")
|
||||
new DoubleDimensionSchema("double"),
|
||||
new StringDimensionSchema("bool_string"),
|
||||
new LongDimensionSchema("bool_long"),
|
||||
new AutoTypeColumnSchema("bool_auto"),
|
||||
new AutoTypeColumnSchema("array_string"),
|
||||
new AutoTypeColumnSchema("array_double"),
|
||||
new AutoTypeColumnSchema("array_long"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
)
|
||||
);
|
||||
AggregatorFactory[] metrics = {
|
||||
|
@ -320,4 +333,105 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest
|
|||
|
||||
Assert.assertEquals(1, index.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTypeHandling() throws IndexSizeExceededException
|
||||
{
|
||||
IncrementalIndex index = indexCreator.createIndex();
|
||||
|
||||
final List<String> dims = Arrays.asList(
|
||||
"string",
|
||||
"float",
|
||||
"long",
|
||||
"double",
|
||||
"bool_string",
|
||||
"bool_long",
|
||||
"bool_auto",
|
||||
"array_string",
|
||||
"array_long",
|
||||
"array_double",
|
||||
"nested"
|
||||
);
|
||||
IncrementalIndexAddResult result = index.add(
|
||||
new MapBasedInputRow(
|
||||
0,
|
||||
dims,
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("string", "a")
|
||||
.put("float", 1.0)
|
||||
.put("long", 1)
|
||||
.put("double", 1.0)
|
||||
.put("bool_string", true)
|
||||
.put("bool_long", true)
|
||||
.put("bool_auto", true)
|
||||
.put("array_string", ImmutableList.of("a", "b", "c"))
|
||||
.put("array_long", ImmutableList.of(1, 2, 3))
|
||||
.put("array_double", ImmutableList.of(1.1, 2.2, 3.3))
|
||||
.put("nested", ImmutableMap.of("x", 1, "y", ImmutableList.of("a", "b")))
|
||||
.build()
|
||||
)
|
||||
);
|
||||
Assert.assertNull(result.getParseException());
|
||||
result = index.add(
|
||||
new MapBasedInputRow(
|
||||
60_000, // next minute so non-rollup still orders iterator correctly
|
||||
dims,
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("string", "b")
|
||||
.put("float", 2.0)
|
||||
.put("long", 2)
|
||||
.put("double", 2.0)
|
||||
.put("bool_string", false)
|
||||
.put("bool_long", false)
|
||||
.put("bool_auto", false)
|
||||
.put("array_string", ImmutableList.of("d", "e", "f"))
|
||||
.put("array_long", ImmutableList.of(4, 5, 6))
|
||||
.put("array_double", ImmutableList.of(4.4, 5.5, 6.6))
|
||||
.put("nested", ImmutableMap.of("x", 2, "y", ImmutableList.of("c", "d")))
|
||||
.build()
|
||||
)
|
||||
);
|
||||
Assert.assertNull(result.getParseException());
|
||||
|
||||
Assert.assertEquals(ColumnType.STRING, index.getColumnCapabilities("string").toColumnType());
|
||||
Assert.assertEquals(ColumnType.FLOAT, index.getColumnCapabilities("float").toColumnType());
|
||||
Assert.assertEquals(ColumnType.LONG, index.getColumnCapabilities("long").toColumnType());
|
||||
Assert.assertEquals(ColumnType.DOUBLE, index.getColumnCapabilities("double").toColumnType());
|
||||
Assert.assertEquals(ColumnType.STRING, index.getColumnCapabilities("bool_string").toColumnType());
|
||||
Assert.assertEquals(ColumnType.LONG, index.getColumnCapabilities("bool_long").toColumnType());
|
||||
// depends on value of 'druid.expressions.useStrictBooleans', current default is false which parses as strings
|
||||
Assert.assertEquals(ColumnType.STRING, index.getColumnCapabilities("bool_auto").toColumnType());
|
||||
Assert.assertEquals(ColumnType.STRING_ARRAY, index.getColumnCapabilities("array_string").toColumnType());
|
||||
Assert.assertEquals(ColumnType.LONG_ARRAY, index.getColumnCapabilities("array_long").toColumnType());
|
||||
Assert.assertEquals(ColumnType.DOUBLE_ARRAY, index.getColumnCapabilities("array_double").toColumnType());
|
||||
Assert.assertEquals(ColumnType.NESTED_DATA, index.getColumnCapabilities("nested").toColumnType());
|
||||
|
||||
|
||||
Iterator<Row> rowIterator = index.iterator();
|
||||
Row row = rowIterator.next();
|
||||
Assert.assertEquals("a", row.getRaw("string"));
|
||||
Assert.assertEquals(1.0f, row.getRaw("float"));
|
||||
Assert.assertEquals(1L, row.getRaw("long"));
|
||||
Assert.assertEquals(1.0, row.getRaw("double"));
|
||||
Assert.assertEquals("true", row.getRaw("bool_string"));
|
||||
Assert.assertEquals(1L, row.getRaw("bool_long"));
|
||||
Assert.assertEquals(StructuredData.wrap(true), row.getRaw("bool_auto"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableList.of("a", "b", "c")), row.getRaw("array_string"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableList.of(1, 2, 3)), row.getRaw("array_long"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableList.of(1.1, 2.2, 3.3)), row.getRaw("array_double"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 1, "y", ImmutableList.of("a", "b"))), row.getRaw("nested"));
|
||||
|
||||
row = rowIterator.next();
|
||||
Assert.assertEquals("b", row.getRaw("string"));
|
||||
Assert.assertEquals(2.0f, row.getRaw("float"));
|
||||
Assert.assertEquals(2L, row.getRaw("long"));
|
||||
Assert.assertEquals(2.0, row.getRaw("double"));
|
||||
Assert.assertEquals("false", row.getRaw("bool_string"));
|
||||
Assert.assertEquals(0L, row.getRaw("bool_long"));
|
||||
Assert.assertEquals(StructuredData.wrap(false), row.getRaw("bool_auto"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableList.of("d", "e", "f")), row.getRaw("array_string"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableList.of(4, 5, 6)), row.getRaw("array_long"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableList.of(4.4, 5.5, 6.6)), row.getRaw("array_double"));
|
||||
Assert.assertEquals(StructuredData.wrap(ImmutableMap.of("x", 2, "y", ImmutableList.of("c", "d"))), row.getRaw("nested"));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue