mirror of https://github.com/apache/druid.git
fix nested column handling of null and "null" (#13714)
* fix nested column handling of null and "null" * fix issue merging nested column value dictionaries that could incorrect lose dictionary values
This commit is contained in:
parent
1beef30bb2
commit
ec1e6ac840
|
@ -90,7 +90,11 @@ public class SimpleDictionaryMergingIterator<T extends Comparable<T>> implements
|
||||||
}
|
}
|
||||||
|
|
||||||
while (!pQueue.isEmpty() && Objects.equals(value, pQueue.peek().peek())) {
|
while (!pQueue.isEmpty() && Objects.equals(value, pQueue.peek().peek())) {
|
||||||
pQueue.remove();
|
PeekingIterator<T> same = pQueue.remove();
|
||||||
|
same.next();
|
||||||
|
if (same.hasNext()) {
|
||||||
|
pQueue.add(same);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
counter++;
|
counter++;
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.segment.nested;
|
package org.apache.druid.segment.nested;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
|
import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
|
||||||
import it.unimi.dsi.fastutil.ints.IntArrays;
|
import it.unimi.dsi.fastutil.ints.IntArrays;
|
||||||
|
@ -52,7 +53,7 @@ import java.nio.channels.WritableByteChannel;
|
||||||
* for all literal writers, which for this type of writer entails building a local dictionary to map into to the global
|
* for all literal writers, which for this type of writer entails building a local dictionary to map into to the global
|
||||||
* dictionary ({@link #localDictionary}) and writes this unsorted localId to an intermediate integer column,
|
* dictionary ({@link #localDictionary}) and writes this unsorted localId to an intermediate integer column,
|
||||||
* {@link #intermediateValueWriter}.
|
* {@link #intermediateValueWriter}.
|
||||||
*
|
* <p>
|
||||||
* When processing the 'raw' value column is complete, the {@link #writeTo(int, FileSmoosher)} method will sort the
|
* When processing the 'raw' value column is complete, the {@link #writeTo(int, FileSmoosher)} method will sort the
|
||||||
* local ids and write them out to a local sorted dictionary, iterate over {@link #intermediateValueWriter} swapping
|
* local ids and write them out to a local sorted dictionary, iterate over {@link #intermediateValueWriter} swapping
|
||||||
* the unsorted local ids with the sorted ids and writing to the compressed id column writer
|
* the unsorted local ids with the sorted ids and writing to the compressed id column writer
|
||||||
|
@ -133,8 +134,15 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
|
||||||
fillNull(row);
|
fillNull(row);
|
||||||
}
|
}
|
||||||
final T value = processValue(val);
|
final T value = processValue(val);
|
||||||
|
final int localId;
|
||||||
|
// null is always 0
|
||||||
|
if (value == null) {
|
||||||
|
localId = localDictionary.add(0);
|
||||||
|
} else {
|
||||||
final int globalId = lookupGlobalId(value);
|
final int globalId = lookupGlobalId(value);
|
||||||
final int localId = localDictionary.add(globalId);
|
Preconditions.checkArgument(globalId >= 0, "Value [%s] is not present in global dictionary", value);
|
||||||
|
localId = localDictionary.add(globalId);
|
||||||
|
}
|
||||||
intermediateValueWriter.write(localId);
|
intermediateValueWriter.write(localId);
|
||||||
writeValue(value);
|
writeValue(value);
|
||||||
cursorPosition++;
|
cursorPosition++;
|
||||||
|
|
|
@ -46,8 +46,11 @@ public class GlobalDictionaryIdLookup
|
||||||
public GlobalDictionaryIdLookup()
|
public GlobalDictionaryIdLookup()
|
||||||
{
|
{
|
||||||
this.stringLookup = new Object2IntLinkedOpenHashMap<>();
|
this.stringLookup = new Object2IntLinkedOpenHashMap<>();
|
||||||
|
stringLookup.defaultReturnValue(-1);
|
||||||
this.longLookup = new Long2IntLinkedOpenHashMap();
|
this.longLookup = new Long2IntLinkedOpenHashMap();
|
||||||
|
longLookup.defaultReturnValue(-1);
|
||||||
this.doubleLookup = new Double2IntLinkedOpenHashMap();
|
this.doubleLookup = new Double2IntLinkedOpenHashMap();
|
||||||
|
doubleLookup.defaultReturnValue(-1);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addString(@Nullable String value)
|
public void addString(@Nullable String value)
|
||||||
|
|
|
@ -238,11 +238,12 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
globalDictionaryIdLookup.addString(null);
|
globalDictionaryIdLookup.addString(null);
|
||||||
for (String value : dictionaryValues) {
|
for (String value : dictionaryValues) {
|
||||||
if (NullHandling.emptyToNullIfNeeded(value) == null) {
|
value = NullHandling.emptyToNullIfNeeded(value);
|
||||||
|
if (value == null) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
value = NullHandling.emptyToNullIfNeeded(value);
|
|
||||||
globalDictionaryIdLookup.addString(value);
|
globalDictionaryIdLookup.addString(value);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -147,9 +147,10 @@ public class NestedFieldLiteralDictionaryEncodedColumn<TStringDictionary extends
|
||||||
return StringUtils.fromUtf8Nullable(globalDictionary.get(globalId));
|
return StringUtils.fromUtf8Nullable(globalDictionary.get(globalId));
|
||||||
} else if (globalId < globalDictionary.size() + globalLongDictionary.size()) {
|
} else if (globalId < globalDictionary.size() + globalLongDictionary.size()) {
|
||||||
return String.valueOf(globalLongDictionary.get(globalId - adjustLongId));
|
return String.valueOf(globalLongDictionary.get(globalId - adjustLongId));
|
||||||
} else {
|
} else if (globalId < globalDictionary.size() + globalLongDictionary.size() + globalDoubleDictionary.size()) {
|
||||||
return String.valueOf(globalDoubleDictionary.get(globalId - adjustDoubleId));
|
return String.valueOf(globalDoubleDictionary.get(globalId - adjustDoubleId));
|
||||||
}
|
}
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.apache.druid.segment.nested;
|
package org.apache.druid.segment.nested;
|
||||||
|
|
||||||
|
import org.apache.druid.common.config.NullHandling;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
|
@ -45,7 +46,10 @@ public final class StringFieldColumnWriter extends GlobalDictionaryEncodedFieldC
|
||||||
@Override
|
@Override
|
||||||
String processValue(Object value)
|
String processValue(Object value)
|
||||||
{
|
{
|
||||||
return String.valueOf(value);
|
if (value == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return NullHandling.emptyToNullIfNeeded(String.valueOf(value));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.commons.io.LineIterator;
|
import org.apache.commons.io.LineIterator;
|
||||||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||||
import org.apache.druid.guice.NestedDataModule;
|
import org.apache.druid.guice.NestedDataModule;
|
||||||
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularity;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.java.util.common.guava.nary.TrinaryFn;
|
import org.apache.druid.java.util.common.guava.nary.TrinaryFn;
|
||||||
|
@ -42,11 +43,15 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||||
import org.apache.druid.timeline.SegmentId;
|
import org.apache.druid.timeline.SegmentId;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
|
import java.io.ByteArrayInputStream;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.SequenceInputStream;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
public class NestedDataTestUtils
|
public class NestedDataTestUtils
|
||||||
|
@ -58,6 +63,9 @@ public class NestedDataTestUtils
|
||||||
public static final String SIMPLE_PARSER_TSV_TRANSFORM_FILE = "simple-nested-test-data-tsv-transform.json";
|
public static final String SIMPLE_PARSER_TSV_TRANSFORM_FILE = "simple-nested-test-data-tsv-transform.json";
|
||||||
public static final String SIMPLE_AGG_FILE = "simple-nested-test-data-aggs.json";
|
public static final String SIMPLE_AGG_FILE = "simple-nested-test-data-aggs.json";
|
||||||
|
|
||||||
|
public static final String TYPES_DATA_FILE = "types-test-data.json";
|
||||||
|
public static final String TYPES_PARSER_FILE = "types-test-data-parser.json";
|
||||||
|
|
||||||
public static final String NUMERIC_DATA_FILE = "numeric-nested-test-data.json";
|
public static final String NUMERIC_DATA_FILE = "numeric-nested-test-data.json";
|
||||||
public static final String NUMERIC_PARSER_FILE = "numeric-nested-test-data-parser.json";
|
public static final String NUMERIC_PARSER_FILE = "numeric-nested-test-data-parser.json";
|
||||||
|
|
||||||
|
@ -160,6 +168,7 @@ public class NestedDataTestUtils
|
||||||
maxRowCount,
|
maxRowCount,
|
||||||
rollup
|
rollup
|
||||||
);
|
);
|
||||||
|
inputDataStream.close();
|
||||||
|
|
||||||
final List<Segment> segments = Lists.transform(
|
final List<Segment> segments = Lists.transform(
|
||||||
ImmutableList.of(segmentDir),
|
ImmutableList.of(segmentDir),
|
||||||
|
@ -207,6 +216,7 @@ public class NestedDataTestUtils
|
||||||
maxRowCount,
|
maxRowCount,
|
||||||
rollup
|
rollup
|
||||||
);
|
);
|
||||||
|
inputDataStream.close();
|
||||||
|
|
||||||
final List<Segment> segments = Lists.transform(
|
final List<Segment> segments = Lists.transform(
|
||||||
ImmutableList.of(segmentDir),
|
ImmutableList.of(segmentDir),
|
||||||
|
@ -223,6 +233,96 @@ public class NestedDataTestUtils
|
||||||
return segments;
|
return segments;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static List<Segment> createSegmentsWithConcatenatedInput(
|
||||||
|
AggregationTestHelper helper,
|
||||||
|
TemporaryFolder tempFolder,
|
||||||
|
Closer closer,
|
||||||
|
Granularity granularity,
|
||||||
|
boolean rollup,
|
||||||
|
int maxRowCount,
|
||||||
|
int numCopies,
|
||||||
|
int numSegments
|
||||||
|
) throws Exception
|
||||||
|
{
|
||||||
|
return createSegmentsWithConcatenatedInput(
|
||||||
|
helper,
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
SIMPLE_DATA_FILE,
|
||||||
|
SIMPLE_PARSER_FILE,
|
||||||
|
null,
|
||||||
|
SIMPLE_AGG_FILE,
|
||||||
|
granularity,
|
||||||
|
rollup,
|
||||||
|
maxRowCount,
|
||||||
|
numCopies,
|
||||||
|
numSegments
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* turn small test data into bigger test data by duplicating itself into a bigger stream
|
||||||
|
*/
|
||||||
|
public static List<Segment> createSegmentsWithConcatenatedInput(
|
||||||
|
AggregationTestHelper helper,
|
||||||
|
TemporaryFolder tempFolder,
|
||||||
|
Closer closer,
|
||||||
|
String inputFileName,
|
||||||
|
String parserJsonFileName,
|
||||||
|
String transformSpecJsonFileName,
|
||||||
|
String aggJsonFileName,
|
||||||
|
Granularity granularity,
|
||||||
|
boolean rollup,
|
||||||
|
int maxRowCount,
|
||||||
|
int numCopies,
|
||||||
|
int numSegments
|
||||||
|
) throws Exception
|
||||||
|
{
|
||||||
|
String parserJson = readFileFromClasspathAsString(parserJsonFileName);
|
||||||
|
String transformSpecJson = transformSpecJsonFileName != null ? readFileFromClasspathAsString(transformSpecJsonFileName) : null;
|
||||||
|
String aggJson = readFileFromClasspathAsString(aggJsonFileName);
|
||||||
|
|
||||||
|
List<File> segmentDirs = Lists.newArrayListWithCapacity(numSegments);
|
||||||
|
for (int i = 0; i < numSegments; i++) {
|
||||||
|
List<InputStream> inputStreams = Lists.newArrayListWithCapacity(numCopies);
|
||||||
|
for (int j = 0; j < numCopies; j++) {
|
||||||
|
inputStreams.add(new FileInputStream(readFileFromClasspath(inputFileName)));
|
||||||
|
if (j + 1 < numCopies) {
|
||||||
|
inputStreams.add(new ByteArrayInputStream(StringUtils.toUtf8("\n")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
SequenceInputStream inputDataStream = new SequenceInputStream(Collections.enumeration(inputStreams));
|
||||||
|
File segmentDir = tempFolder.newFolder();
|
||||||
|
helper.createIndex(
|
||||||
|
inputDataStream,
|
||||||
|
parserJson,
|
||||||
|
transformSpecJson,
|
||||||
|
aggJson,
|
||||||
|
segmentDir,
|
||||||
|
0,
|
||||||
|
granularity,
|
||||||
|
maxRowCount,
|
||||||
|
rollup
|
||||||
|
);
|
||||||
|
inputDataStream.close();
|
||||||
|
segmentDirs.add(segmentDir);
|
||||||
|
}
|
||||||
|
|
||||||
|
final List<Segment> segments = Lists.transform(
|
||||||
|
segmentDirs,
|
||||||
|
dir -> {
|
||||||
|
try {
|
||||||
|
return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
|
||||||
|
}
|
||||||
|
catch (IOException ex) {
|
||||||
|
throw new RuntimeException(ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
return segments;
|
||||||
|
}
|
||||||
|
|
||||||
public static Segment createIncrementalIndex(
|
public static Segment createIncrementalIndex(
|
||||||
String inputFileName,
|
String inputFileName,
|
||||||
String parserJsonFileName,
|
String parserJsonFileName,
|
||||||
|
@ -258,6 +358,7 @@ public class NestedDataTestUtils
|
||||||
maxRowCount,
|
maxRowCount,
|
||||||
rollup
|
rollup
|
||||||
);
|
);
|
||||||
|
inputDataStream.close();
|
||||||
return new IncrementalIndexSegment(index, SegmentId.dummy("test_datasource"));
|
return new IncrementalIndexSegment(index, SegmentId.dummy("test_datasource"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -375,7 +375,7 @@ public class SchemaEvolutionTest
|
||||||
|
|
||||||
// Only nonexistent(4)
|
// Only nonexistent(4)
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
timeseriesResult(TestHelper.createExpectedMap(
|
timeseriesResult(TestHelper.makeMap(
|
||||||
"a",
|
"a",
|
||||||
NullHandling.defaultLongValue(),
|
NullHandling.defaultLongValue(),
|
||||||
"b",
|
"b",
|
||||||
|
|
|
@ -24,41 +24,21 @@ import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.guice.NestedDataModule;
|
import org.apache.druid.guice.NestedDataModule;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.java.util.common.UOE;
|
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
import org.apache.druid.java.util.common.guava.Sequence;
|
import org.apache.druid.java.util.common.guava.Sequence;
|
||||||
import org.apache.druid.java.util.common.guava.Yielder;
|
|
||||||
import org.apache.druid.java.util.common.guava.Yielders;
|
|
||||||
import org.apache.druid.java.util.common.io.Closer;
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.query.Druids;
|
import org.apache.druid.query.Druids;
|
||||||
import org.apache.druid.query.NestedDataTestUtils;
|
import org.apache.druid.query.NestedDataTestUtils;
|
||||||
import org.apache.druid.query.Query;
|
import org.apache.druid.query.Query;
|
||||||
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
|
||||||
import org.apache.druid.query.filter.BoundDimFilter;
|
import org.apache.druid.query.filter.BoundDimFilter;
|
||||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||||
import org.apache.druid.query.ordering.StringComparators;
|
import org.apache.druid.query.ordering.StringComparators;
|
||||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
|
||||||
import org.apache.druid.segment.ColumnValueSelector;
|
|
||||||
import org.apache.druid.segment.Cursor;
|
|
||||||
import org.apache.druid.segment.DoubleColumnSelector;
|
|
||||||
import org.apache.druid.segment.LongColumnSelector;
|
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
import org.apache.druid.segment.StorageAdapter;
|
|
||||||
import org.apache.druid.segment.VirtualColumns;
|
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
|
import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
|
||||||
import org.apache.druid.segment.nested.NestedPathFinder;
|
|
||||||
import org.apache.druid.segment.nested.NestedPathPart;
|
|
||||||
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
|
|
||||||
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
|
|
||||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
|
||||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
|
||||||
import org.apache.druid.segment.vector.VectorCursor;
|
|
||||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
|
||||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
|
||||||
import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
|
import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
|
||||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -74,14 +54,6 @@ import java.util.List;
|
||||||
public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
{
|
{
|
||||||
private static final Logger LOG = new Logger(NestedDataScanQueryTest.class);
|
private static final Logger LOG = new Logger(NestedDataScanQueryTest.class);
|
||||||
private static final String NESTED_LONG_FIELD = "long";
|
|
||||||
private static final String NESTED_DOUBLE_FIELD = "double";
|
|
||||||
private static final String NESTED_MIXED_NUMERIC_FIELD = "mixed_numeric";
|
|
||||||
private static final String NESTED_MIXED_FIELD = "mixed";
|
|
||||||
private static final String NESTED_SPARSE_LONG_FIELD = "sparse_long";
|
|
||||||
private static final String NESTED_SPARSE_DOUBLE_FIELD = "sparse_double";
|
|
||||||
private static final String NESTED_SPARSE_MIXED_NUMERIC_FIELD = "sparse_mixed_numeric";
|
|
||||||
private static final String NESTED_SPARSE_MIXED_FIELD = "sparse_mixed";
|
|
||||||
|
|
||||||
private final AggregationTestHelper helper;
|
private final AggregationTestHelper helper;
|
||||||
private final Closer closer;
|
private final Closer closer;
|
||||||
|
@ -342,6 +314,75 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
logResults(results);
|
logResults(results);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIngestWithMoreMergesAndScanSegments() throws Exception
|
||||||
|
{
|
||||||
|
Query<ScanResultValue> scanQuery = Druids.newScanQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(
|
||||||
|
new MultipleIntervalSegmentSpec(
|
||||||
|
Collections.singletonList(Intervals.ETERNITY)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
|
.limit(100)
|
||||||
|
.context(ImmutableMap.of())
|
||||||
|
.build();
|
||||||
|
|
||||||
|
|
||||||
|
List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput(
|
||||||
|
helper,
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
Granularities.HOUR,
|
||||||
|
false,
|
||||||
|
5,
|
||||||
|
10,
|
||||||
|
1
|
||||||
|
);
|
||||||
|
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
|
||||||
|
|
||||||
|
List<ScanResultValue> results = seq.toList();
|
||||||
|
logResults(results);
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Assert.assertEquals(80, ((List) results.get(0).getEvents()).size());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIngestWithMoreMergesAndScanSegmentsRollup() throws Exception
|
||||||
|
{
|
||||||
|
Query<ScanResultValue> scanQuery = Druids.newScanQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(
|
||||||
|
new MultipleIntervalSegmentSpec(
|
||||||
|
Collections.singletonList(Intervals.ETERNITY)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
|
.limit(100)
|
||||||
|
.context(ImmutableMap.of())
|
||||||
|
.build();
|
||||||
|
|
||||||
|
|
||||||
|
// same rows over and over so expect same 8 rows after rollup
|
||||||
|
List<Segment> segs = NestedDataTestUtils.createSegmentsWithConcatenatedInput(
|
||||||
|
helper,
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
Granularities.HOUR,
|
||||||
|
true,
|
||||||
|
5,
|
||||||
|
100,
|
||||||
|
1
|
||||||
|
);
|
||||||
|
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(segs, scanQuery);
|
||||||
|
|
||||||
|
List<ScanResultValue> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Assert.assertEquals(8, ((List) results.get(0).getEvents()).size());
|
||||||
|
logResults(results);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIngestAndScanSegmentsAndFilter() throws Exception
|
public void testIngestAndScanSegmentsAndFilter() throws Exception
|
||||||
{
|
{
|
||||||
|
@ -412,295 +453,61 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testExpectedTypes() throws Exception
|
public void testIngestAndScanSegmentsRealtimeSchemaDiscovery() throws Exception
|
||||||
{
|
{
|
||||||
// "Line matches the illegal pattern 'ObjectColumnSelector, LongColumnSelector, FloatColumnSelector
|
Query<ScanResultValue> scanQuery = Druids.newScanQueryBuilder()
|
||||||
// and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."
|
.dataSource("test_datasource")
|
||||||
//CHECKSTYLE.OFF: Regexp
|
.intervals(
|
||||||
ColumnSelectorFactory columnSelectorFactory = getNumericColumnSelectorFactory(
|
new MultipleIntervalSegmentSpec(
|
||||||
makeNestedNumericVirtualColumns()
|
Collections.singletonList(Intervals.ETERNITY)
|
||||||
);
|
)
|
||||||
|
)
|
||||||
ColumnValueSelector longValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
NESTED_LONG_FIELD
|
.limit(100)
|
||||||
);
|
.context(ImmutableMap.of())
|
||||||
Assert.assertNotNull(longValueSelector);
|
.build();
|
||||||
Assert.assertTrue(longValueSelector instanceof LongColumnSelector);
|
List<Segment> realtimeSegs = ImmutableList.of(
|
||||||
|
NestedDataTestUtils.createIncrementalIndex(
|
||||||
ColumnValueSelector doubleValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
NestedDataTestUtils.TYPES_DATA_FILE,
|
||||||
NESTED_DOUBLE_FIELD
|
NestedDataTestUtils.TYPES_PARSER_FILE,
|
||||||
);
|
NestedDataTestUtils.SIMPLE_AGG_FILE,
|
||||||
Assert.assertNotNull(doubleValueSelector);
|
Granularities.DAY,
|
||||||
Assert.assertTrue(doubleValueSelector instanceof DoubleColumnSelector);
|
true,
|
||||||
|
|
||||||
ColumnValueSelector mixedNumericValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
|
||||||
NESTED_MIXED_NUMERIC_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(mixedNumericValueSelector);
|
|
||||||
Assert.assertTrue(mixedNumericValueSelector instanceof ColumnValueSelector);
|
|
||||||
|
|
||||||
ColumnValueSelector mixedValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
|
||||||
NESTED_MIXED_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(mixedValueSelector);
|
|
||||||
Assert.assertTrue(mixedValueSelector instanceof ColumnValueSelector);
|
|
||||||
|
|
||||||
|
|
||||||
ColumnValueSelector sparseLongValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
|
||||||
NESTED_SPARSE_LONG_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(sparseLongValueSelector);
|
|
||||||
Assert.assertTrue(sparseLongValueSelector instanceof LongColumnSelector);
|
|
||||||
|
|
||||||
ColumnValueSelector sparseDoubleValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
|
||||||
NESTED_SPARSE_DOUBLE_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(sparseDoubleValueSelector);
|
|
||||||
Assert.assertTrue(sparseDoubleValueSelector instanceof DoubleColumnSelector);
|
|
||||||
|
|
||||||
ColumnValueSelector sparseMixedNumericValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
|
||||||
NESTED_SPARSE_MIXED_NUMERIC_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(sparseMixedNumericValueSelector);
|
|
||||||
Assert.assertTrue(sparseMixedNumericValueSelector instanceof ColumnValueSelector);
|
|
||||||
|
|
||||||
ColumnValueSelector sparseMixedValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
|
||||||
NESTED_SPARSE_MIXED_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(sparseMixedValueSelector);
|
|
||||||
Assert.assertTrue(sparseMixedValueSelector instanceof ColumnValueSelector);
|
|
||||||
//CHECKSTYLE.ON: Regexp
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testExpectedTypesVectorSelectors() throws Exception
|
|
||||||
{
|
|
||||||
// "Line matches the illegal pattern 'ObjectColumnSelector, LongColumnSelector, FloatColumnSelector
|
|
||||||
// and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."
|
|
||||||
//CHECKSTYLE.OFF: Regexp
|
|
||||||
VectorColumnSelectorFactory factory = getVectorColumnSelectorFactory(
|
|
||||||
makeNestedNumericVirtualColumns()
|
|
||||||
);
|
|
||||||
|
|
||||||
// can make numeric value selectors for single typed numeric types
|
|
||||||
VectorValueSelector longValueSelector = factory.makeValueSelector(
|
|
||||||
NESTED_LONG_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(longValueSelector);
|
|
||||||
Assert.assertTrue(longValueSelector instanceof BaseLongVectorValueSelector);
|
|
||||||
|
|
||||||
VectorValueSelector doubleValueSelector = factory.makeValueSelector(
|
|
||||||
NESTED_DOUBLE_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(doubleValueSelector);
|
|
||||||
Assert.assertTrue(doubleValueSelector instanceof BaseDoubleVectorValueSelector);
|
|
||||||
|
|
||||||
Assert.assertThrows(UOE.class, () -> factory.makeValueSelector(NESTED_MIXED_NUMERIC_FIELD));
|
|
||||||
Assert.assertThrows(UOE.class, () -> factory.makeValueSelector(NESTED_MIXED_FIELD));
|
|
||||||
|
|
||||||
// can also make single value dimension selectors for all nested column types
|
|
||||||
SingleValueDimensionVectorSelector longDimensionSelector = factory.makeSingleValueDimensionSelector(
|
|
||||||
DefaultDimensionSpec.of(NESTED_LONG_FIELD)
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(longDimensionSelector);
|
|
||||||
|
|
||||||
SingleValueDimensionVectorSelector doubleDimensionSelector = factory.makeSingleValueDimensionSelector(
|
|
||||||
DefaultDimensionSpec.of(NESTED_DOUBLE_FIELD)
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(doubleDimensionSelector);
|
|
||||||
|
|
||||||
SingleValueDimensionVectorSelector mixedNumericValueSelector = factory.makeSingleValueDimensionSelector(
|
|
||||||
DefaultDimensionSpec.of(NESTED_MIXED_NUMERIC_FIELD)
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(mixedNumericValueSelector);
|
|
||||||
|
|
||||||
SingleValueDimensionVectorSelector mixedValueSelector = factory.makeSingleValueDimensionSelector(
|
|
||||||
DefaultDimensionSpec.of(NESTED_MIXED_FIELD)
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(mixedValueSelector);
|
|
||||||
|
|
||||||
// and object selectors
|
|
||||||
VectorObjectSelector longObjectSelector = factory.makeObjectSelector(
|
|
||||||
NESTED_LONG_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(longObjectSelector);
|
|
||||||
|
|
||||||
VectorObjectSelector doubleObjectSelector = factory.makeObjectSelector(
|
|
||||||
NESTED_DOUBLE_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(doubleObjectSelector);
|
|
||||||
|
|
||||||
VectorObjectSelector mixedNumericObjectSelector = factory.makeObjectSelector(
|
|
||||||
NESTED_MIXED_NUMERIC_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(mixedNumericObjectSelector);
|
|
||||||
|
|
||||||
VectorObjectSelector mixedObjectSelector = factory.makeObjectSelector(
|
|
||||||
NESTED_MIXED_FIELD
|
|
||||||
);
|
|
||||||
Assert.assertNotNull(mixedObjectSelector);
|
|
||||||
//CHECKSTYLE.ON: Regexp
|
|
||||||
}
|
|
||||||
|
|
||||||
private VirtualColumns makeNestedNumericVirtualColumns()
|
|
||||||
{
|
|
||||||
List<NestedPathPart> longParts = NestedPathFinder.parseJqPath(".long");
|
|
||||||
List<NestedPathPart> doubleParts = NestedPathFinder.parseJqPath(".double");
|
|
||||||
List<NestedPathPart> mixedNumericParts = NestedPathFinder.parseJqPath(".mixed_numeric");
|
|
||||||
List<NestedPathPart> mixedParts = NestedPathFinder.parseJqPath(".mixed");
|
|
||||||
List<NestedPathPart> sparseLongParts = NestedPathFinder.parseJqPath(".sparse_long");
|
|
||||||
List<NestedPathPart> sparseDoubleParts = NestedPathFinder.parseJqPath(".sparse_double");
|
|
||||||
List<NestedPathPart> sparseMixedNumericParts = NestedPathFinder.parseJqPath(".sparse_mixed_numeric");
|
|
||||||
List<NestedPathPart> sparseMixedParts = NestedPathFinder.parseJqPath(".sparse_mixed");
|
|
||||||
|
|
||||||
NestedFieldVirtualColumn longVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_LONG_FIELD,
|
|
||||||
ColumnType.LONG,
|
|
||||||
longParts,
|
|
||||||
false,
|
false,
|
||||||
null,
|
1000
|
||||||
null
|
|
||||||
);
|
|
||||||
NestedFieldVirtualColumn doubleVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_DOUBLE_FIELD,
|
|
||||||
ColumnType.DOUBLE,
|
|
||||||
doubleParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
NestedFieldVirtualColumn mixedNumericVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_MIXED_NUMERIC_FIELD,
|
|
||||||
null,
|
|
||||||
mixedNumericParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
NestedFieldVirtualColumn mixedVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_MIXED_FIELD,
|
|
||||||
null,
|
|
||||||
mixedParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
|
|
||||||
NestedFieldVirtualColumn sparseLongVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_SPARSE_LONG_FIELD,
|
|
||||||
ColumnType.LONG,
|
|
||||||
sparseLongParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
NestedFieldVirtualColumn sparseDoubleVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_SPARSE_DOUBLE_FIELD,
|
|
||||||
ColumnType.DOUBLE,
|
|
||||||
sparseDoubleParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
NestedFieldVirtualColumn sparseMixedNumericVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_SPARSE_MIXED_NUMERIC_FIELD,
|
|
||||||
null,
|
|
||||||
sparseMixedNumericParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
NestedFieldVirtualColumn sparseMixedVirtualColumn = new NestedFieldVirtualColumn(
|
|
||||||
"nest",
|
|
||||||
NESTED_SPARSE_MIXED_FIELD,
|
|
||||||
null,
|
|
||||||
sparseMixedParts,
|
|
||||||
false,
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
|
|
||||||
return VirtualColumns.create(
|
|
||||||
ImmutableList.of(
|
|
||||||
longVirtualColumn,
|
|
||||||
doubleVirtualColumn,
|
|
||||||
mixedNumericVirtualColumn,
|
|
||||||
mixedVirtualColumn,
|
|
||||||
sparseLongVirtualColumn,
|
|
||||||
sparseDoubleVirtualColumn,
|
|
||||||
sparseMixedNumericVirtualColumn,
|
|
||||||
sparseMixedVirtualColumn
|
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
List<Segment> segs = NestedDataTestUtils.createSegments(
|
||||||
|
|
||||||
private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
|
|
||||||
{
|
|
||||||
List<Segment> segments = NestedDataTestUtils.createSegments(
|
|
||||||
helper,
|
helper,
|
||||||
tempFolder,
|
tempFolder,
|
||||||
closer,
|
closer,
|
||||||
NestedDataTestUtils.NUMERIC_DATA_FILE,
|
NestedDataTestUtils.TYPES_DATA_FILE,
|
||||||
NestedDataTestUtils.NUMERIC_PARSER_FILE,
|
NestedDataTestUtils.TYPES_PARSER_FILE,
|
||||||
NestedDataTestUtils.SIMPLE_AGG_FILE,
|
NestedDataTestUtils.SIMPLE_AGG_FILE,
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
true,
|
true,
|
||||||
1000
|
100
|
||||||
);
|
);
|
||||||
Assert.assertEquals(1, segments.size());
|
|
||||||
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
|
||||||
Sequence<Cursor> cursorSequence = storageAdapter.makeCursors(
|
|
||||||
null,
|
|
||||||
Intervals.ETERNITY,
|
|
||||||
virtualColumns,
|
|
||||||
Granularities.DAY,
|
|
||||||
false,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
final Yielder<Cursor> yielder = Yielders.each(cursorSequence);
|
|
||||||
closer.register(yielder);
|
|
||||||
final Cursor cursor = yielder.get();
|
|
||||||
return cursor.getColumnSelectorFactory();
|
|
||||||
}
|
|
||||||
|
|
||||||
private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
|
|
||||||
{
|
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
|
||||||
List<Segment> segments = NestedDataTestUtils.createSegments(
|
final Sequence<ScanResultValue> seq2 = helper.runQueryOnSegmentsObjs(segs, scanQuery);
|
||||||
helper,
|
|
||||||
tempFolder,
|
List<ScanResultValue> resultsRealtime = seq.toList();
|
||||||
closer,
|
List<ScanResultValue> resultsSegments = seq2.toList();
|
||||||
NestedDataTestUtils.NUMERIC_DATA_FILE,
|
logResults(resultsSegments);
|
||||||
NestedDataTestUtils.NUMERIC_PARSER_FILE,
|
logResults(resultsRealtime);
|
||||||
NestedDataTestUtils.SIMPLE_AGG_FILE,
|
Assert.assertEquals(1, resultsRealtime.size());
|
||||||
Granularities.DAY,
|
Assert.assertEquals(resultsRealtime.size(), resultsSegments.size());
|
||||||
true,
|
Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
|
||||||
1000
|
|
||||||
);
|
|
||||||
Assert.assertEquals(1, segments.size());
|
|
||||||
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
|
||||||
VectorCursor cursor = storageAdapter.makeVectorCursor(
|
|
||||||
null,
|
|
||||||
Intervals.ETERNITY,
|
|
||||||
virtualColumns,
|
|
||||||
false,
|
|
||||||
512,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
return cursor.getColumnSelectorFactory();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void logResults(List<ScanResultValue> results)
|
private static void logResults(List<ScanResultValue> results)
|
||||||
{
|
{
|
||||||
StringBuilder bob = new StringBuilder();
|
StringBuilder bob = new StringBuilder();
|
||||||
|
int ctr = 0;
|
||||||
for (Object event : (List) results.get(0).getEvents()) {
|
for (Object event : (List) results.get(0).getEvents()) {
|
||||||
bob.append("[").append(event).append("]").append("\n");
|
bob.append("row:").append(++ctr).append(" - ").append(event).append("\n");
|
||||||
}
|
}
|
||||||
LOG.info("results:\n%s", bob);
|
LOG.info("results:\n%s", bob);
|
||||||
}
|
}
|
||||||
|
|
|
@ -651,7 +651,7 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest
|
||||||
new Result<>(
|
new Result<>(
|
||||||
QueryRunnerTestHelper.EMPTY_INTERVAL.getIntervals().get(0).getStart(),
|
QueryRunnerTestHelper.EMPTY_INTERVAL.getIntervals().get(0).getStart(),
|
||||||
new TimeseriesResultValue(
|
new TimeseriesResultValue(
|
||||||
TestHelper.createExpectedMap(
|
TestHelper.makeMap(
|
||||||
"rows",
|
"rows",
|
||||||
0L,
|
0L,
|
||||||
"index",
|
"index",
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package org.apache.druid.segment;
|
package org.apache.druid.segment;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.common.config.NullHandling;
|
import org.apache.druid.common.config.NullHandling;
|
||||||
|
@ -46,8 +45,8 @@ import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import javax.annotation.Nonnull;
|
import javax.annotation.Nonnull;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
|
public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
|
||||||
{
|
{
|
||||||
|
@ -563,20 +562,8 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
|
||||||
Object... kv
|
Object... kv
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
HashMap<String, Object> event = new HashMap<>();
|
final Map<String, Object> event = TestHelper.makeMap(explicitNull, kv);
|
||||||
event.put("time", timestamp);
|
event.put("time", timestamp);
|
||||||
Preconditions.checkArgument(kv.length % 2 == 0);
|
|
||||||
String currentKey = null;
|
|
||||||
for (int i = 0; i < kv.length; i++) {
|
|
||||||
if (i % 2 == 0) {
|
|
||||||
currentKey = (String) kv[i];
|
|
||||||
} else {
|
|
||||||
if (explicitNull || kv[i] != null) {
|
|
||||||
event.put(currentKey, kv[i]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return new MapBasedInputRow(timestamp, ImmutableList.copyOf(event.keySet()), event);
|
return new MapBasedInputRow(timestamp, ImmutableList.copyOf(event.keySet()), event);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -866,7 +866,7 @@ public class SchemalessTestFullTest extends InitializedNullHandlingTest
|
||||||
new Result<>(
|
new Result<>(
|
||||||
DateTimes.of("2011-01-12T00:00:00.000Z"),
|
DateTimes.of("2011-01-12T00:00:00.000Z"),
|
||||||
new TimeseriesResultValue(
|
new TimeseriesResultValue(
|
||||||
TestHelper.createExpectedMap(
|
TestHelper.makeMap(
|
||||||
"rows", 1L,
|
"rows", 1L,
|
||||||
"index", NullHandling.replaceWithDefault() ? 0.0D : null,
|
"index", NullHandling.replaceWithDefault() ? 0.0D : null,
|
||||||
"addRowsIndexConstant", NullHandling.replaceWithDefault() ? 2.0D : null,
|
"addRowsIndexConstant", NullHandling.replaceWithDefault() ? 2.0D : null,
|
||||||
|
@ -881,7 +881,7 @@ public class SchemalessTestFullTest extends InitializedNullHandlingTest
|
||||||
new Result<>(
|
new Result<>(
|
||||||
DateTimes.of("2011-01-12T00:00:00.000Z"),
|
DateTimes.of("2011-01-12T00:00:00.000Z"),
|
||||||
new TimeseriesResultValue(
|
new TimeseriesResultValue(
|
||||||
TestHelper.createExpectedMap(
|
TestHelper.makeMap(
|
||||||
"rows", 0L,
|
"rows", 0L,
|
||||||
"index", NullHandling.replaceWithDefault() ? 0.0D : null,
|
"index", NullHandling.replaceWithDefault() ? 0.0D : null,
|
||||||
"addRowsIndexConstant", NullHandling.replaceWithDefault() ? 1.0D : null,
|
"addRowsIndexConstant", NullHandling.replaceWithDefault() ? 1.0D : null,
|
||||||
|
|
|
@ -0,0 +1,57 @@
|
||||||
|
/*
|
||||||
|
* 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.segment;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.apache.druid.segment.data.Indexed;
|
||||||
|
import org.apache.druid.segment.data.ListIndexed;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class SimpleDictionaryMergingIteratorTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testMergingIterator()
|
||||||
|
{
|
||||||
|
final Indexed[] sortedLookups = new Indexed[]{
|
||||||
|
new ListIndexed(null, "", "null", "z"),
|
||||||
|
new ListIndexed("", "a", "b", "c", "d", "e", "f", "g", "h"),
|
||||||
|
new ListIndexed(null, "b", "c", "null", "z"),
|
||||||
|
new ListIndexed(null, "hello")
|
||||||
|
};
|
||||||
|
SimpleDictionaryMergingIterator<String> dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
|
||||||
|
sortedLookups,
|
||||||
|
NestedDataColumnMerger.STRING_MERGING_COMPARATOR
|
||||||
|
);
|
||||||
|
|
||||||
|
List<String> expectedSequence = Lists.newArrayListWithExpectedSize(13);
|
||||||
|
expectedSequence.add(null);
|
||||||
|
expectedSequence.addAll(ImmutableList.of("", "a", "b", "c", "d", "e", "f", "g", "h", "hello", "null", "z"));
|
||||||
|
|
||||||
|
List<String> actualSequence = Lists.newArrayListWithExpectedSize(13);
|
||||||
|
while (dictionaryMergeIterator.hasNext()) {
|
||||||
|
actualSequence.add(dictionaryMergeIterator.next());
|
||||||
|
}
|
||||||
|
Assert.assertEquals(expectedSequence, actualSequence);
|
||||||
|
}
|
||||||
|
}
|
|
@ -449,14 +449,21 @@ public class TestHelper
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Map<String, Object> createExpectedMap(Object... vals)
|
public static Map<String, Object> makeMap(Object... vals)
|
||||||
|
{
|
||||||
|
return makeMap(true, vals);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Map<String, Object> makeMap(boolean explicitNulls, Object... vals)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(vals.length % 2 == 0);
|
Preconditions.checkArgument(vals.length % 2 == 0);
|
||||||
|
|
||||||
Map<String, Object> theVals = new HashMap<>();
|
Map<String, Object> theVals = new HashMap<>();
|
||||||
for (int i = 0; i < vals.length; i += 2) {
|
for (int i = 0; i < vals.length; i += 2) {
|
||||||
|
if (explicitNulls || vals[i + 1] != null) {
|
||||||
theVals.put(vals[i].toString(), vals[i + 1]);
|
theVals.put(vals[i].toString(), vals[i + 1]);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return theVals;
|
return theVals;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,13 +21,13 @@ package org.apache.druid.segment.nested;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.util.concurrent.Futures;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
import com.google.common.util.concurrent.MoreExecutors;
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
|
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
|
||||||
|
import org.apache.druid.common.config.NullHandling;
|
||||||
import org.apache.druid.guice.NestedDataModule;
|
import org.apache.druid.guice.NestedDataModule;
|
||||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||||
import org.apache.druid.java.util.common.io.Closer;
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
|
@ -94,12 +94,12 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
||||||
DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(new RoaringBitmapFactory());
|
DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(new RoaringBitmapFactory());
|
||||||
|
|
||||||
List<Map<String, Object>> data = ImmutableList.of(
|
List<Map<String, Object>> data = ImmutableList.of(
|
||||||
ImmutableMap.of("x", 1L, "y", 1.0, "z", "a", "v", "100"),
|
TestHelper.makeMap("x", 1L, "y", 1.0, "z", "a", "v", "100", "nullish", "notnull"),
|
||||||
ImmutableMap.of("y", 3.0, "z", "d", "v", 1000L),
|
TestHelper.makeMap("y", 3.0, "z", "d", "v", 1000L, "nullish", null),
|
||||||
ImmutableMap.of("x", 5L, "y", 5.0, "z", "b"),
|
TestHelper.makeMap("x", 5L, "y", 5.0, "z", "b", "nullish", ""),
|
||||||
ImmutableMap.of("x", 3L, "y", 4.0, "z", "c", "v", 3000.333),
|
TestHelper.makeMap("x", 3L, "y", 4.0, "z", "c", "v", 3000.333, "nullish", "null"),
|
||||||
ImmutableMap.of("x", 2L, "v", "40000"),
|
TestHelper.makeMap("x", 2L, "v", "40000"),
|
||||||
ImmutableMap.of("x", 4L, "y", 2.0, "z", "e", "v", 11111L)
|
TestHelper.makeMap("x", 4L, "y", 2.0, "z", "e", "v", 11111L, "nullish", null)
|
||||||
);
|
);
|
||||||
|
|
||||||
Closer closer = Closer.create();
|
Closer closer = Closer.create();
|
||||||
|
@ -320,7 +320,18 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
||||||
DruidPredicateIndex vPredicateIndex = vIndexSupplier.as(DruidPredicateIndex.class);
|
DruidPredicateIndex vPredicateIndex = vIndexSupplier.as(DruidPredicateIndex.class);
|
||||||
NullValueIndex vNulls = vIndexSupplier.as(NullValueIndex.class);
|
NullValueIndex vNulls = vIndexSupplier.as(NullValueIndex.class);
|
||||||
|
|
||||||
Assert.assertEquals(ImmutableList.of(vPath, xPath, yPath, zPath), column.getNestedFields());
|
final List<NestedPathPart> nullishPath = NestedPathFinder.parseJsonPath("$.nullish");
|
||||||
|
Assert.assertEquals(ImmutableSet.of(ColumnType.STRING), column.getColumnTypes(nullishPath));
|
||||||
|
Assert.assertEquals(ColumnType.STRING, column.getColumnHolder(nullishPath).getCapabilities().toColumnType());
|
||||||
|
ColumnValueSelector<?> nullishSelector = column.makeColumnValueSelector(nullishPath, offset);
|
||||||
|
DimensionSelector nullishDimSelector = column.makeDimensionSelector(nullishPath, offset, null);
|
||||||
|
ColumnIndexSupplier nullishIndexSupplier = column.getColumnIndexSupplier(nullishPath);
|
||||||
|
Assert.assertNotNull(nullishIndexSupplier);
|
||||||
|
StringValueSetIndex nullishValueIndex = nullishIndexSupplier.as(StringValueSetIndex.class);
|
||||||
|
DruidPredicateIndex nullishPredicateIndex = nullishIndexSupplier.as(DruidPredicateIndex.class);
|
||||||
|
NullValueIndex nullishNulls = nullishIndexSupplier.as(NullValueIndex.class);
|
||||||
|
|
||||||
|
Assert.assertEquals(ImmutableList.of(nullishPath, vPath, xPath, yPath, zPath), column.getNestedFields());
|
||||||
|
|
||||||
for (int i = 0; i < data.size(); i++) {
|
for (int i = 0; i < data.size(); i++) {
|
||||||
Map row = data.get(i);
|
Map row = data.get(i);
|
||||||
|
@ -333,6 +344,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
||||||
testPath(row, i, "x", xSelector, xDimSelector, xValueIndex, xPredicateIndex, xNulls, ColumnType.LONG);
|
testPath(row, i, "x", xSelector, xDimSelector, xValueIndex, xPredicateIndex, xNulls, ColumnType.LONG);
|
||||||
testPath(row, i, "y", ySelector, yDimSelector, yValueIndex, yPredicateIndex, yNulls, ColumnType.DOUBLE);
|
testPath(row, i, "y", ySelector, yDimSelector, yValueIndex, yPredicateIndex, yNulls, ColumnType.DOUBLE);
|
||||||
testPath(row, i, "z", zSelector, zDimSelector, zValueIndex, zPredicateIndex, zNulls, ColumnType.STRING);
|
testPath(row, i, "z", zSelector, zDimSelector, zValueIndex, zPredicateIndex, zNulls, ColumnType.STRING);
|
||||||
|
testPath(row, i, "nullish", nullishSelector, nullishDimSelector, nullishValueIndex, nullishPredicateIndex, nullishNulls, ColumnType.STRING);
|
||||||
|
|
||||||
offset.increment();
|
offset.increment();
|
||||||
}
|
}
|
||||||
|
@ -350,16 +362,22 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
||||||
@Nullable ColumnType singleType
|
@Nullable ColumnType singleType
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (row.containsKey(path) && row.get(path) != null) {
|
final Object inputValue = row.get(path);
|
||||||
Assert.assertEquals(row.get(path), valueSelector.getObject());
|
// in default value mode, even though the input row had an empty string, the selector spits out null, so we want
|
||||||
|
// to take the null checking path
|
||||||
|
final boolean isStringAndNullEquivalent =
|
||||||
|
inputValue instanceof String && NullHandling.isNullOrEquivalent((String) inputValue);
|
||||||
|
|
||||||
|
if (row.containsKey(path) && inputValue != null && !isStringAndNullEquivalent) {
|
||||||
|
Assert.assertEquals(inputValue, valueSelector.getObject());
|
||||||
if (ColumnType.LONG.equals(singleType)) {
|
if (ColumnType.LONG.equals(singleType)) {
|
||||||
Assert.assertEquals(row.get(path), valueSelector.getLong());
|
Assert.assertEquals(inputValue, valueSelector.getLong());
|
||||||
} else if (ColumnType.DOUBLE.equals(singleType)) {
|
} else if (ColumnType.DOUBLE.equals(singleType)) {
|
||||||
Assert.assertEquals((double) row.get(path), valueSelector.getDouble(), 0.0);
|
Assert.assertEquals((double) inputValue, valueSelector.getDouble(), 0.0);
|
||||||
}
|
}
|
||||||
Assert.assertFalse(valueSelector.isNull());
|
Assert.assertFalse(valueSelector.isNull());
|
||||||
|
|
||||||
final String theString = String.valueOf(row.get(path));
|
final String theString = String.valueOf(inputValue);
|
||||||
Assert.assertEquals(theString, dimSelector.getObject());
|
Assert.assertEquals(theString, dimSelector.getObject());
|
||||||
String dimSelectorLookupVal = dimSelector.lookupName(dimSelector.getRow().get(0));
|
String dimSelectorLookupVal = dimSelector.lookupName(dimSelector.getRow().get(0));
|
||||||
Assert.assertEquals(theString, dimSelectorLookupVal);
|
Assert.assertEquals(theString, dimSelectorLookupVal);
|
||||||
|
|
|
@ -0,0 +1,380 @@
|
||||||
|
/*
|
||||||
|
* 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.segment.nested;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.Module;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.druid.guice.NestedDataModule;
|
||||||
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
|
import org.apache.druid.java.util.common.UOE;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.java.util.common.guava.Sequence;
|
||||||
|
import org.apache.druid.java.util.common.guava.Yielder;
|
||||||
|
import org.apache.druid.java.util.common.guava.Yielders;
|
||||||
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
|
import org.apache.druid.query.NestedDataTestUtils;
|
||||||
|
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||||
|
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.ColumnValueSelector;
|
||||||
|
import org.apache.druid.segment.Cursor;
|
||||||
|
import org.apache.druid.segment.DoubleColumnSelector;
|
||||||
|
import org.apache.druid.segment.LongColumnSelector;
|
||||||
|
import org.apache.druid.segment.Segment;
|
||||||
|
import org.apache.druid.segment.StorageAdapter;
|
||||||
|
import org.apache.druid.segment.VirtualColumns;
|
||||||
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
|
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
|
||||||
|
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
|
||||||
|
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.vector.VectorCursor;
|
||||||
|
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||||
|
import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class NestedFieldLiteralColumnSelectorsTest
|
||||||
|
{
|
||||||
|
private static final String NESTED_LONG_FIELD = "long";
|
||||||
|
private static final String NESTED_DOUBLE_FIELD = "double";
|
||||||
|
private static final String NESTED_MIXED_NUMERIC_FIELD = "mixed_numeric";
|
||||||
|
private static final String NESTED_MIXED_FIELD = "mixed";
|
||||||
|
private static final String NESTED_SPARSE_LONG_FIELD = "sparse_long";
|
||||||
|
private static final String NESTED_SPARSE_DOUBLE_FIELD = "sparse_double";
|
||||||
|
private static final String NESTED_SPARSE_MIXED_NUMERIC_FIELD = "sparse_mixed_numeric";
|
||||||
|
private static final String NESTED_SPARSE_MIXED_FIELD = "sparse_mixed";
|
||||||
|
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||||
|
|
||||||
|
private final AggregationTestHelper helper;
|
||||||
|
private final Closer closer;
|
||||||
|
|
||||||
|
public NestedFieldLiteralColumnSelectorsTest()
|
||||||
|
{
|
||||||
|
NestedDataModule.registerHandlersAndSerde();
|
||||||
|
List<? extends Module> mods = NestedDataModule.getJacksonModulesList();
|
||||||
|
this.helper = AggregationTestHelper.createScanQueryAggregationTestHelper(
|
||||||
|
mods,
|
||||||
|
tempFolder
|
||||||
|
);
|
||||||
|
this.closer = Closer.create();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void teardown() throws IOException
|
||||||
|
{
|
||||||
|
closer.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExpectedTypes() throws Exception
|
||||||
|
{
|
||||||
|
// "Line matches the illegal pattern 'ObjectColumnSelector, LongColumnSelector, FloatColumnSelector
|
||||||
|
// and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."
|
||||||
|
//CHECKSTYLE.OFF: Regexp
|
||||||
|
ColumnSelectorFactory columnSelectorFactory = getNumericColumnSelectorFactory(
|
||||||
|
makeNestedNumericVirtualColumns()
|
||||||
|
);
|
||||||
|
|
||||||
|
ColumnValueSelector longValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_LONG_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(longValueSelector);
|
||||||
|
Assert.assertTrue(longValueSelector instanceof LongColumnSelector);
|
||||||
|
|
||||||
|
ColumnValueSelector doubleValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_DOUBLE_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(doubleValueSelector);
|
||||||
|
Assert.assertTrue(doubleValueSelector instanceof DoubleColumnSelector);
|
||||||
|
|
||||||
|
ColumnValueSelector mixedNumericValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_MIXED_NUMERIC_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(mixedNumericValueSelector);
|
||||||
|
Assert.assertTrue(mixedNumericValueSelector instanceof ColumnValueSelector);
|
||||||
|
|
||||||
|
ColumnValueSelector mixedValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_MIXED_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(mixedValueSelector);
|
||||||
|
Assert.assertTrue(mixedValueSelector instanceof ColumnValueSelector);
|
||||||
|
|
||||||
|
|
||||||
|
ColumnValueSelector sparseLongValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_SPARSE_LONG_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(sparseLongValueSelector);
|
||||||
|
Assert.assertTrue(sparseLongValueSelector instanceof LongColumnSelector);
|
||||||
|
|
||||||
|
ColumnValueSelector sparseDoubleValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_SPARSE_DOUBLE_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(sparseDoubleValueSelector);
|
||||||
|
Assert.assertTrue(sparseDoubleValueSelector instanceof DoubleColumnSelector);
|
||||||
|
|
||||||
|
ColumnValueSelector sparseMixedNumericValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_SPARSE_MIXED_NUMERIC_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(sparseMixedNumericValueSelector);
|
||||||
|
Assert.assertTrue(sparseMixedNumericValueSelector instanceof ColumnValueSelector);
|
||||||
|
|
||||||
|
ColumnValueSelector sparseMixedValueSelector = columnSelectorFactory.makeColumnValueSelector(
|
||||||
|
NESTED_SPARSE_MIXED_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(sparseMixedValueSelector);
|
||||||
|
Assert.assertTrue(sparseMixedValueSelector instanceof ColumnValueSelector);
|
||||||
|
//CHECKSTYLE.ON: Regexp
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testExpectedTypesVectorSelectors() throws Exception
|
||||||
|
{
|
||||||
|
// "Line matches the illegal pattern 'ObjectColumnSelector, LongColumnSelector, FloatColumnSelector
|
||||||
|
// and DoubleColumnSelector must not be used in an instanceof statement, see Javadoc of those interfaces."
|
||||||
|
//CHECKSTYLE.OFF: Regexp
|
||||||
|
VectorColumnSelectorFactory factory = getVectorColumnSelectorFactory(
|
||||||
|
makeNestedNumericVirtualColumns()
|
||||||
|
);
|
||||||
|
|
||||||
|
// can make numeric value selectors for single typed numeric types
|
||||||
|
VectorValueSelector longValueSelector = factory.makeValueSelector(
|
||||||
|
NESTED_LONG_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(longValueSelector);
|
||||||
|
Assert.assertTrue(longValueSelector instanceof BaseLongVectorValueSelector);
|
||||||
|
|
||||||
|
VectorValueSelector doubleValueSelector = factory.makeValueSelector(
|
||||||
|
NESTED_DOUBLE_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(doubleValueSelector);
|
||||||
|
Assert.assertTrue(doubleValueSelector instanceof BaseDoubleVectorValueSelector);
|
||||||
|
|
||||||
|
Assert.assertThrows(UOE.class, () -> factory.makeValueSelector(NESTED_MIXED_NUMERIC_FIELD));
|
||||||
|
Assert.assertThrows(UOE.class, () -> factory.makeValueSelector(NESTED_MIXED_FIELD));
|
||||||
|
|
||||||
|
// can also make single value dimension selectors for all nested column types
|
||||||
|
SingleValueDimensionVectorSelector longDimensionSelector = factory.makeSingleValueDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of(NESTED_LONG_FIELD)
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(longDimensionSelector);
|
||||||
|
|
||||||
|
SingleValueDimensionVectorSelector doubleDimensionSelector = factory.makeSingleValueDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of(NESTED_DOUBLE_FIELD)
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(doubleDimensionSelector);
|
||||||
|
|
||||||
|
SingleValueDimensionVectorSelector mixedNumericValueSelector = factory.makeSingleValueDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of(NESTED_MIXED_NUMERIC_FIELD)
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(mixedNumericValueSelector);
|
||||||
|
|
||||||
|
SingleValueDimensionVectorSelector mixedValueSelector = factory.makeSingleValueDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of(NESTED_MIXED_FIELD)
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(mixedValueSelector);
|
||||||
|
|
||||||
|
// and object selectors
|
||||||
|
VectorObjectSelector longObjectSelector = factory.makeObjectSelector(
|
||||||
|
NESTED_LONG_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(longObjectSelector);
|
||||||
|
|
||||||
|
VectorObjectSelector doubleObjectSelector = factory.makeObjectSelector(
|
||||||
|
NESTED_DOUBLE_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(doubleObjectSelector);
|
||||||
|
|
||||||
|
VectorObjectSelector mixedNumericObjectSelector = factory.makeObjectSelector(
|
||||||
|
NESTED_MIXED_NUMERIC_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(mixedNumericObjectSelector);
|
||||||
|
|
||||||
|
VectorObjectSelector mixedObjectSelector = factory.makeObjectSelector(
|
||||||
|
NESTED_MIXED_FIELD
|
||||||
|
);
|
||||||
|
Assert.assertNotNull(mixedObjectSelector);
|
||||||
|
//CHECKSTYLE.ON: Regexp
|
||||||
|
}
|
||||||
|
|
||||||
|
private VirtualColumns makeNestedNumericVirtualColumns()
|
||||||
|
{
|
||||||
|
List<NestedPathPart> longParts = NestedPathFinder.parseJqPath(".long");
|
||||||
|
List<NestedPathPart> doubleParts = NestedPathFinder.parseJqPath(".double");
|
||||||
|
List<NestedPathPart> mixedNumericParts = NestedPathFinder.parseJqPath(".mixed_numeric");
|
||||||
|
List<NestedPathPart> mixedParts = NestedPathFinder.parseJqPath(".mixed");
|
||||||
|
List<NestedPathPart> sparseLongParts = NestedPathFinder.parseJqPath(".sparse_long");
|
||||||
|
List<NestedPathPart> sparseDoubleParts = NestedPathFinder.parseJqPath(".sparse_double");
|
||||||
|
List<NestedPathPart> sparseMixedNumericParts = NestedPathFinder.parseJqPath(".sparse_mixed_numeric");
|
||||||
|
List<NestedPathPart> sparseMixedParts = NestedPathFinder.parseJqPath(".sparse_mixed");
|
||||||
|
|
||||||
|
NestedFieldVirtualColumn longVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_LONG_FIELD,
|
||||||
|
ColumnType.LONG,
|
||||||
|
longParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
NestedFieldVirtualColumn doubleVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_DOUBLE_FIELD,
|
||||||
|
ColumnType.DOUBLE,
|
||||||
|
doubleParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
NestedFieldVirtualColumn mixedNumericVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_MIXED_NUMERIC_FIELD,
|
||||||
|
null,
|
||||||
|
mixedNumericParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
NestedFieldVirtualColumn mixedVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_MIXED_FIELD,
|
||||||
|
null,
|
||||||
|
mixedParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
NestedFieldVirtualColumn sparseLongVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_SPARSE_LONG_FIELD,
|
||||||
|
ColumnType.LONG,
|
||||||
|
sparseLongParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
NestedFieldVirtualColumn sparseDoubleVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_SPARSE_DOUBLE_FIELD,
|
||||||
|
ColumnType.DOUBLE,
|
||||||
|
sparseDoubleParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
NestedFieldVirtualColumn sparseMixedNumericVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_SPARSE_MIXED_NUMERIC_FIELD,
|
||||||
|
null,
|
||||||
|
sparseMixedNumericParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
NestedFieldVirtualColumn sparseMixedVirtualColumn = new NestedFieldVirtualColumn(
|
||||||
|
"nest",
|
||||||
|
NESTED_SPARSE_MIXED_FIELD,
|
||||||
|
null,
|
||||||
|
sparseMixedParts,
|
||||||
|
false,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
return VirtualColumns.create(
|
||||||
|
ImmutableList.of(
|
||||||
|
longVirtualColumn,
|
||||||
|
doubleVirtualColumn,
|
||||||
|
mixedNumericVirtualColumn,
|
||||||
|
mixedVirtualColumn,
|
||||||
|
sparseLongVirtualColumn,
|
||||||
|
sparseDoubleVirtualColumn,
|
||||||
|
sparseMixedNumericVirtualColumn,
|
||||||
|
sparseMixedVirtualColumn
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
private ColumnSelectorFactory getNumericColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
|
||||||
|
{
|
||||||
|
List<Segment> segments = NestedDataTestUtils.createSegments(
|
||||||
|
helper,
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
NestedDataTestUtils.NUMERIC_DATA_FILE,
|
||||||
|
NestedDataTestUtils.NUMERIC_PARSER_FILE,
|
||||||
|
NestedDataTestUtils.SIMPLE_AGG_FILE,
|
||||||
|
Granularities.DAY,
|
||||||
|
true,
|
||||||
|
1000
|
||||||
|
);
|
||||||
|
Assert.assertEquals(1, segments.size());
|
||||||
|
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
||||||
|
Sequence<Cursor> cursorSequence = storageAdapter.makeCursors(
|
||||||
|
null,
|
||||||
|
Intervals.ETERNITY,
|
||||||
|
virtualColumns,
|
||||||
|
Granularities.DAY,
|
||||||
|
false,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
final Yielder<Cursor> yielder = Yielders.each(cursorSequence);
|
||||||
|
closer.register(yielder);
|
||||||
|
final Cursor cursor = yielder.get();
|
||||||
|
return cursor.getColumnSelectorFactory();
|
||||||
|
}
|
||||||
|
|
||||||
|
private VectorColumnSelectorFactory getVectorColumnSelectorFactory(VirtualColumns virtualColumns) throws Exception
|
||||||
|
{
|
||||||
|
List<Segment> segments = NestedDataTestUtils.createSegments(
|
||||||
|
helper,
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
NestedDataTestUtils.NUMERIC_DATA_FILE,
|
||||||
|
NestedDataTestUtils.NUMERIC_PARSER_FILE,
|
||||||
|
NestedDataTestUtils.SIMPLE_AGG_FILE,
|
||||||
|
Granularities.DAY,
|
||||||
|
true,
|
||||||
|
1000
|
||||||
|
);
|
||||||
|
Assert.assertEquals(1, segments.size());
|
||||||
|
StorageAdapter storageAdapter = segments.get(0).asStorageAdapter();
|
||||||
|
VectorCursor cursor = storageAdapter.makeVectorCursor(
|
||||||
|
null,
|
||||||
|
Intervals.ETERNITY,
|
||||||
|
virtualColumns,
|
||||||
|
false,
|
||||||
|
512,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
return cursor.getColumnSelectorFactory();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,16 @@
|
||||||
|
{
|
||||||
|
"type": "string",
|
||||||
|
"parseSpec": {
|
||||||
|
"format": "json",
|
||||||
|
"timestampSpec": {
|
||||||
|
"column": "timestamp",
|
||||||
|
"format": "auto"
|
||||||
|
},
|
||||||
|
"dimensionsSpec": {
|
||||||
|
"dimensions": [],
|
||||||
|
"dimensionExclusions": [],
|
||||||
|
"spatialDimensions": [],
|
||||||
|
"useNestedColumnIndexerForSchemaDiscovery": true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,8 @@
|
||||||
|
{"timestamp": "2021-01-01", "str":"a", "long":1, "double":1.0, "variant": 1}
|
||||||
|
{"timestamp": "2021-01-01", "str":"", "long":2, "variant": "b"}
|
||||||
|
{"timestamp": "2021-01-01", "str":"null", "long":3, "double":2.0, "variant": 3.0}
|
||||||
|
{"timestamp": "2021-01-01", "str":"b", "long":4, "double":3.3, "variant": "4"}
|
||||||
|
{"timestamp": "2021-01-01", "str":"c", "long": null, "double":4.4, "variant": "hello"}
|
||||||
|
{"timestamp": "2021-01-01", "str":"d", "long":5, "double":5.9}
|
||||||
|
{"timestamp": "2021-01-01", "str":null, "double":null, "variant": 51}
|
||||||
|
{"timestamp": "2021-01-01", "long":6, "double":1.0, "variant": null}
|
|
@ -140,11 +140,11 @@ public class LookupDimensionSpecTest
|
||||||
},
|
},
|
||||||
new Object[]{
|
new Object[]{
|
||||||
new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, true, null),
|
new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, null, null, true, null),
|
||||||
TestHelper.createExpectedMap("not there", null)
|
TestHelper.makeMap("not there", null)
|
||||||
},
|
},
|
||||||
new Object[]{
|
new Object[]{
|
||||||
new LookupDimensionSpec("dimName", "outputName", null, false, null, "lookupName", true, LOOKUP_REF_MANAGER),
|
new LookupDimensionSpec("dimName", "outputName", null, false, null, "lookupName", true, LOOKUP_REF_MANAGER),
|
||||||
TestHelper.createExpectedMap("not there", null)
|
TestHelper.makeMap("not there", null)
|
||||||
},
|
},
|
||||||
new Object[]{
|
new Object[]{
|
||||||
new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null,
|
new LookupDimensionSpec("dimName", "outputName", MAP_LOOKUP_EXTRACTOR, false, "Missing_value", null,
|
||||||
|
|
Loading…
Reference in New Issue