mirror of https://github.com/apache/druid.git
auto columns fixes (#14422)
changes: * auto columns no longer participate in generic 'null column' handling, this was a mistake to try to support and caused ingestion failures due to mismatched ColumnFormat, and will be replaced in the future with nested common format constant column functionality (not in this PR) * fix bugs with auto columns which contain empty objects, empty arrays, or primitive types mixed with either of these empty constructs * fix bug with bound filter when upper is null equivalent but is strict
This commit is contained in:
parent
be5a6593a9
commit
8454cc619a
|
@ -84,7 +84,7 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
|||
new LongDimensionSchema("long"),
|
||||
new DoubleDimensionSchema("double"),
|
||||
new StringDimensionSchema("bool"),
|
||||
new AutoTypeColumnSchema("variant"),
|
||||
new StringDimensionSchema("variant"),
|
||||
new AutoTypeColumnSchema("array"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
),
|
||||
|
@ -110,7 +110,7 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
|||
.add("long", ColumnType.LONG)
|
||||
.add("double", ColumnType.DOUBLE)
|
||||
.add("bool", ColumnType.STRING)
|
||||
.add("variant", ColumnType.NESTED_DATA)
|
||||
.add("variant", ColumnType.STRING)
|
||||
.add("array", ColumnType.LONG_ARRAY)
|
||||
.add("nested", ColumnType.NESTED_DATA)
|
||||
.build(),
|
||||
|
@ -147,7 +147,7 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
|||
new LongDimensionSchema("long"),
|
||||
new DoubleDimensionSchema("double"),
|
||||
new LongDimensionSchema("bool"),
|
||||
new AutoTypeColumnSchema("variant"),
|
||||
new StringDimensionSchema("variant"),
|
||||
new AutoTypeColumnSchema("array"),
|
||||
new AutoTypeColumnSchema("nested")
|
||||
),
|
||||
|
@ -173,7 +173,7 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
|
|||
.add("long", ColumnType.LONG)
|
||||
.add("double", ColumnType.DOUBLE)
|
||||
.add("bool", ColumnType.LONG)
|
||||
.add("variant", ColumnType.NESTED_DATA)
|
||||
.add("variant", ColumnType.STRING)
|
||||
.add("array", ColumnType.LONG_ARRAY)
|
||||
.add("nested", ColumnType.NESTED_DATA)
|
||||
.build(),
|
||||
|
|
|
@ -346,6 +346,11 @@ public class SegmentAnalyzer
|
|||
.withTypeName(typeName);
|
||||
|
||||
try (final BaseColumn theColumn = columnHolder != null ? columnHolder.getColumn() : null) {
|
||||
if (capabilities != null) {
|
||||
bob.hasMultipleValues(capabilities.hasMultipleValues().isTrue())
|
||||
.hasNulls(capabilities.hasNulls().isMaybeTrue());
|
||||
}
|
||||
|
||||
if (theColumn != null && !(theColumn instanceof ComplexColumn)) {
|
||||
return bob.withErrorMessage(
|
||||
StringUtils.format(
|
||||
|
@ -358,9 +363,6 @@ public class SegmentAnalyzer
|
|||
}
|
||||
final ComplexColumn complexColumn = (ComplexColumn) theColumn;
|
||||
|
||||
bob.hasMultipleValues(capabilities.hasMultipleValues().isTrue())
|
||||
.hasNulls(capabilities.hasNulls().isMaybeTrue());
|
||||
|
||||
long size = 0;
|
||||
if (analyzingSize() && complexColumn != null) {
|
||||
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.segment.column.ColumnCapabilities;
|
|||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnFormat;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.ColumnTypeFactory;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexRowHolder;
|
||||
|
@ -56,7 +57,19 @@ import java.util.TreeMap;
|
|||
|
||||
public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, StructuredData, StructuredData>
|
||||
{
|
||||
/**
|
||||
* have we seen any null values?
|
||||
*/
|
||||
protected volatile boolean hasNulls = false;
|
||||
/**
|
||||
* Have we seen any objects? Empty objects in particular are sneaky, they don't have any nested paths, so we also
|
||||
* broadly track if we have processed any objects {@link StructuredDataProcessor.ProcessResults#hasObjects()}
|
||||
*/
|
||||
protected volatile boolean hasNestedData = false;
|
||||
protected volatile boolean isConstant = true;
|
||||
@Nullable
|
||||
protected volatile Object constantValue = null;
|
||||
private volatile boolean firstRow = true;
|
||||
|
||||
protected SortedMap<String, FieldIndexer> fieldIndexers = new TreeMap<>();
|
||||
protected final ValueDictionary globalDictionary = new ValueDictionary();
|
||||
|
@ -111,6 +124,12 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
boolean reportParseExceptions
|
||||
)
|
||||
{
|
||||
if (firstRow) {
|
||||
constantValue = dimValues;
|
||||
firstRow = false;
|
||||
} else if (isConstant) {
|
||||
isConstant = Objects.equals(dimValues, constantValue);
|
||||
}
|
||||
final long oldDictSizeInBytes = globalDictionary.sizeInBytes();
|
||||
final int oldFieldKeySize = estimatedFieldKeySize;
|
||||
final StructuredData data;
|
||||
|
@ -122,7 +141,12 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
} else {
|
||||
data = new StructuredData(dimValues);
|
||||
}
|
||||
StructuredDataProcessor.ProcessResults info = indexerProcessor.processFields(data == null ? null : data.getValue());
|
||||
final StructuredDataProcessor.ProcessResults info = indexerProcessor.processFields(
|
||||
data == null ? null : data.getValue()
|
||||
);
|
||||
if (info.hasObjects()) {
|
||||
hasNestedData = true;
|
||||
}
|
||||
// 'raw' data is currently preserved 'as-is', and not replaced with object references to the global dictionaries
|
||||
long effectiveSizeBytes = info.getEstimatedSize();
|
||||
// then, we add the delta of size change to the global dictionaries to account for any new space added by the
|
||||
|
@ -136,6 +160,12 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
public void setSparseIndexed()
|
||||
{
|
||||
this.hasNulls = true;
|
||||
if (firstRow) {
|
||||
firstRow = false;
|
||||
} else if (constantValue != null) {
|
||||
constantValue = null;
|
||||
isConstant = false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -164,10 +194,6 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
fields.put(entry.getKey(), entry.getValue().getTypes());
|
||||
}
|
||||
}
|
||||
// special handling for when column only has arrays with null elements, treat it as a string array
|
||||
if (fields.isEmpty() && fieldIndexers.size() == 1) {
|
||||
fields.put(fieldIndexers.firstKey(), new FieldTypeInfo.MutableTypeSet().add(ColumnType.STRING_ARRAY));
|
||||
}
|
||||
return fields;
|
||||
}
|
||||
|
||||
|
@ -284,20 +310,43 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
.setHasNulls(hasNulls);
|
||||
}
|
||||
|
||||
private ColumnType getLogicalType()
|
||||
public ColumnType getLogicalType()
|
||||
{
|
||||
if (fieldIndexers.isEmpty()) {
|
||||
if (hasNestedData) {
|
||||
return ColumnType.NESTED_DATA;
|
||||
}
|
||||
if (isConstant && constantValue == null) {
|
||||
// we didn't see anything, so we can be anything, so why not a string?
|
||||
return ColumnType.STRING;
|
||||
}
|
||||
if (fieldIndexers.size() == 1 && fieldIndexers.containsKey(NestedPathFinder.JSON_PATH_ROOT)) {
|
||||
FieldIndexer rootField = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
ColumnType singleType = rootField.getTypes().getSingleType();
|
||||
return singleType == null ? ColumnType.NESTED_DATA : singleType;
|
||||
ColumnType logicalType = null;
|
||||
for (ColumnType type : FieldTypeInfo.convertToSet(rootField.getTypes().getByteValue())) {
|
||||
logicalType = ColumnType.leastRestrictiveType(logicalType, type);
|
||||
}
|
||||
if (logicalType != null) {
|
||||
// special handle empty arrays
|
||||
if (!rootField.getTypes().hasUntypedArray() || logicalType.isArray()) {
|
||||
return logicalType;
|
||||
}
|
||||
return ColumnTypeFactory.getInstance().ofArray(logicalType);
|
||||
}
|
||||
}
|
||||
return ColumnType.NESTED_DATA;
|
||||
}
|
||||
|
||||
public boolean isConstant()
|
||||
{
|
||||
return isConstant;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public Object getConstantValue()
|
||||
{
|
||||
return constantValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnFormat getFormat()
|
||||
{
|
||||
|
@ -517,26 +566,27 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
eval.type()
|
||||
);
|
||||
}
|
||||
|
||||
final Object[] theArray = eval.asArray();
|
||||
switch (columnType.getElementType().getType()) {
|
||||
case LONG:
|
||||
typeSet.add(ColumnType.LONG_ARRAY);
|
||||
final Object[] longArray = eval.asArray();
|
||||
sizeEstimate = valueDictionary.addLongArray(longArray);
|
||||
return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
|
||||
sizeEstimate = valueDictionary.addLongArray(theArray);
|
||||
return new StructuredDataProcessor.ProcessedValue<>(theArray, sizeEstimate);
|
||||
case DOUBLE:
|
||||
typeSet.add(ColumnType.DOUBLE_ARRAY);
|
||||
final Object[] doubleArray = eval.asArray();
|
||||
sizeEstimate = valueDictionary.addDoubleArray(doubleArray);
|
||||
return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
|
||||
sizeEstimate = valueDictionary.addDoubleArray(theArray);
|
||||
return new StructuredDataProcessor.ProcessedValue<>(theArray, sizeEstimate);
|
||||
case STRING:
|
||||
final Object[] stringArray = eval.asArray();
|
||||
// empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
|
||||
// the type set
|
||||
if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
|
||||
// empty arrays and arrays with all nulls are detected as string arrays, but don't count them as part of
|
||||
// the type set yet, we'll handle that later when serializing
|
||||
if (theArray.length == 0 || Arrays.stream(theArray).allMatch(Objects::isNull)) {
|
||||
typeSet.addUntypedArray();
|
||||
} else {
|
||||
typeSet.add(ColumnType.STRING_ARRAY);
|
||||
}
|
||||
sizeEstimate = valueDictionary.addStringArray(stringArray);
|
||||
return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
|
||||
sizeEstimate = valueDictionary.addStringArray(theArray);
|
||||
return new StructuredDataProcessor.ProcessedValue<>(theArray, sizeEstimate);
|
||||
default:
|
||||
throw new IAE("Unhandled type: %s", columnType);
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.io.Closer;
|
|||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.column.ColumnDescriptor;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.ColumnTypeFactory;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
|
@ -82,7 +83,6 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
|
||||
private ColumnType logicalType;
|
||||
private boolean isVariantType = false;
|
||||
private boolean hasOnlyNulls = false;
|
||||
|
||||
public AutoTypeColumnMerger(
|
||||
String name,
|
||||
|
@ -113,6 +113,11 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
|
||||
final SortedMap<String, FieldTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
|
||||
|
||||
boolean forceNested = false;
|
||||
Object constantValue = null;
|
||||
boolean hasArrays = false;
|
||||
boolean isConstant = true;
|
||||
|
||||
for (int i = 0; i < adapters.size(); i++) {
|
||||
final IndexableAdapter adapter = adapters.get(i);
|
||||
final IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
|
@ -121,31 +126,47 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
if (mergable == null) {
|
||||
continue;
|
||||
}
|
||||
forceNested = forceNested || mergable.isForceNestedType();
|
||||
isConstant = isConstant && mergable.isConstant();
|
||||
constantValue = mergable.getConstantValue();
|
||||
|
||||
final SortedValueDictionary dimValues = mergable.getValueDictionary();
|
||||
|
||||
boolean allNulls = dimValues == null || dimValues.allNull();
|
||||
sortedLookup = dimValues;
|
||||
if (!allNulls) {
|
||||
sortedLookup = dimValues;
|
||||
mergable.mergeFieldsInto(mergedFields);
|
||||
sortedLookups[i] = dimValues.getSortedStrings();
|
||||
sortedLongLookups[i] = dimValues.getSortedLongs();
|
||||
sortedDoubleLookups[i] = dimValues.getSortedDoubles();
|
||||
sortedArrayLookups[i] = dimValues.getSortedArrays();
|
||||
hasArrays = sortedArrayLookups[i].size() > 0;
|
||||
numMergeIndex++;
|
||||
}
|
||||
}
|
||||
|
||||
// no data, we don't need to write this column
|
||||
if (numMergeIndex == 0 && mergedFields.size() == 0) {
|
||||
hasOnlyNulls = true;
|
||||
return;
|
||||
}
|
||||
|
||||
// check to see if we can specialize the serializer after merging all the adapters
|
||||
final FieldTypeInfo.MutableTypeSet rootTypes = mergedFields.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
final boolean rootOnly = mergedFields.size() == 1 && rootTypes != null;
|
||||
if (rootOnly && rootTypes.getSingleType() != null) {
|
||||
|
||||
|
||||
// for backwards compat; remove this constant handling in druid 28 along with
|
||||
// indexSpec.optimizeJsonConstantColumns in favor of always writing constant columns
|
||||
// we also handle the numMergeIndex == 0 here, which also indicates that the column is a null constant
|
||||
if (!forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) {
|
||||
logicalType = ColumnType.STRING;
|
||||
serializer = new ScalarStringColumnSerializer(
|
||||
name,
|
||||
indexSpec,
|
||||
segmentWriteOutMedium,
|
||||
closer
|
||||
);
|
||||
} else if (!forceNested && rootOnly && rootTypes.getSingleType() != null) {
|
||||
logicalType = rootTypes.getSingleType();
|
||||
// empty arrays can be missed since they don't have a type, so handle them here
|
||||
if (!logicalType.isArray() && hasArrays) {
|
||||
logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType);
|
||||
}
|
||||
switch (logicalType.getType()) {
|
||||
case LONG:
|
||||
serializer = new ScalarLongColumnSerializer(
|
||||
|
@ -187,13 +208,17 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
logicalType
|
||||
);
|
||||
}
|
||||
} else if (rootOnly) {
|
||||
} else if (!forceNested && rootOnly) {
|
||||
// mixed type column, but only root path, we can use VariantArrayColumnSerializer
|
||||
// pick the least restrictive type for the logical type
|
||||
isVariantType = true;
|
||||
for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) {
|
||||
logicalType = ColumnType.leastRestrictiveType(logicalType, type);
|
||||
}
|
||||
// empty arrays can be missed since they don't have a type, so handle them here
|
||||
if (!logicalType.isArray() && hasArrays) {
|
||||
logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType);
|
||||
}
|
||||
serializer = new VariantColumnSerializer(
|
||||
name,
|
||||
rootTypes.getByteValue(),
|
||||
|
@ -307,7 +332,8 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
@Override
|
||||
public boolean hasOnlyNulls()
|
||||
{
|
||||
return hasOnlyNulls;
|
||||
// we handle this internally using a constant column instead of using the generic null part serde
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -38,7 +38,7 @@ import java.util.Objects;
|
|||
/**
|
||||
* IndexSpec defines segment storage format options to be used at indexing time,
|
||||
* such as bitmap type, and column compression formats.
|
||||
*
|
||||
* <p>
|
||||
* IndexSpec is specified as part of the TuningConfig for the corresponding index task.
|
||||
*/
|
||||
public class IndexSpec
|
||||
|
@ -58,31 +58,24 @@ public class IndexSpec
|
|||
|
||||
@Nullable
|
||||
private final CompressionStrategy jsonCompression;
|
||||
|
||||
@Nullable
|
||||
private final SegmentizerFactory segmentLoader;
|
||||
|
||||
/**
|
||||
* Creates an IndexSpec with the given storage format settings.
|
||||
*
|
||||
*
|
||||
* @param bitmapSerdeFactory type of bitmap to use (e.g. roaring or concise), null to use the default.
|
||||
* Defaults to the bitmap type specified by the (deprecated) "druid.processing.bitmap.type"
|
||||
* setting, or, if none was set, uses the default defined in {@link BitmapSerde}
|
||||
*
|
||||
* @param dimensionCompression compression format for dimension columns, null to use the default.
|
||||
* Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY}
|
||||
*
|
||||
* @param bitmapSerdeFactory type of bitmap to use (e.g. roaring or concise), null to use the default.
|
||||
* Defaults to the bitmap type specified by the (deprecated) "druid.processing.bitmap.type"
|
||||
* setting, or, if none was set, uses the default defined in {@link BitmapSerde}
|
||||
* @param dimensionCompression compression format for dimension columns, null to use the default.
|
||||
* Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY}
|
||||
* @param stringDictionaryEncoding encoding strategy for string dictionaries of dictionary encoded string columns
|
||||
*
|
||||
* @param metricCompression compression format for primitive type metric columns, null to use the default.
|
||||
* Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY}
|
||||
*
|
||||
* @param longEncoding encoding strategy for metric and dimension columns with type long, null to use the default.
|
||||
* Defaults to {@link CompressionFactory#DEFAULT_LONG_ENCODING_STRATEGY}
|
||||
*
|
||||
* @param segmentLoader specify a {@link SegmentizerFactory} which will be written to 'factory.json' and used to load
|
||||
* the written segment
|
||||
* @param metricCompression compression format for primitive type metric columns, null to use the default.
|
||||
* Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY}
|
||||
* @param longEncoding encoding strategy for metric and dimension columns with type long, null to use the default.
|
||||
* Defaults to {@link CompressionFactory#DEFAULT_LONG_ENCODING_STRATEGY}
|
||||
* @param segmentLoader specify a {@link SegmentizerFactory} which will be written to 'factory.json' and used to load
|
||||
* the written segment
|
||||
*/
|
||||
@JsonCreator
|
||||
public IndexSpec(
|
||||
|
@ -165,7 +158,9 @@ public class IndexSpec
|
|||
{
|
||||
return objectMapper.convertValue(
|
||||
this,
|
||||
new TypeReference<Map<String, Object>>() {}
|
||||
new TypeReference<Map<String, Object>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -191,7 +186,15 @@ public class IndexSpec
|
|||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(bitmapSerdeFactory, dimensionCompression, stringDictionaryEncoding, metricCompression, longEncoding, jsonCompression, segmentLoader);
|
||||
return Objects.hash(
|
||||
bitmapSerdeFactory,
|
||||
dimensionCompression,
|
||||
stringDictionaryEncoding,
|
||||
metricCompression,
|
||||
longEncoding,
|
||||
jsonCompression,
|
||||
segmentLoader
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -236,6 +239,7 @@ public class IndexSpec
|
|||
this.dimensionCompression = dimensionCompression;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withStringDictionaryEncoding(StringEncodingStrategy stringDictionaryEncoding)
|
||||
{
|
||||
this.stringDictionaryEncoding = stringDictionaryEncoding;
|
||||
|
|
|
@ -73,14 +73,40 @@ public interface IndexableAdapter
|
|||
{
|
||||
private final SortedValueDictionary valueDictionary;
|
||||
private final SortedMap<String, FieldTypeInfo.MutableTypeSet> fields;
|
||||
private final boolean forceNestedType;
|
||||
private final boolean isConstant;
|
||||
@Nullable
|
||||
private final Object constantValue;
|
||||
|
||||
public NestedColumnMergable(
|
||||
SortedValueDictionary valueDictionary,
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields,
|
||||
boolean forceNestedType,
|
||||
boolean isConstant,
|
||||
@Nullable Object constantValue
|
||||
)
|
||||
{
|
||||
this.valueDictionary = valueDictionary;
|
||||
this.fields = fields;
|
||||
this.forceNestedType = forceNestedType;
|
||||
this.isConstant = isConstant;
|
||||
this.constantValue = constantValue;
|
||||
}
|
||||
|
||||
public boolean isForceNestedType()
|
||||
{
|
||||
return forceNestedType;
|
||||
}
|
||||
|
||||
public boolean isConstant()
|
||||
{
|
||||
return isConstant;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public Object getConstantValue()
|
||||
{
|
||||
return constantValue;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -96,9 +122,9 @@ public interface IndexableAdapter
|
|||
final FieldTypeInfo.MutableTypeSet types = entry.getValue();
|
||||
mergeInto.compute(fieldPath, (k, v) -> {
|
||||
if (v == null) {
|
||||
return new FieldTypeInfo.MutableTypeSet(types.getByteValue());
|
||||
return types;
|
||||
}
|
||||
return v.merge(types.getByteValue());
|
||||
return v.merge(types.getByteValue(), types.hasUntypedArray());
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -99,8 +99,8 @@ public class NestedDataColumnMerger implements DimensionMergerV9
|
|||
final SortedValueDictionary dimValues = mergable.getValueDictionary();
|
||||
|
||||
boolean allNulls = dimValues == null || dimValues.allNull();
|
||||
sortedLookup = dimValues;
|
||||
if (!allNulls) {
|
||||
sortedLookup = dimValues;
|
||||
mergable.mergeFieldsInto(mergedFields);
|
||||
sortedLookups[i] = dimValues.getSortedStrings();
|
||||
sortedLongLookups[i] = dimValues.getSortedLongs();
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.druid.segment.column.ColumnCapabilities;
|
|||
import org.apache.druid.segment.column.ColumnFormat;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedValueIndex;
|
||||
import org.apache.druid.segment.data.BitmapValues;
|
||||
|
@ -192,7 +193,10 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
column.getArrayDictionary(),
|
||||
column
|
||||
),
|
||||
column.getFieldTypeInfo()
|
||||
column.getFieldTypeInfo(),
|
||||
ColumnType.NESTED_DATA.equals(column.getLogicalType()),
|
||||
false,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -92,10 +92,10 @@ public class BoundFilter implements Filter
|
|||
);
|
||||
if (rangeBitmaps != null) {
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (boundDimFilter.hasLowerBound() && !NullHandling.isNullOrEquivalent(boundDimFilter.getLower())) {
|
||||
return rangeBitmaps;
|
||||
} else {
|
||||
if (doesMatchNull()) {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
} else {
|
||||
return rangeBitmaps;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -123,10 +123,10 @@ public class BoundFilter implements Filter
|
|||
);
|
||||
if (rangeBitmaps != null) {
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (boundDimFilter.hasLowerBound() && !NullHandling.isNullOrEquivalent(boundDimFilter.getLower())) {
|
||||
return rangeBitmaps;
|
||||
} else {
|
||||
if (doesMatchNull()) {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
} else {
|
||||
return rangeBitmaps;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.segment.NestedDataColumnIndexer;
|
|||
import org.apache.druid.segment.TransformableRowIterator;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnFormat;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.data.BitmapValues;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -155,14 +156,20 @@ public class IncrementalIndexAdapter implements IndexableAdapter
|
|||
|
||||
return new NestedColumnMergable(
|
||||
nestedDataColumnIndexer.getSortedValueLookups(),
|
||||
nestedDataColumnIndexer.getFieldTypeInfo()
|
||||
nestedDataColumnIndexer.getFieldTypeInfo(),
|
||||
true,
|
||||
false,
|
||||
null
|
||||
);
|
||||
}
|
||||
if (indexer instanceof AutoTypeColumnIndexer) {
|
||||
AutoTypeColumnIndexer autoIndexer = (AutoTypeColumnIndexer) indexer;
|
||||
return new NestedColumnMergable(
|
||||
autoIndexer.getSortedValueLookups(),
|
||||
autoIndexer.getFieldTypeInfo()
|
||||
autoIndexer.getFieldTypeInfo(),
|
||||
autoIndexer.getLogicalType().equals(ColumnType.NESTED_DATA),
|
||||
autoIndexer.isConstant(),
|
||||
autoIndexer.getConstantValue()
|
||||
);
|
||||
}
|
||||
return null;
|
||||
|
|
|
@ -326,7 +326,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
@Override
|
||||
public ColumnValueSelector<?> makeColumnValueSelector(ReadableOffset offset)
|
||||
{
|
||||
if (fields.size() == 1 && rootFieldPath.equals(fields.get(0))) {
|
||||
if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(fields.get(0))) {
|
||||
return makeColumnValueSelector(
|
||||
ImmutableList.of(),
|
||||
offset
|
||||
|
@ -366,7 +366,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
@Override
|
||||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
if (fields.size() == 1 && rootFieldPath.equals(fields.get(0))) {
|
||||
if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(fields.get(0))) {
|
||||
return makeVectorObjectSelector(
|
||||
Collections.emptyList(),
|
||||
offset
|
||||
|
@ -437,7 +437,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
@Override
|
||||
public VectorValueSelector makeVectorValueSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
if (fields.size() == 1 && rootFieldPath.equals(fields.get(0))) {
|
||||
if (!logicalType.equals(ColumnType.NESTED_DATA) && fields.size() == 1 && rootFieldPath.equals(fields.get(0))) {
|
||||
return makeVectorValueSelector(
|
||||
Collections.emptyList(),
|
||||
offset
|
||||
|
|
|
@ -44,13 +44,20 @@ public class FieldTypeInfo
|
|||
private static final byte STRING_MASK = 1;
|
||||
private static final byte LONG_MASK = 1 << 2;
|
||||
private static final byte DOUBLE_MASK = 1 << 3;
|
||||
|
||||
private static final byte STRING_ARRAY_MASK = 1 << 4;
|
||||
|
||||
private static final byte LONG_ARRAY_MASK = 1 << 5;
|
||||
|
||||
private static final byte DOUBLE_ARRAY_MASK = 1 << 6;
|
||||
|
||||
private static final ColumnType[] TYPES = new ColumnType[]{
|
||||
ColumnType.STRING,
|
||||
null, // mistakes were made...
|
||||
ColumnType.LONG,
|
||||
ColumnType.DOUBLE,
|
||||
ColumnType.STRING_ARRAY,
|
||||
ColumnType.LONG_ARRAY,
|
||||
ColumnType.DOUBLE_ARRAY
|
||||
};
|
||||
|
||||
public static FieldTypeInfo read(ByteBuffer buffer, int length)
|
||||
{
|
||||
FieldTypeInfo typeInfo = new FieldTypeInfo(buffer);
|
||||
|
@ -124,6 +131,7 @@ public class FieldTypeInfo
|
|||
public static class MutableTypeSet
|
||||
{
|
||||
private byte types;
|
||||
private boolean hasEmptyArray;
|
||||
|
||||
public MutableTypeSet()
|
||||
{
|
||||
|
@ -135,6 +143,12 @@ public class FieldTypeInfo
|
|||
this.types = types;
|
||||
}
|
||||
|
||||
public MutableTypeSet(byte types, boolean hasEmptyArray)
|
||||
{
|
||||
this.types = types;
|
||||
this.hasEmptyArray = hasEmptyArray;
|
||||
}
|
||||
|
||||
public MutableTypeSet add(ColumnType type)
|
||||
{
|
||||
switch (type.getType()) {
|
||||
|
@ -169,9 +183,24 @@ public class FieldTypeInfo
|
|||
return this;
|
||||
}
|
||||
|
||||
public MutableTypeSet merge(byte other)
|
||||
/**
|
||||
* Set a flag when we encounter an empty array or array with only null elements
|
||||
*/
|
||||
public MutableTypeSet addUntypedArray()
|
||||
{
|
||||
hasEmptyArray = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean hasUntypedArray()
|
||||
{
|
||||
return hasEmptyArray;
|
||||
}
|
||||
|
||||
public MutableTypeSet merge(byte other, boolean hasEmptyArray)
|
||||
{
|
||||
types |= other;
|
||||
this.hasEmptyArray = this.hasEmptyArray || hasEmptyArray;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -208,13 +237,13 @@ public class FieldTypeInfo
|
|||
return false;
|
||||
}
|
||||
MutableTypeSet typeSet = (MutableTypeSet) o;
|
||||
return types == typeSet.types;
|
||||
return types == typeSet.types && hasEmptyArray == typeSet.hasEmptyArray;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(types);
|
||||
return Objects.hash(types, hasEmptyArray);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -257,33 +286,11 @@ public class FieldTypeInfo
|
|||
@Nullable
|
||||
private static ColumnType getSingleType(byte types)
|
||||
{
|
||||
int count = 0;
|
||||
ColumnType singleType = null;
|
||||
if ((types & STRING_MASK) > 0) {
|
||||
singleType = ColumnType.STRING;
|
||||
count++;
|
||||
if (Integer.bitCount(types) == 1) {
|
||||
return TYPES[Integer.numberOfTrailingZeros(types)];
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
if ((types & LONG_MASK) > 0) {
|
||||
singleType = ColumnType.LONG;
|
||||
count++;
|
||||
}
|
||||
if ((types & DOUBLE_MASK) > 0) {
|
||||
singleType = ColumnType.DOUBLE;
|
||||
count++;
|
||||
}
|
||||
if ((types & STRING_ARRAY_MASK) > 0) {
|
||||
singleType = ColumnType.STRING_ARRAY;
|
||||
count++;
|
||||
}
|
||||
if ((types & LONG_ARRAY_MASK) > 0) {
|
||||
singleType = ColumnType.LONG_ARRAY;
|
||||
count++;
|
||||
}
|
||||
if ((types & DOUBLE_ARRAY_MASK) > 0) {
|
||||
singleType = ColumnType.DOUBLE_ARRAY;
|
||||
count++;
|
||||
}
|
||||
return count == 1 ? singleType : null;
|
||||
}
|
||||
|
||||
public static Set<ColumnType> convertToSet(byte types)
|
||||
|
|
|
@ -75,9 +75,11 @@ public interface NestedCommonFormatColumn extends BaseColumn
|
|||
|
||||
default SortedMap<String, FieldTypeInfo.MutableTypeSet> getFieldTypeInfo()
|
||||
{
|
||||
FieldTypeInfo.MutableTypeSet rootOnlyType = new FieldTypeInfo.MutableTypeSet().add(getLogicalType());
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields = new TreeMap<>();
|
||||
fields.put(NestedPathFinder.JSON_PATH_ROOT, rootOnlyType);
|
||||
if (!ColumnType.NESTED_DATA.equals(getLogicalType())) {
|
||||
FieldTypeInfo.MutableTypeSet rootOnlyType = new FieldTypeInfo.MutableTypeSet().add(getLogicalType());
|
||||
fields.put(NestedPathFinder.JSON_PATH_ROOT, rootOnlyType);
|
||||
}
|
||||
return fields;
|
||||
}
|
||||
|
||||
|
|
|
@ -47,6 +47,7 @@ import java.nio.ByteOrder;
|
|||
public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColumn>
|
||||
{
|
||||
public static NestedDataColumnSupplier read(
|
||||
ColumnType logicalType,
|
||||
boolean hasNulls,
|
||||
ByteBuffer bb,
|
||||
ColumnBuilder columnBuilder,
|
||||
|
@ -72,20 +73,9 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
|||
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||
|
||||
ColumnType simpleType;
|
||||
fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper);
|
||||
fieldInfo = FieldTypeInfo.read(bb, fields.size());
|
||||
|
||||
if (fields.size() == 0) {
|
||||
// all nulls, in the future we'll deal with this better... but for now lets just call it a string because
|
||||
// it is the most permissive (besides json)
|
||||
simpleType = ColumnType.STRING;
|
||||
} else if (fields.size() == 1 && NestedPathFinder.JSON_PATH_ROOT.equals(fields.get(0))) {
|
||||
simpleType = fieldInfo.getTypes(0).getSingleType();
|
||||
} else {
|
||||
simpleType = null;
|
||||
}
|
||||
|
||||
final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
||||
mapper,
|
||||
columnName,
|
||||
|
@ -192,7 +182,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
|||
mapper,
|
||||
bitmapSerdeFactory,
|
||||
byteOrder,
|
||||
simpleType
|
||||
logicalType
|
||||
);
|
||||
}
|
||||
catch (IOException ex) {
|
||||
|
|
|
@ -89,7 +89,7 @@ public abstract class StructuredDataProcessor
|
|||
private ProcessResults processMapField(Queue<Field> toProcess, MapField map)
|
||||
{
|
||||
// just guessing a size for a Map as some constant, it might be bigger than this...
|
||||
final ProcessResults processResults = new ProcessResults().withSize(16);
|
||||
final ProcessResults processResults = new ProcessResults().withSize(16).setHasObjects();
|
||||
for (Map.Entry<String, ?> entry : map.getMap().entrySet()) {
|
||||
// add estimated size of string key
|
||||
processResults.addSize(estimateStringSize(entry.getKey()));
|
||||
|
@ -242,22 +242,38 @@ public abstract class StructuredDataProcessor
|
|||
private Set<ArrayList<NestedPathPart>> literalFields;
|
||||
private int estimatedSize;
|
||||
|
||||
private boolean hasObjects;
|
||||
|
||||
public ProcessResults()
|
||||
{
|
||||
literalFields = new HashSet<>();
|
||||
estimatedSize = 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set of paths which contain simple values
|
||||
*/
|
||||
public Set<ArrayList<NestedPathPart>> getLiteralFields()
|
||||
{
|
||||
return literalFields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Estimated size in bytes of the data processed
|
||||
*/
|
||||
public int getEstimatedSize()
|
||||
{
|
||||
return estimatedSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if
|
||||
*/
|
||||
public boolean hasObjects()
|
||||
{
|
||||
return hasObjects;
|
||||
}
|
||||
|
||||
public ProcessResults addSize(int size)
|
||||
{
|
||||
this.estimatedSize += size;
|
||||
|
@ -277,10 +293,17 @@ public abstract class StructuredDataProcessor
|
|||
return this;
|
||||
}
|
||||
|
||||
public ProcessResults setHasObjects()
|
||||
{
|
||||
this.hasObjects = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ProcessResults merge(ProcessResults other)
|
||||
{
|
||||
this.literalFields.addAll(other.literalFields);
|
||||
this.estimatedSize += other.estimatedSize;
|
||||
this.hasObjects = this.hasObjects || other.hasObjects;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.druid.segment.DimensionHandlerUtils;
|
|||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.ColumnTypeFactory;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||
|
@ -104,9 +105,27 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
|||
this.arrayDictionary = arrayDictionary;
|
||||
this.encodedValueColumn = encodedValueColumn;
|
||||
this.nullValueBitmap = nullValueBitmap;
|
||||
this.logicalType = logicalType;
|
||||
this.logicalExpressionType = ExpressionType.fromColumnTypeStrict(logicalType);
|
||||
this.variantTypes = variantTypeSetByte == null ? null : new FieldTypeInfo.TypeSet(variantTypeSetByte);
|
||||
// use the variant type bytes if set, in current code the logical type should have been computed via this same means
|
||||
// however older versions of the code had a bug which could incorrectly classify mixed types as nested data
|
||||
if (variantTypeSetByte != null) {
|
||||
ColumnType theType = null;
|
||||
for (ColumnType type : FieldTypeInfo.convertToSet(variantTypeSetByte)) {
|
||||
theType = ColumnType.leastRestrictiveType(theType, type);
|
||||
}
|
||||
if (theType != null) {
|
||||
// sign bit is used to indicate empty arrays, this
|
||||
if (variantTypeSetByte < 0 && !theType.isArray()) {
|
||||
theType = ColumnTypeFactory.getInstance().ofArray(theType);
|
||||
}
|
||||
this.logicalType = theType;
|
||||
} else {
|
||||
this.logicalType = logicalType;
|
||||
}
|
||||
} else {
|
||||
this.logicalType = logicalType;
|
||||
}
|
||||
this.adjustLongId = stringDictionary.size();
|
||||
this.adjustDoubleId = adjustLongId + longDictionary.size();
|
||||
this.adjustArrayId = adjustDoubleId + doubleDictionary.size();
|
||||
|
|
|
@ -79,7 +79,14 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
@JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory
|
||||
)
|
||||
{
|
||||
return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, isVariantType, byteOrder, bitmapSerdeFactory, null);
|
||||
return new NestedCommonFormatColumnPartSerde(
|
||||
logicalType,
|
||||
hasNulls,
|
||||
isVariantType,
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
private final ColumnType logicalType;
|
||||
|
@ -202,6 +209,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
}
|
||||
return (buffer, builder, columnConfig) -> {
|
||||
NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
|
||||
logicalType,
|
||||
hasNulls,
|
||||
buffer,
|
||||
builder,
|
||||
|
@ -300,7 +308,14 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
|
||||
public NestedCommonFormatColumnPartSerde build()
|
||||
{
|
||||
return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, isVariantType, byteOrder, bitmapSerdeFactory, serializer);
|
||||
return new NestedCommonFormatColumnPartSerde(
|
||||
logicalType,
|
||||
hasNulls,
|
||||
isVariantType,
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
serializer
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -77,6 +77,7 @@ public class NestedDataTestUtils
|
|||
public static final String ARRAY_TYPES_DATA_FILE = "nested-array-test-data.json";
|
||||
|
||||
public static final String ARRAY_TYPES_DATA_FILE_2 = "nested-array-test-data-2.json";
|
||||
public static final String ALL_TYPES_TEST_DATA_FILE = "nested-all-types-test-data.json";
|
||||
|
||||
public static final String INCREMENTAL_SEGMENTS_NAME = "incremental";
|
||||
public static final String DEFAULT_SEGMENTS_NAME = "segments";
|
||||
|
|
|
@ -731,6 +731,88 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryTypeGauntlet() throws Exception
|
||||
{
|
||||
Druids.ScanQueryBuilder builder = Druids.newScanQueryBuilder()
|
||||
.dataSource("test_datasource")
|
||||
.intervals(
|
||||
new MultipleIntervalSegmentSpec(
|
||||
Collections.singletonList(Intervals.ETERNITY)
|
||||
)
|
||||
)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.limit(100)
|
||||
.context(ImmutableMap.of());
|
||||
Query<ScanResultValue> scanQuery = builder.build();
|
||||
final AggregatorFactory[] aggs = new AggregatorFactory[]{new CountAggregatorFactory("count")};
|
||||
List<Segment> realtimeSegs = ImmutableList.of(
|
||||
NestedDataTestUtils.createIncrementalIndex(
|
||||
tempFolder,
|
||||
NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE,
|
||||
NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
|
||||
NestedDataTestUtils.TIMESTAMP_SPEC,
|
||||
NestedDataTestUtils.AUTO_DISCOVERY,
|
||||
TransformSpec.NONE,
|
||||
aggs,
|
||||
Granularities.NONE,
|
||||
true
|
||||
)
|
||||
);
|
||||
List<Segment> segs = NestedDataTestUtils.createSegments(
|
||||
tempFolder,
|
||||
closer,
|
||||
NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE,
|
||||
NestedDataTestUtils.DEFAULT_JSON_INPUT_FORMAT,
|
||||
NestedDataTestUtils.TIMESTAMP_SPEC,
|
||||
NestedDataTestUtils.AUTO_DISCOVERY,
|
||||
TransformSpec.NONE,
|
||||
aggs,
|
||||
Granularities.NONE,
|
||||
true,
|
||||
IndexSpec.DEFAULT
|
||||
);
|
||||
|
||||
|
||||
final Sequence<ScanResultValue> seq = helper.runQueryOnSegmentsObjs(realtimeSegs, scanQuery);
|
||||
final Sequence<ScanResultValue> seq2 = helper.runQueryOnSegmentsObjs(segs, scanQuery);
|
||||
|
||||
List<ScanResultValue> resultsRealtime = seq.toList();
|
||||
List<ScanResultValue> resultsSegments = seq2.toList();
|
||||
logResults(resultsSegments);
|
||||
logResults(resultsRealtime);
|
||||
Assert.assertEquals(1, resultsRealtime.size());
|
||||
Assert.assertEquals(resultsRealtime.size(), resultsSegments.size());
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(
|
||||
"["
|
||||
+ "[1672531200000, null, 0, 0.0, true, 51, 1, [], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, null, 2, 0.0, false, b, b, 2, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, a, 1, 1.0, true, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, b, 4, 3.3, true, 4, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, c, 0, 4.4, true, hello, {}, [], {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, d, 5, 5.9, false, null, a, 6, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]"
|
||||
+ "]",
|
||||
resultsSegments.get(0).getEvents().toString()
|
||||
);
|
||||
} else {
|
||||
Assert.assertEquals(
|
||||
"["
|
||||
+ "[1672531200000, null, null, null, true, 51, 1, [], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, , 2, null, false, b, b, 2, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, a, 1, 1.0, true, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, b, 4, 3.3, true, 4, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, c, null, 4.4, true, hello, {}, [], {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, d, 5, 5.9, false, null, a, 6, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], "
|
||||
+ "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]"
|
||||
+ "]",
|
||||
resultsSegments.get(0).getEvents().toString()
|
||||
);
|
||||
}
|
||||
Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
|
||||
}
|
||||
|
||||
private static void logResults(List<ScanResultValue> results)
|
||||
{
|
||||
StringBuilder bob = new StringBuilder();
|
||||
|
|
|
@ -590,6 +590,145 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
|
|||
Assert.assertNull(valueSelector.getObject());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantNull()
|
||||
{
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true);
|
||||
Assert.assertEquals(0, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true);
|
||||
|
||||
Assert.assertEquals(0, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(null, true);
|
||||
Assert.assertEquals(0, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
|
||||
|
||||
Assert.assertTrue(indexer.hasNulls);
|
||||
Assert.assertFalse(indexer.hasNestedData);
|
||||
Assert.assertTrue(indexer.isConstant());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantString()
|
||||
{
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true);
|
||||
Assert.assertEquals(166, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true);
|
||||
|
||||
Assert.assertEquals(52, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent("abcd", true);
|
||||
Assert.assertEquals(52, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
|
||||
Assert.assertFalse(indexer.hasNulls);
|
||||
Assert.assertFalse(indexer.hasNestedData);
|
||||
Assert.assertTrue(indexer.isConstant());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantLong()
|
||||
{
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true);
|
||||
Assert.assertEquals(94, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true);
|
||||
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(1234L, true);
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
|
||||
Assert.assertFalse(indexer.hasNulls);
|
||||
Assert.assertFalse(indexer.hasNestedData);
|
||||
Assert.assertTrue(indexer.isConstant());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantEmptyArray()
|
||||
{
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true);
|
||||
Assert.assertEquals(54, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true);
|
||||
|
||||
Assert.assertEquals(8, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(), true);
|
||||
Assert.assertEquals(8, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
|
||||
Assert.assertFalse(indexer.hasNulls);
|
||||
Assert.assertFalse(indexer.hasNestedData);
|
||||
Assert.assertTrue(indexer.isConstant());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantArray()
|
||||
{
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true);
|
||||
Assert.assertEquals(246, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 4, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true);
|
||||
|
||||
Assert.assertEquals(104, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 4, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 3L), true);
|
||||
Assert.assertEquals(104, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 4, indexer.getCardinality());
|
||||
|
||||
Assert.assertFalse(indexer.hasNulls);
|
||||
Assert.assertFalse(indexer.hasNestedData);
|
||||
Assert.assertTrue(indexer.isConstant());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantEmptyObject()
|
||||
{
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true);
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true);
|
||||
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of(), true);
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
|
||||
Assert.assertFalse(indexer.hasNulls);
|
||||
Assert.assertTrue(indexer.hasNestedData);
|
||||
Assert.assertTrue(indexer.isConstant());
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
private static IncrementalIndex makeIncrementalIndex(long minTimestamp)
|
||||
{
|
||||
|
|
|
@ -322,6 +322,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
.put(
|
||||
"mmappedAutoTypes",
|
||||
input -> {
|
||||
input.indexSpec(IndexSpec.builder().build());
|
||||
input.mapSchema(
|
||||
schema ->
|
||||
new IncrementalIndexSchema(
|
||||
|
|
|
@ -207,7 +207,13 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
SortedMap<String, FieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
|
||||
|
||||
IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo())
|
||||
new IndexableAdapter.NestedColumnMergable(
|
||||
indexer.getSortedValueLookups(),
|
||||
indexer.getFieldTypeInfo(),
|
||||
false,
|
||||
false,
|
||||
null
|
||||
)
|
||||
);
|
||||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
@ -300,6 +306,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
ColumnBuilder bob = new ColumnBuilder();
|
||||
bob.setFileMapper(fileMapper);
|
||||
NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
|
||||
ColumnType.NESTED_DATA,
|
||||
false,
|
||||
baseBuffer,
|
||||
bob,
|
||||
|
|
|
@ -179,7 +179,13 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest
|
|||
SortedMap<String, FieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
|
||||
|
||||
IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo())
|
||||
new IndexableAdapter.NestedColumnMergable(
|
||||
indexer.getSortedValueLookups(),
|
||||
indexer.getFieldTypeInfo(),
|
||||
true,
|
||||
false,
|
||||
null
|
||||
)
|
||||
);
|
||||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
|
|
@ -117,7 +117,7 @@ public class NestedFieldTypeInfoTest
|
|||
FieldTypeInfo.MutableTypeSet merge = new FieldTypeInfo.MutableTypeSet();
|
||||
for (ColumnType columnType : columnTypes) {
|
||||
typeSet.add(columnType);
|
||||
merge.merge(new FieldTypeInfo.MutableTypeSet().add(columnType).getByteValue());
|
||||
merge.merge(new FieldTypeInfo.MutableTypeSet().add(columnType).getByteValue(), false);
|
||||
}
|
||||
|
||||
Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue());
|
||||
|
|
|
@ -137,7 +137,13 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest
|
|||
SortedMap<String, FieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
|
||||
|
||||
IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo())
|
||||
new IndexableAdapter.NestedColumnMergable(
|
||||
indexer.getSortedValueLookups(),
|
||||
indexer.getFieldTypeInfo(),
|
||||
false,
|
||||
false,
|
||||
null
|
||||
)
|
||||
);
|
||||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
|
|
@ -137,7 +137,13 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest
|
|||
SortedMap<String, FieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
|
||||
|
||||
IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo())
|
||||
new IndexableAdapter.NestedColumnMergable(
|
||||
indexer.getSortedValueLookups(),
|
||||
indexer.getFieldTypeInfo(),
|
||||
false,
|
||||
false,
|
||||
null
|
||||
)
|
||||
);
|
||||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
|
|
@ -137,7 +137,13 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest
|
|||
SortedMap<String, FieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
|
||||
|
||||
IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo())
|
||||
new IndexableAdapter.NestedColumnMergable(
|
||||
indexer.getSortedValueLookups(),
|
||||
indexer.getFieldTypeInfo(),
|
||||
false,
|
||||
false,
|
||||
null
|
||||
)
|
||||
);
|
||||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
|
@ -238,12 +237,18 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest
|
|||
SortedMap<String, FieldTypeInfo.MutableTypeSet> sortedFields = new TreeMap<>();
|
||||
|
||||
IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
new IndexableAdapter.NestedColumnMergable(indexer.getSortedValueLookups(), indexer.getFieldTypeInfo())
|
||||
new IndexableAdapter.NestedColumnMergable(
|
||||
indexer.getSortedValueLookups(),
|
||||
indexer.getFieldTypeInfo(),
|
||||
false,
|
||||
false,
|
||||
null
|
||||
)
|
||||
);
|
||||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
||||
expectedTypes = sortedFields.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
expectedTypes = new FieldTypeInfo.MutableTypeSet((byte) (sortedFields.get(NestedPathFinder.JSON_PATH_ROOT).getByteValue() & 0x7F));
|
||||
for (ColumnType type : FieldTypeInfo.convertToSet(expectedTypes.getByteValue())) {
|
||||
expectedLogicalType = ColumnType.leastRestrictiveType(expectedLogicalType, type);
|
||||
}
|
||||
|
@ -374,9 +379,10 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Assert.assertNotNull(nullValueIndex);
|
||||
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields = column.getFieldTypeInfo();
|
||||
Assert.assertEquals(1, fields.size());
|
||||
Assert.assertEquals(
|
||||
ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, expectedType),
|
||||
fields
|
||||
expectedType,
|
||||
fields.get(NestedPathFinder.JSON_PATH_ROOT)
|
||||
);
|
||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(expectedLogicalType);
|
||||
|
||||
|
|
|
@ -0,0 +1,7 @@
|
|||
{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||
{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantEmptyObj":"b", "variantEmtpyArray":2, "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||
{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "obj":{"a": 300}, "complexObj":{"x": 4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||
{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "4", "variantEmptyObj":{}, "variantEmtpyArray":4, "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||
{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantEmptyObj":{}, "variantEmtpyArray":[], "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||
{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
||||
{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantEmptyObj":1, "variantEmtpyArray":[], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
|
|
@ -87,6 +87,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
private static final String DATA_SOURCE_MIXED = "nested_mix";
|
||||
private static final String DATA_SOURCE_MIXED_2 = "nested_mix_2";
|
||||
private static final String DATA_SOURCE_ARRAYS = "arrays";
|
||||
private static final String DATA_SOURCE_ALL = "all_auto";
|
||||
|
||||
private static final List<ImmutableMap<String, Object>> RAW_ROWS = ImmutableList.of(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
|
@ -294,6 +295,31 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.inputTmpDir(temporaryFolder.newFolder())
|
||||
.buildMMappedIndex();
|
||||
|
||||
final QueryableIndex indexAllTypesAuto =
|
||||
IndexBuilder.create()
|
||||
.tmpDir(temporaryFolder.newFolder())
|
||||
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
|
||||
.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withTimestampSpec(NestedDataTestUtils.AUTO_SCHEMA.getTimestampSpec())
|
||||
.withDimensionsSpec(NestedDataTestUtils.AUTO_SCHEMA.getDimensionsSpec())
|
||||
.withMetrics(
|
||||
new CountAggregatorFactory("cnt")
|
||||
)
|
||||
.withRollup(false)
|
||||
.build()
|
||||
)
|
||||
.inputSource(
|
||||
ResourceInputSource.of(
|
||||
NestedDataTestUtils.class.getClassLoader(),
|
||||
NestedDataTestUtils.ALL_TYPES_TEST_DATA_FILE
|
||||
)
|
||||
)
|
||||
.inputFormat(TestDataBuilder.DEFAULT_JSON_INPUT_FORMAT)
|
||||
.inputTmpDir(temporaryFolder.newFolder())
|
||||
.buildMMappedIndex();
|
||||
|
||||
|
||||
SpecificSegmentsQuerySegmentWalker walker = new SpecificSegmentsQuerySegmentWalker(conglomerate);
|
||||
walker.add(
|
||||
DataSegment.builder()
|
||||
|
@ -349,6 +375,15 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.size(0)
|
||||
.build(),
|
||||
indexArrays
|
||||
).add(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE_ALL)
|
||||
.version("1")
|
||||
.interval(indexAllTypesAuto.getDataInterval())
|
||||
.shardSpec(new LinearShardSpec(1))
|
||||
.size(0)
|
||||
.build(),
|
||||
indexAllTypesAuto
|
||||
);
|
||||
|
||||
return walker;
|
||||
|
@ -5140,4 +5175,112 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
)
|
||||
.run();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanAllTypesAuto()
|
||||
{
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
"SELECT * FROM druid.all_auto",
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(DATA_SOURCE_ALL)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns(
|
||||
"__time",
|
||||
"arrayBool",
|
||||
"arrayDouble",
|
||||
"arrayDoubleNulls",
|
||||
"arrayLong",
|
||||
"arrayLongNulls",
|
||||
"arrayNestedLong",
|
||||
"arrayObject",
|
||||
"arrayString",
|
||||
"arrayStringNulls",
|
||||
"arrayVariant",
|
||||
"bool",
|
||||
"cDoubleArray",
|
||||
"cEmptyArray",
|
||||
"cEmptyObj",
|
||||
"cEmptyObjectArray",
|
||||
"cLongArray",
|
||||
"cNullArray",
|
||||
"cObj",
|
||||
"cObjectArray",
|
||||
"cdouble",
|
||||
"clong",
|
||||
"cnt",
|
||||
"complexObj",
|
||||
"cstr",
|
||||
"cstringArray",
|
||||
"double",
|
||||
"long",
|
||||
"null",
|
||||
"obj",
|
||||
"str",
|
||||
"variant",
|
||||
"variantEmptyObj",
|
||||
"variantEmtpyArray"
|
||||
)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
ImmutableList.of(
|
||||
new Object[]{1672531200000L, "", 0L, 0.0D, "true", "51", "1", "[]", "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", null, "[\"a\",\"b\"]", null, "[2,3]", null, "[null]", null, "[\"true\",\"false\",\"true\"]", null, "[{\"x\":1},{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "", 2L, 0.0D, "false", "b", "\"b\"", "2", "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2}],\"z\":{\"a\":[5.5],\"b\":false}}", "[\"a\",\"b\",\"c\"]", "[null,\"b\"]", "[2,3]", null, "[3.3,4.4,5.5]", "[999.0,null,5.5]", "[null,null,2.2]", "[\"true\",\"true\"]", "[null,[null],[]]", "[{\"x\":3},{\"x\":4}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "a", 1L, 1.0D, "true", "1", "1", "1", "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"a\",\"b\"]", "[\"a\",\"b\"]", "[1,2,3]", "[1,null,3]", "[1.1,2.2,3.3]", "[1.1,2.2,null]", "[\"a\",\"1\",\"2.2\"]", "[\"true\",\"false\",\"true\"]", "[[1,2,null],[3,4]]", "[{\"x\":1},{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "b", 4L, 3.3D, "true", "4", "{}", "4", "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"d\",\"e\"]", "[\"b\",\"b\"]", "[1,4]", "[1]", "[2.2,3.3,4.0]", null, "[\"a\",\"b\",\"c\"]", "[null,\"false\",\"true\"]", "[[1,2],[3,4],[5,6,7]]", "[{\"x\":null},{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "c", 0L, 4.4D, "true", "hello", "{}", "[]", "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, null, "[1,2,3]", "[]", "[1.1,2.2,3.3]", null, null, "[\"false\"]", null, "[{\"x\":1000},{\"y\":2000}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "d", 5L, 5.9D, "false", "", "\"a\"", "6", "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", null, "[\"a\",\"b\"]", null, null, "[null,2,9]", null, "[999.0,5.5,null]", "[\"a\",\"1\",\"2.2\"]", "[]", "[[1],[1,2,null]]", "[{\"a\":1},{\"b\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "null", 3L, 2.0D, "", "3.0", "3.3", "3", "{\"a\":300}", "{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}", "[\"b\",\"c\"]", "[\"d\",null,\"b\"]", "[1,2,3,4]", "[1,2,3]", "[1.1,3.3]", "[null,2.2,null]", "[1,null,1]", "[\"true\",null,\"true\"]", "[[1],null,[1,2,3]]", "[null,{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}
|
||||
) :
|
||||
ImmutableList.of(
|
||||
new Object[]{1672531200000L, null, null, null, "true", "51", "1", "[]", "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", null, "[\"a\",\"b\"]", null, "[2,3]", null, "[null]", null, "[\"true\",\"false\",\"true\"]", null, "[{\"x\":1},{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "", 2L, null, "false", "b", "\"b\"", "2", "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2}],\"z\":{\"a\":[5.5],\"b\":false}}", "[\"a\",\"b\",\"c\"]", "[null,\"b\"]", "[2,3]", null, "[3.3,4.4,5.5]", "[999.0,null,5.5]", "[null,null,2.2]", "[\"true\",\"true\"]", "[null,[null],[]]", "[{\"x\":3},{\"x\":4}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "a", 1L, 1.0D, "true", "1", "1", "1", "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"a\",\"b\"]", "[\"a\",\"b\"]", "[1,2,3]", "[1,null,3]", "[1.1,2.2,3.3]", "[1.1,2.2,null]", "[\"a\",\"1\",\"2.2\"]", "[\"true\",\"false\",\"true\"]", "[[1,2,null],[3,4]]", "[{\"x\":1},{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "b", 4L, 3.3D, "true", "4", "{}", "4", "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"d\",\"e\"]", "[\"b\",\"b\"]", "[1,4]", "[1]", "[2.2,3.3,4.0]", null, "[\"a\",\"b\",\"c\"]", "[null,\"false\",\"true\"]", "[[1,2],[3,4],[5,6,7]]", "[{\"x\":null},{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "c", null, 4.4D, "true", "hello", "{}", "[]", "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, null, "[1,2,3]", "[]", "[1.1,2.2,3.3]", null, null, "[\"false\"]", null, "[{\"x\":1000},{\"y\":2000}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "d", 5L, 5.9D, "false", null, "\"a\"", "6", "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", null, "[\"a\",\"b\"]", null, null, "[null,2,9]", null, "[999.0,5.5,null]", "[\"a\",\"1\",\"2.2\"]", "[]", "[[1],[1,2,null]]", "[{\"a\":1},{\"b\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L},
|
||||
new Object[]{1672531200000L, "null", 3L, 2.0D, null, "3.0", "3.3", "3", "{\"a\":300}", "{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}", "[\"b\",\"c\"]", "[\"d\",null,\"b\"]", "[1,2,3,4]", "[1,2,3]", "[1.1,3.3]", "[null,2.2,null]", "[1,null,1]", "[\"true\",null,\"true\"]", "[[1],null,[1,2,3]]", "[null,{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("__time", ColumnType.LONG)
|
||||
.add("str", ColumnType.STRING)
|
||||
.add("long", ColumnType.LONG)
|
||||
.add("double", ColumnType.DOUBLE)
|
||||
.add("bool", ColumnType.STRING)
|
||||
.add("variant", ColumnType.STRING)
|
||||
.add("variantEmptyObj", ColumnType.NESTED_DATA)
|
||||
.add("variantEmtpyArray", ColumnType.LONG_ARRAY)
|
||||
.add("obj", ColumnType.NESTED_DATA)
|
||||
.add("complexObj", ColumnType.NESTED_DATA)
|
||||
.add("arrayString", ColumnType.STRING_ARRAY)
|
||||
.add("arrayStringNulls", ColumnType.STRING_ARRAY)
|
||||
.add("arrayLong", ColumnType.LONG_ARRAY)
|
||||
.add("arrayLongNulls", ColumnType.LONG_ARRAY)
|
||||
.add("arrayDouble", ColumnType.DOUBLE_ARRAY)
|
||||
.add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
|
||||
.add("arrayVariant", ColumnType.STRING_ARRAY)
|
||||
.add("arrayBool", ColumnType.STRING_ARRAY)
|
||||
.add("arrayNestedLong", ColumnType.NESTED_DATA)
|
||||
.add("arrayObject", ColumnType.NESTED_DATA)
|
||||
.add("null", ColumnType.STRING)
|
||||
.add("cstr", ColumnType.STRING)
|
||||
.add("clong", ColumnType.LONG)
|
||||
.add("cdouble", ColumnType.DOUBLE)
|
||||
.add("cObj", ColumnType.NESTED_DATA)
|
||||
.add("cstringArray", ColumnType.STRING_ARRAY)
|
||||
.add("cLongArray", ColumnType.LONG_ARRAY)
|
||||
.add("cDoubleArray", ColumnType.DOUBLE_ARRAY)
|
||||
.add("cEmptyArray", ColumnType.NESTED_DATA)
|
||||
.add("cEmptyObj", ColumnType.NESTED_DATA)
|
||||
.add("cNullArray", ColumnType.NESTED_DATA)
|
||||
.add("cEmptyObjectArray", ColumnType.NESTED_DATA)
|
||||
.add("cObjectArray", ColumnType.NESTED_DATA)
|
||||
.add("cnt", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue