mirror of https://github.com/apache/druid.git
fix NPE that can happen when merging all null nested v4 format columns (#14068)
This commit is contained in:
parent
5ce1b0903e
commit
29652bd246
|
@ -134,7 +134,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
||||||
Supplier<TStringDictionary> stringDictionary,
|
Supplier<TStringDictionary> stringDictionary,
|
||||||
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
||||||
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
||||||
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
@Nullable Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
||||||
SmooshedFileMapper fileMapper,
|
SmooshedFileMapper fileMapper,
|
||||||
BitmapSerdeFactory bitmapSerdeFactory,
|
BitmapSerdeFactory bitmapSerdeFactory,
|
||||||
ByteOrder byteOrder,
|
ByteOrder byteOrder,
|
||||||
|
@ -220,6 +220,9 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
||||||
@Override
|
@Override
|
||||||
public Indexed<Object[]> getArrayDictionary()
|
public Indexed<Object[]> getArrayDictionary()
|
||||||
{
|
{
|
||||||
|
if (arrayDictionarySupplier == null) {
|
||||||
|
return Indexed.empty();
|
||||||
|
}
|
||||||
Iterable<Object[]> arrays = () -> {
|
Iterable<Object[]> arrays = () -> {
|
||||||
final TStringDictionary stringDictionary = stringDictionarySupplier.get();
|
final TStringDictionary stringDictionary = stringDictionarySupplier.get();
|
||||||
final FixedIndexed<Long> longDictionary = longDictionarySupplier.get();
|
final FixedIndexed<Long> longDictionary = longDictionarySupplier.get();
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||||
|
import org.apache.druid.segment.AutoTypeColumnSchema;
|
||||||
import org.apache.druid.segment.IncrementalIndexSegment;
|
import org.apache.druid.segment.IncrementalIndexSegment;
|
||||||
import org.apache.druid.segment.IndexBuilder;
|
import org.apache.druid.segment.IndexBuilder;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
|
@ -92,7 +93,7 @@ public class NestedDataTestUtils
|
||||||
.useSchemaDiscovery(true)
|
.useSchemaDiscovery(true)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
public static final DimensionsSpec TSV_SCHEMA =
|
public static final DimensionsSpec TSV_V4_SCHEMA =
|
||||||
DimensionsSpec.builder()
|
DimensionsSpec.builder()
|
||||||
.setDimensions(
|
.setDimensions(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
|
@ -100,7 +101,22 @@ public class NestedDataTestUtils
|
||||||
new NestedDataDimensionSchema("nest_json"),
|
new NestedDataDimensionSchema("nest_json"),
|
||||||
new NestedDataDimensionSchema("nester_json"),
|
new NestedDataDimensionSchema("nester_json"),
|
||||||
new NestedDataDimensionSchema("variant_json"),
|
new NestedDataDimensionSchema("variant_json"),
|
||||||
new NestedDataDimensionSchema("list_json")
|
new NestedDataDimensionSchema("list_json"),
|
||||||
|
new NestedDataDimensionSchema("nonexistent")
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final DimensionsSpec TSV_SCHEMA =
|
||||||
|
DimensionsSpec.builder()
|
||||||
|
.setDimensions(
|
||||||
|
Arrays.asList(
|
||||||
|
new AutoTypeColumnSchema("dim"),
|
||||||
|
new AutoTypeColumnSchema("nest_json"),
|
||||||
|
new AutoTypeColumnSchema("nester_json"),
|
||||||
|
new AutoTypeColumnSchema("variant_json"),
|
||||||
|
new AutoTypeColumnSchema("list_json"),
|
||||||
|
new AutoTypeColumnSchema("nonexistent")
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.build();
|
.build();
|
||||||
|
@ -110,12 +126,6 @@ public class NestedDataTestUtils
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
public static final InputRowSchema SIMPLE_DATA_TSV_SCHEMA = new InputRowSchema(
|
|
||||||
TIMESTAMP_SPEC,
|
|
||||||
TSV_SCHEMA,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
|
|
||||||
public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = new DelimitedInputFormat(
|
public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = new DelimitedInputFormat(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"timestamp",
|
"timestamp",
|
||||||
|
@ -161,6 +171,22 @@ public class NestedDataTestUtils
|
||||||
tempFolder,
|
tempFolder,
|
||||||
closer,
|
closer,
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
|
TSV_SCHEMA,
|
||||||
|
true
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static List<Segment> createSimpleSegmentsTsvV4(
|
||||||
|
TemporaryFolder tempFolder,
|
||||||
|
Closer closer
|
||||||
|
)
|
||||||
|
throws Exception
|
||||||
|
{
|
||||||
|
return createSimpleNestedTestDataTsvSegments(
|
||||||
|
tempFolder,
|
||||||
|
closer,
|
||||||
|
Granularities.NONE,
|
||||||
|
TSV_V4_SCHEMA,
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -169,6 +195,7 @@ public class NestedDataTestUtils
|
||||||
TemporaryFolder tempFolder,
|
TemporaryFolder tempFolder,
|
||||||
Closer closer,
|
Closer closer,
|
||||||
Granularity granularity,
|
Granularity granularity,
|
||||||
|
DimensionsSpec dimensionsSpec,
|
||||||
boolean rollup
|
boolean rollup
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
|
@ -178,7 +205,7 @@ public class NestedDataTestUtils
|
||||||
SIMPLE_DATA_TSV_FILE,
|
SIMPLE_DATA_TSV_FILE,
|
||||||
SIMPLE_DATA_TSV_INPUT_FORMAT,
|
SIMPLE_DATA_TSV_INPUT_FORMAT,
|
||||||
TIMESTAMP_SPEC,
|
TIMESTAMP_SPEC,
|
||||||
SIMPLE_DATA_TSV_SCHEMA.getDimensionsSpec(),
|
dimensionsSpec,
|
||||||
SIMPLE_DATA_TSV_TRANSFORM,
|
SIMPLE_DATA_TSV_TRANSFORM,
|
||||||
COUNT,
|
COUNT,
|
||||||
granularity,
|
granularity,
|
||||||
|
|
|
@ -262,6 +262,35 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
||||||
Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
|
Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testIngestAndScanSegmentsTsvV4() throws Exception
|
||||||
|
{
|
||||||
|
Query<ScanResultValue> scanQuery = Druids.newScanQueryBuilder()
|
||||||
|
.dataSource("test_datasource")
|
||||||
|
.intervals(
|
||||||
|
new MultipleIntervalSegmentSpec(
|
||||||
|
Collections.singletonList(Intervals.ETERNITY)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.virtualColumns(
|
||||||
|
new NestedFieldVirtualColumn("nest", "$.x", "x"),
|
||||||
|
new NestedFieldVirtualColumn("nester", "$.x[0]", "x_0"),
|
||||||
|
new NestedFieldVirtualColumn("nester", "$.y.c[1]", "y_c_1")
|
||||||
|
)
|
||||||
|
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
|
.limit(100)
|
||||||
|
.context(ImmutableMap.of())
|
||||||
|
.build();
|
||||||
|
List<Segment> segs = NestedDataTestUtils.createSimpleSegmentsTsvV4(tempFolder, closer);
|
||||||
|
|
||||||
|
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 testIngestAndScanSegmentsTsv() throws Exception
|
public void testIngestAndScanSegmentsTsv() throws Exception
|
||||||
{
|
{
|
||||||
|
|
Loading…
Reference in New Issue