Field writers for numerical arrays (#14900)

Row-based frames, and by extension, MSQ now supports numeric array types. This means that all queries consuming or producing arrays would also work with MSQ. Numeric arrays can also be ingested via MSQ. Post this patch, queries like, SELECT [1, 2] would work with MSQ since they consume a numeric array, instead of failing with an unsupported column type exception.
This commit is contained in:
Laksh Singla 2023-10-04 23:16:47 +05:30 committed by GitHub
parent 88476e0e83
commit 30cf76db99
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
49 changed files with 2939 additions and 533 deletions

View File

@ -75,6 +75,10 @@ public class DimensionSchemaUtils
switch (type.getElementType().getType()) {
case STRING:
return new StringDimensionSchema(column, DimensionSchema.MultiValueHandling.ARRAY, null);
case LONG:
case FLOAT:
case DOUBLE:
return new AutoTypeColumnSchema(column);
default:
throw new ISE("Cannot create dimension for type [%s]", type.toString());
}

View File

@ -31,7 +31,6 @@ import org.apache.druid.msq.indexing.error.TooManyClusteredByColumnsFault;
import org.apache.druid.msq.indexing.error.TooManyColumnsFault;
import org.apache.druid.msq.indexing.error.TooManyInputFilesFault;
import org.apache.druid.msq.indexing.error.TooManyPartitionsFault;
import org.apache.druid.msq.indexing.error.UnknownFault;
import org.apache.druid.msq.test.MSQTestBase;
import org.apache.druid.msq.test.MSQTestFileUtils;
import org.apache.druid.segment.column.ColumnType;
@ -220,31 +219,6 @@ public class MSQFaultsTest extends MSQTestBase
}
@Test
public void testInsertWithUnsupportedColumnType()
{
RowSignature dummyRowSignature = RowSignature.builder().add("__time", ColumnType.LONG).build();
testIngestQuery()
.setSql(StringUtils.format(
" insert into foo1 SELECT\n"
+ " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n"
+ " col1\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{ \"files\": [\"ignored\"],\"type\":\"local\"}',\n"
+ " '{\"type\": \"json\"}',\n"
+ " '[{\"name\": \"timestamp\", \"type\": \"string\"},{\"name\": \"col1\", \"type\": \"long_array\"} ]'\n"
+ " )\n"
+ ") PARTITIONED by day"
))
.setExpectedDataSource("foo1")
.setExpectedRowSignature(dummyRowSignature)
.setExpectedMSQFault(UnknownFault.forMessage(
"org.apache.druid.java.util.common.ISE: Cannot create dimension for type [ARRAY<LONG>]"))
.verifyResults();
}
@Test
public void testInsertWithManyColumns()
{

View File

@ -63,6 +63,7 @@ import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -1463,6 +1464,193 @@ public class MSQInsertTest extends MSQTestBase
.verifyResults();
}
@Test
public void testInsertArrays() throws IOException
{
List<Object[]> expectedRows = Arrays.asList(
new Object[]{
1672531200000L,
null,
null,
new Object[]{1L, 2L, 3L},
new Object[]{},
new Object[]{1.1d, 2.2d, 3.3d},
null
},
new Object[]{
1672531200000L,
null,
Arrays.asList("a", "b"),
null,
new Object[]{2L, 3L},
null,
new Object[]{null}
},
new Object[]{
1672531200000L,
Arrays.asList("a", "b"),
null,
null,
new Object[]{null, 2L, 9L},
null,
new Object[]{999.0d, 5.5d, null}
},
new Object[]{
1672531200000L,
Arrays.asList("a", "b"),
Arrays.asList("a", "b"),
new Object[]{1L, 2L, 3L},
new Object[]{1L, null, 3L},
new Object[]{1.1d, 2.2d, 3.3d},
new Object[]{1.1d, 2.2d, null}
},
new Object[]{
1672531200000L,
Arrays.asList("a", "b", "c"),
Arrays.asList(null, "b"),
new Object[]{2L, 3L},
null,
new Object[]{3.3d, 4.4d, 5.5d},
new Object[]{999.0d, null, 5.5d}
},
new Object[]{
1672531200000L,
Arrays.asList("b", "c"),
Arrays.asList("d", null, "b"),
new Object[]{1L, 2L, 3L, 4L},
new Object[]{1L, 2L, 3L},
new Object[]{1.1d, 3.3d},
new Object[]{null, 2.2d, null}
},
new Object[]{
1672531200000L,
Arrays.asList("d", "e"),
Arrays.asList("b", "b"),
new Object[]{1L, 4L},
new Object[]{1L},
new Object[]{2.2d, 3.3d, 4.0d},
null
},
new Object[]{
1672617600000L,
null,
null,
new Object[]{1L, 2L, 3L},
null,
new Object[]{1.1d, 2.2d, 3.3d},
new Object[]{}
},
new Object[]{
1672617600000L,
null,
Arrays.asList("a", "b"),
null,
new Object[]{2L, 3L},
null,
new Object[]{null, 1.1d}
},
new Object[]{
1672617600000L,
Arrays.asList("a", "b"),
null,
null,
new Object[]{null, 2L, 9L},
null,
new Object[]{999.0d, 5.5d, null}
},
new Object[]{
1672617600000L,
Arrays.asList("a", "b"),
Collections.emptyList(),
new Object[]{1L, 2L, 3L},
new Object[]{1L, null, 3L},
new Object[]{1.1d, 2.2d, 3.3d},
new Object[]{1.1d, 2.2d, null}
},
new Object[]{
1672617600000L,
Arrays.asList("a", "b", "c"),
Arrays.asList(null, "b"),
new Object[]{2L, 3L},
null,
new Object[]{3.3d, 4.4d, 5.5d},
new Object[]{999.0d, null, 5.5d}
},
new Object[]{
1672617600000L,
Arrays.asList("b", "c"),
Arrays.asList("d", null, "b"),
new Object[]{1L, 2L, 3L, 4L},
new Object[]{1L, 2L, 3L},
new Object[]{1.1d, 3.3d},
new Object[]{null, 2.2d, null}
},
new Object[]{
1672617600000L,
Arrays.asList("d", "e"),
Arrays.asList("b", "b"),
new Object[]{1L, 4L},
new Object[]{null},
new Object[]{2.2d, 3.3d, 4.0},
null
}
);
RowSignature rowSignatureWithoutTimeAndStringColumns =
RowSignature.builder()
.add("arrayLong", ColumnType.LONG_ARRAY)
.add("arrayLongNulls", ColumnType.LONG_ARRAY)
.add("arrayDouble", ColumnType.DOUBLE_ARRAY)
.add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY)
.build();
RowSignature fileSignature = RowSignature.builder()
.add("timestamp", ColumnType.STRING)
.add("arrayString", ColumnType.STRING_ARRAY)
.add("arrayStringNulls", ColumnType.STRING_ARRAY)
.addAll(rowSignatureWithoutTimeAndStringColumns)
.build();
// MSQ writes strings instead of string arrays
RowSignature rowSignature = RowSignature.builder()
.add("__time", ColumnType.LONG)
.add("arrayString", ColumnType.STRING)
.add("arrayStringNulls", ColumnType.STRING)
.addAll(rowSignatureWithoutTimeAndStringColumns)
.build();
final Map<String, Object> adjustedContext = new HashMap<>(context);
final File tmpFile = temporaryFolder.newFile();
final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
final InputStream decompressing = CompressionUtils.decompress(resourceStream, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING);
decompressing.close();
final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile);
testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n"
+ " TIME_PARSE(\"timestamp\") as __time,\n"
+ " arrayString,\n"
+ " arrayStringNulls,\n"
+ " arrayLong,\n"
+ " arrayLongNulls,\n"
+ " arrayDouble,\n"
+ " arrayDoubleNulls\n"
+ "FROM TABLE(\n"
+ " EXTERN(\n"
+ " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n"
+ " '{\"type\": \"json\"}',\n"
+ " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n"
+ " )\n"
+ ") PARTITIONED BY day")
.setQueryContext(adjustedContext)
.setExpectedResultRows(expectedRows)
.setExpectedDataSource("foo1")
.setExpectedRowSignature(rowSignature)
.verifyResults();
}
@Nonnull
private List<Object[]> expectedFooRows()
{

View File

@ -36,7 +36,7 @@ import org.junit.Before;
/**
* Runs {@link CalciteArraysQueryTest} but with MSQ engine
*/
public class CalciteArraysSelectQueryMSQTest extends CalciteArraysQueryTest
public class CalciteArraysQueryMSQTest extends CalciteArraysQueryTest
{
private TestGroupByBuffers groupByBuffers;

View File

@ -0,0 +1,62 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
/**
* Reader for fields written by {@link NumericArrayFieldWriter#getDoubleArrayFieldWriter}
*/
public class DoubleArrayFieldReader extends NumericArrayFieldReader
{
@Override
public ColumnValueSelector<?> makeColumnValueSelector(
Memory memory,
ReadableFieldPointer fieldPointer
)
{
return new NumericArrayFieldSelector<Double>(memory, fieldPointer)
{
final SettableFieldPointer fieldPointer = new SettableFieldPointer();
final ColumnValueSelector<?> columnValueSelector =
DoubleFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer);
@Nullable
@Override
public Double getIndividualValueAtMemory(long position)
{
fieldPointer.setPosition(position);
if (columnValueSelector.isNull()) {
return null;
}
return columnValueSelector.getDouble();
}
@Override
public int getIndividualFieldSize()
{
return Byte.BYTES + Double.BYTES;
}
};
}
}

View File

@ -21,74 +21,60 @@ package org.apache.druid.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.DoubleColumnSelector;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.column.ValueTypes;
import javax.annotation.Nullable;
/**
* Reads values written by {@link DoubleFieldWriter}.
* Reads the values produced by {@link DoubleFieldWriter}
*
* Values are sortable as bytes without decoding.
*
* Format:
*
* - 1 byte: {@link DoubleFieldWriter#NULL_BYTE} or {@link DoubleFieldWriter#NOT_NULL_BYTE}
* - 8 bytes: encoded double, using {@link DoubleFieldWriter#transform}
* @see DoubleFieldWriter
* @see NumericFieldWriter for the details of the byte-format that it expects for reading
*/
public class DoubleFieldReader implements FieldReader
public class DoubleFieldReader extends NumericFieldReader
{
DoubleFieldReader()
public static DoubleFieldReader forPrimitive()
{
return new DoubleFieldReader(false);
}
public static DoubleFieldReader forArray()
{
return new DoubleFieldReader(true);
}
private DoubleFieldReader(final boolean forArray)
{
super(forArray);
}
@Override
public ColumnValueSelector<?> makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer)
public ValueType getValueType()
{
return new Selector(memory, fieldPointer);
return ValueType.DOUBLE;
}
@Override
public DimensionSelector makeDimensionSelector(
Memory memory,
ReadableFieldPointer fieldPointer,
@Nullable ExtractionFn extractionFn
public ColumnValueSelector<?> getColumnValueSelector(
final Memory memory,
final ReadableFieldPointer fieldPointer,
final byte nullIndicatorByte
)
{
return ValueTypes.makeNumericWrappingDimensionSelector(
ValueType.DOUBLE,
makeColumnValueSelector(memory, fieldPointer),
extractionFn
);
return new DoubleFieldReader.DoubleFieldSelector(memory, fieldPointer, nullIndicatorByte);
}
@Override
public boolean isNull(Memory memory, long position)
private static class DoubleFieldSelector extends NumericFieldReader.Selector implements DoubleColumnSelector
{
return memory.getByte(position) == DoubleFieldWriter.NULL_BYTE;
}
@Override
public boolean isComparable()
{
return true;
}
final Memory dataRegion;
final ReadableFieldPointer fieldPointer;
/**
* Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}.
*/
private static class Selector implements DoubleColumnSelector
{
private final Memory dataRegion;
private final ReadableFieldPointer fieldPointer;
private Selector(final Memory dataRegion, final ReadableFieldPointer fieldPointer)
public DoubleFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte)
{
super(dataRegion, fieldPointer, nullIndicatorByte);
this.dataRegion = dataRegion;
this.fieldPointer = fieldPointer;
}
@ -98,19 +84,19 @@ public class DoubleFieldReader implements FieldReader
{
assert NullHandling.replaceWithDefault() || !isNull();
final long bits = dataRegion.getLong(fieldPointer.position() + Byte.BYTES);
return DoubleFieldWriter.detransform(bits);
}
@Override
public boolean isNull()
{
return dataRegion.getByte(fieldPointer.position()) == DoubleFieldWriter.NULL_BYTE;
return TransformUtils.detransformToDouble(bits);
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
// Do nothing.
}
@Override
public boolean isNull()
{
return super._isNull();
}
}
}

View File

@ -25,65 +25,49 @@ import org.apache.druid.segment.BaseDoubleColumnValueSelector;
/**
* Wraps a {@link BaseDoubleColumnValueSelector} and writes field values.
*
* See {@link DoubleFieldReader} for format details.
* @see NumericFieldWriter for the details of the byte-format that it writes as
*/
public class DoubleFieldWriter implements FieldWriter
public class DoubleFieldWriter extends NumericFieldWriter
{
public static final int SIZE = Double.BYTES + Byte.BYTES;
// Different from the values in NullHandling, since we want to be able to sort as bytes, and we want
// nulls to come before non-nulls.
public static final byte NULL_BYTE = 0x00;
public static final byte NOT_NULL_BYTE = 0x01;
private final BaseDoubleColumnValueSelector selector;
public DoubleFieldWriter(final BaseDoubleColumnValueSelector selector)
public static DoubleFieldWriter forPrimitive(final BaseDoubleColumnValueSelector selector)
{
return new DoubleFieldWriter(selector, false);
}
public static DoubleFieldWriter forArray(final BaseDoubleColumnValueSelector selector)
{
return new DoubleFieldWriter(selector, true);
}
private DoubleFieldWriter(final BaseDoubleColumnValueSelector selector, final boolean forArray)
{
super(selector, forArray);
this.selector = selector;
}
@Override
public long writeTo(final WritableMemory memory, final long position, final long maxSize)
public int getNumericSizeBytes()
{
if (maxSize < SIZE) {
return -1;
}
if (selector.isNull()) {
memory.putByte(position, NULL_BYTE);
memory.putLong(position + Byte.BYTES, transform(0));
} else {
memory.putByte(position, NOT_NULL_BYTE);
memory.putLong(position + Byte.BYTES, transform(selector.getDouble()));
}
return SIZE;
return Double.BYTES;
}
@Override
public void close()
public void writeSelectorToMemory(WritableMemory memory, long position)
{
// Nothing to close.
writeToMemory(memory, position, selector.getDouble());
}
/**
* Transforms a double into a form where it can be compared as unsigned bytes without decoding.
*/
public static long transform(final double n)
@Override
public void writeNullToMemory(WritableMemory memory, long position)
{
final long bits = Double.doubleToLongBits(n);
final long mask = ((bits & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE;
return Long.reverseBytes(bits ^ mask);
writeToMemory(memory, position, 0);
}
/**
* Inverse of {@link #transform}.
*/
public static double detransform(final long bits)
private void writeToMemory(WritableMemory memory, long position, double value)
{
final long reversedBits = Long.reverseBytes(bits);
final long mask = (((reversedBits ^ Long.MIN_VALUE) & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE;
return Double.longBitsToDouble(reversedBits ^ mask);
memory.putLong(position, TransformUtils.transformFromDouble(value));
}
}

View File

@ -24,7 +24,6 @@ import org.apache.druid.frame.key.RowKey;
import org.apache.druid.frame.key.RowKeyReader;
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ValueType;
/**
* Helper used to read field values from row-based frames or {@link RowKey}.
@ -46,23 +45,37 @@ public class FieldReaders
{
switch (Preconditions.checkNotNull(columnType, "columnType").getType()) {
case LONG:
return new LongFieldReader();
return LongFieldReader.forPrimitive();
case FLOAT:
return new FloatFieldReader();
return FloatFieldReader.forPrimitive();
case DOUBLE:
return new DoubleFieldReader();
return DoubleFieldReader.forPrimitive();
case STRING:
return new StringFieldReader(false);
return new StringFieldReader();
case COMPLEX:
return ComplexFieldReader.createFromType(columnType);
case ARRAY:
if (columnType.getElementType().getType() == ValueType.STRING) {
return new StringFieldReader(true);
switch (Preconditions.checkNotNull(columnType.getElementType().getType(), "array elementType")) {
case STRING:
return new StringArrayFieldReader();
case LONG:
return new LongArrayFieldReader();
case FLOAT:
return new FloatArrayFieldReader();
case DOUBLE:
return new DoubleArrayFieldReader();
default:
throw new UnsupportedColumnTypeException(columnName, columnType);
}
// Fall through to error for other array types

View File

@ -66,21 +66,30 @@ public class FieldWriters
switch (columnType.getType()) {
case LONG:
return makeLongWriter(columnSelectorFactory, columnName);
case FLOAT:
return makeFloatWriter(columnSelectorFactory, columnName);
case DOUBLE:
return makeDoubleWriter(columnSelectorFactory, columnName);
case STRING:
return makeStringWriter(columnSelectorFactory, columnName);
case COMPLEX:
return makeComplexWriter(columnSelectorFactory, columnName, columnType.getComplexTypeName());
case ARRAY:
switch (columnType.getElementType().getType()) {
case STRING:
return makeStringArrayWriter(columnSelectorFactory, columnName);
default:
throw new UnsupportedColumnTypeException(columnName, columnType);
case LONG:
return makeLongArrayWriter(columnSelectorFactory, columnName);
case FLOAT:
return makeFloatArrayWriter(columnSelectorFactory, columnName);
case DOUBLE:
return makeDoubleArrayWriter(columnSelectorFactory, columnName);
}
case COMPLEX:
return makeComplexWriter(columnSelectorFactory, columnName, columnType.getComplexTypeName());
default:
throw new UnsupportedColumnTypeException(columnName, columnType);
}
@ -92,7 +101,7 @@ public class FieldWriters
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return new LongFieldWriter(selector);
return LongFieldWriter.forPrimitive(selector);
}
private static FieldWriter makeFloatWriter(
@ -101,7 +110,7 @@ public class FieldWriters
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return new FloatFieldWriter(selector);
return FloatFieldWriter.forPrimitive(selector);
}
private static FieldWriter makeDoubleWriter(
@ -110,7 +119,7 @@ public class FieldWriters
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return new DoubleFieldWriter(selector);
return DoubleFieldWriter.forPrimitive(selector);
}
private static FieldWriter makeStringWriter(
@ -127,11 +136,37 @@ public class FieldWriters
final String columnName
)
{
//noinspection rawtypes
final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return new StringArrayFieldWriter(selector);
}
private static FieldWriter makeLongArrayWriter(
final ColumnSelectorFactory selectorFactory,
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return NumericArrayFieldWriter.getLongArrayFieldWriter(selector);
}
private static FieldWriter makeFloatArrayWriter(
final ColumnSelectorFactory selectorFactory,
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return NumericArrayFieldWriter.getFloatArrayFieldWriter(selector);
}
private static FieldWriter makeDoubleArrayWriter(
final ColumnSelectorFactory selectorFactory,
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
return NumericArrayFieldWriter.getDoubleArrayFieldWriter(selector);
}
private static FieldWriter makeComplexWriter(
final ColumnSelectorFactory selectorFactory,
final String columnName,

View File

@ -0,0 +1,62 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
/**
* Reader for fields written by {@link NumericArrayFieldWriter#getFloatArrayFieldWriter}
*/
public class FloatArrayFieldReader extends NumericArrayFieldReader
{
@Override
public ColumnValueSelector<?> makeColumnValueSelector(
Memory memory,
ReadableFieldPointer fieldPointer
)
{
return new NumericArrayFieldSelector<Float>(memory, fieldPointer)
{
final SettableFieldPointer fieldPointer = new SettableFieldPointer();
final ColumnValueSelector<?> columnValueSelector =
FloatFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer);
@Nullable
@Override
public Float getIndividualValueAtMemory(long position)
{
fieldPointer.setPosition(position);
if (columnValueSelector.isNull()) {
return null;
}
return columnValueSelector.getFloat();
}
@Override
public int getIndividualFieldSize()
{
return Byte.BYTES + Float.BYTES;
}
};
}
}

View File

@ -21,71 +21,60 @@ package org.apache.druid.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.FloatColumnSelector;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.column.ValueTypes;
import javax.annotation.Nullable;
/**
* Reads values written by {@link FloatFieldWriter}.
*
* Values are sortable as bytes without decoding.
*
* Format:
*
* - 1 byte: {@link FloatFieldWriter#NULL_BYTE} or {@link FloatFieldWriter#NOT_NULL_BYTE}
* - 4 bytes: encoded float, using {@link FloatFieldWriter#transform}
* @see FloatFieldWriter
* @see NumericFieldWriter for the details of the byte-format that it expects for reading
*/
public class FloatFieldReader implements FieldReader
public class FloatFieldReader extends NumericFieldReader
{
FloatFieldReader()
public static FloatFieldReader forPrimitive()
{
return new FloatFieldReader(false);
}
public static FloatFieldReader forArray()
{
return new FloatFieldReader(true);
}
private FloatFieldReader(final boolean forArray)
{
super(forArray);
}
@Override
public ColumnValueSelector<?> makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer)
public ValueType getValueType()
{
return new Selector(memory, fieldPointer);
return ValueType.FLOAT;
}
@Override
public DimensionSelector makeDimensionSelector(
Memory memory,
ReadableFieldPointer fieldPointer,
@Nullable ExtractionFn extractionFn
public ColumnValueSelector<?> getColumnValueSelector(
final Memory memory,
final ReadableFieldPointer fieldPointer,
final byte nullIndicatorByte
)
{
return ValueTypes.makeNumericWrappingDimensionSelector(
ValueType.FLOAT,
makeColumnValueSelector(memory, fieldPointer),
extractionFn
);
return new FloatFieldSelector(memory, fieldPointer, nullIndicatorByte);
}
@Override
public boolean isNull(Memory memory, long position)
private static class FloatFieldSelector extends NumericFieldReader.Selector implements FloatColumnSelector
{
return memory.getByte(position) == FloatFieldWriter.NULL_BYTE;
}
@Override
public boolean isComparable()
{
return true;
}
final Memory dataRegion;
final ReadableFieldPointer fieldPointer;
private static class Selector implements FloatColumnSelector
{
private final Memory dataRegion;
private final ReadableFieldPointer fieldPointer;
private Selector(final Memory dataRegion, final ReadableFieldPointer fieldPointer)
public FloatFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte)
{
super(dataRegion, fieldPointer, nullIndicatorByte);
this.dataRegion = dataRegion;
this.fieldPointer = fieldPointer;
}
@ -95,19 +84,19 @@ public class FloatFieldReader implements FieldReader
{
assert NullHandling.replaceWithDefault() || !isNull();
final int bits = dataRegion.getInt(fieldPointer.position() + Byte.BYTES);
return FloatFieldWriter.detransform(bits);
}
@Override
public boolean isNull()
{
return dataRegion.getByte(fieldPointer.position()) == FloatFieldWriter.NULL_BYTE;
return TransformUtils.detransformToFloat(bits);
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
// Do nothing.
}
@Override
public boolean isNull()
{
return super._isNull();
}
}
}

View File

@ -25,40 +25,26 @@ import org.apache.druid.segment.BaseFloatColumnValueSelector;
/**
* Wraps a {@link BaseFloatColumnValueSelector} and writes field values.
*
* See {@link FloatFieldReader} for format details.
* @see NumericFieldWriter for the details of the byte-format that it writes as
*/
public class FloatFieldWriter implements FieldWriter
public class FloatFieldWriter extends NumericFieldWriter
{
public static final int SIZE = Float.BYTES + Byte.BYTES;
// Different from the values in NullHandling, since we want to be able to sort as bytes, and we want
// nulls to come before non-nulls.
public static final byte NULL_BYTE = 0x00;
public static final byte NOT_NULL_BYTE = 0x01;
private final BaseFloatColumnValueSelector selector;
public FloatFieldWriter(final BaseFloatColumnValueSelector selector)
public static FloatFieldWriter forPrimitive(final BaseFloatColumnValueSelector selector)
{
this.selector = selector;
return new FloatFieldWriter(selector, false);
}
@Override
public long writeTo(final WritableMemory memory, final long position, final long maxSize)
public static FloatFieldWriter forArray(final BaseFloatColumnValueSelector selector)
{
if (maxSize < SIZE) {
return -1;
}
return new FloatFieldWriter(selector, true);
}
if (selector.isNull()) {
memory.putByte(position, NULL_BYTE);
memory.putInt(position + Byte.BYTES, transform(0));
} else {
memory.putByte(position, NOT_NULL_BYTE);
memory.putInt(position + Byte.BYTES, transform(selector.getFloat()));
}
return SIZE;
private FloatFieldWriter(final BaseFloatColumnValueSelector selector, final boolean forArray)
{
super(selector, forArray);
this.selector = selector;
}
@Override
@ -67,23 +53,26 @@ public class FloatFieldWriter implements FieldWriter
// Nothing to close.
}
/**
* Transforms a float into a form where it can be compared as unsigned bytes without decoding.
*/
public static int transform(final float n)
@Override
public int getNumericSizeBytes()
{
final int bits = Float.floatToIntBits(n);
final int mask = ((bits & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE;
return Integer.reverseBytes(bits ^ mask);
return Float.BYTES;
}
/**
* Inverse of {@link #transform}.
*/
public static float detransform(final int bits)
@Override
public void writeSelectorToMemory(WritableMemory memory, long position)
{
final int reversedBits = Integer.reverseBytes(bits);
final int mask = (((reversedBits ^ Integer.MIN_VALUE) & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE;
return Float.intBitsToFloat(reversedBits ^ mask);
writeToMemory(memory, position, selector.getFloat());
}
@Override
public void writeNullToMemory(WritableMemory memory, long position)
{
writeToMemory(memory, position, 0);
}
private void writeToMemory(WritableMemory memory, long position, float value)
{
memory.putInt(position, TransformUtils.transformFromFloat(value));
}
}

View File

@ -0,0 +1,62 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
/**
* Reader for fields written by {@link NumericArrayFieldWriter#getLongArrayFieldWriter}
*/
public class LongArrayFieldReader extends NumericArrayFieldReader
{
@Override
public ColumnValueSelector<?> makeColumnValueSelector(
Memory memory,
ReadableFieldPointer fieldPointer
)
{
return new NumericArrayFieldSelector<Long>(memory, fieldPointer)
{
final SettableFieldPointer fieldPointer = new SettableFieldPointer();
final ColumnValueSelector<?> columnValueSelector =
LongFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer);
@Nullable
@Override
public Long getIndividualValueAtMemory(long position)
{
fieldPointer.setPosition(position);
if (columnValueSelector.isNull()) {
return null;
}
return columnValueSelector.getLong();
}
@Override
public int getIndividualFieldSize()
{
return Byte.BYTES + Long.BYTES;
}
};
}
}

View File

@ -21,75 +21,61 @@ package org.apache.druid.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.LongColumnSelector;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.column.ValueTypes;
import javax.annotation.Nullable;
/**
* Reads values written by {@link LongFieldWriter}.
*
* Values are sortable as bytes without decoding.
*
* Format:
*
* - 1 byte: {@link LongFieldWriter#NULL_BYTE} or {@link LongFieldWriter#NOT_NULL_BYTE}
* - 8 bytes: encoded long: big-endian order, with sign flipped
* @see LongFieldWriter
* @see NumericFieldWriter for the details of the byte-format that it expects for reading
*/
public class LongFieldReader implements FieldReader
public class LongFieldReader extends NumericFieldReader
{
LongFieldReader()
public static LongFieldReader forPrimitive()
{
return new LongFieldReader(false);
}
public static LongFieldReader forArray()
{
return new LongFieldReader(true);
}
private LongFieldReader(final boolean forArray)
{
super(forArray);
}
@Override
public ColumnValueSelector<?> makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer)
public ValueType getValueType()
{
return new Selector(memory, fieldPointer);
return ValueType.LONG;
}
@Override
public DimensionSelector makeDimensionSelector(
Memory memory,
ReadableFieldPointer fieldPointer,
@Nullable ExtractionFn extractionFn
public ColumnValueSelector<?> getColumnValueSelector(
final Memory memory,
final ReadableFieldPointer fieldPointer,
final byte nullIndicatorByte
)
{
return ValueTypes.makeNumericWrappingDimensionSelector(
ValueType.LONG,
makeColumnValueSelector(memory, fieldPointer),
extractionFn
);
return new LongFieldSelector(memory, fieldPointer, nullIndicatorByte);
}
@Override
public boolean isNull(Memory memory, long position)
private static class LongFieldSelector extends NumericFieldReader.Selector implements LongColumnSelector
{
return memory.getByte(position) == LongFieldWriter.NULL_BYTE;
}
@Override
public boolean isComparable()
{
return true;
}
final Memory dataRegion;
final ReadableFieldPointer fieldPointer;
/**
* Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}.
*/
private static class Selector implements LongColumnSelector
{
private final Memory memory;
private final ReadableFieldPointer fieldPointer;
private Selector(final Memory memory, final ReadableFieldPointer fieldPointer)
public LongFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte)
{
this.memory = memory;
super(dataRegion, fieldPointer, nullIndicatorByte);
this.dataRegion = dataRegion;
this.fieldPointer = fieldPointer;
}
@ -97,20 +83,20 @@ public class LongFieldReader implements FieldReader
public long getLong()
{
assert NullHandling.replaceWithDefault() || !isNull();
final long bits = memory.getLong(fieldPointer.position() + Byte.BYTES);
return LongFieldWriter.detransform(bits);
final long bits = dataRegion.getLong(fieldPointer.position() + Byte.BYTES);
return TransformUtils.detransformToLong(bits);
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
}
@Override
public boolean isNull()
{
return memory.getByte(fieldPointer.position()) == LongFieldWriter.NULL_BYTE;
}
@Override
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
{
// Do nothing.
return super._isNull();
}
}
}

View File

@ -23,63 +23,55 @@ import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.segment.BaseLongColumnValueSelector;
/**
* Wraps a {@link BaseLongColumnValueSelector} and writes individual values into rframe rows.
* Wraps a {@link BaseLongColumnValueSelector} and writes individual values into frame rows.
*
* See {@link LongFieldReader} for format details.
* @see NumericFieldWriter for the details of the byte-format that it writes as
*/
public class LongFieldWriter implements FieldWriter
public class LongFieldWriter extends NumericFieldWriter
{
public static final int SIZE = Long.BYTES + Byte.BYTES;
public static final byte NULL_BYTE = 0x00;
public static final byte NOT_NULL_BYTE = 0x01;
private final BaseLongColumnValueSelector selector;
public static LongFieldWriter forPrimitive(final BaseLongColumnValueSelector selector)
{
return new LongFieldWriter(selector, false);
}
static LongFieldWriter forArray(final BaseLongColumnValueSelector selector)
{
return new LongFieldWriter(selector, true);
}
// Different from the values in NullHandling, since we want to be able to sort as bytes, and we want
// nulls to come before non-nulls.
public LongFieldWriter(final BaseLongColumnValueSelector selector)
private LongFieldWriter(final BaseLongColumnValueSelector selector, final boolean forArray)
{
super(selector, forArray);
this.selector = selector;
}
@Override
public long writeTo(final WritableMemory memory, final long position, final long maxSize)
public int getNumericSizeBytes()
{
if (maxSize < SIZE) {
return -1;
}
if (selector.isNull()) {
memory.putByte(position, NULL_BYTE);
memory.putLong(position + Byte.BYTES, transform(0));
} else {
memory.putByte(position, NOT_NULL_BYTE);
memory.putLong(position + Byte.BYTES, transform(selector.getLong()));
}
return SIZE;
return Long.BYTES;
}
@Override
public void close()
public void writeSelectorToMemory(WritableMemory memory, long position)
{
// Nothing to do.
writeToMemory(memory, position, selector.getLong());
}
/**
* Transforms a long into a form where it can be compared as unsigned bytes without decoding.
*/
public static long transform(final long n)
@Override
public void writeNullToMemory(WritableMemory memory, long position)
{
// Must flip the first (sign) bit so comparison-as-bytes works.
return Long.reverseBytes(n ^ Long.MIN_VALUE);
writeToMemory(memory, position, 0);
}
/**
* Reverse the {@link #transform(long)} function.
*/
public static long detransform(final long bits)
private void writeToMemory(WritableMemory memory, long position, long value)
{
return Long.reverseBytes(bits) ^ Long.MIN_VALUE;
memory.putLong(position, TransformUtils.transformFromLong(value));
}
}

View File

@ -0,0 +1,59 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.error.DruidException;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.segment.DimensionSelector;
import javax.annotation.Nullable;
/**
* Reader class for the fields written by {@link NumericArrayFieldWriter}. See the Javadoc for the writer for more
* information on the format
*
* The numeric array fields are byte comparable
*/
public abstract class NumericArrayFieldReader implements FieldReader
{
@Override
public DimensionSelector makeDimensionSelector(
Memory memory,
ReadableFieldPointer fieldPointer,
@Nullable ExtractionFn extractionFn
)
{
throw DruidException.defensive("Cannot call makeDimensionSelector on field of type ARRAY");
}
@Override
public boolean isNull(Memory memory, long position)
{
final byte firstByte = memory.getByte(position);
return firstByte == NumericArrayFieldWriter.NULL_ROW;
}
@Override
public boolean isComparable()
{
return true;
}
}

View File

@ -0,0 +1,204 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.error.DruidException;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
/**
* Base implementation of the column value selector that the concrete numeric field reader implementations inherit from.
* The selector contains the logic to construct an array written by {@link NumericArrayFieldWriter}, and present it as
* a column value selector.
*
* The inheritors of this class are expected to implement
* 1. {@link #getIndividualValueAtMemory} Which extracts the element from the field where it was written to. Returns
* null if the value at that location represents a null element
* 2. {@link #getIndividualFieldSize} Which informs the method about the field size corresponding to each element in
* the numeric array's serialized representation
*
* @param <ElementType> Type of the individual array elements
*/
public abstract class NumericArrayFieldSelector<ElementType extends Number> implements ColumnValueSelector
{
/**
* Memory containing the serialized values of the array
*/
protected final Memory memory;
/**
* Pointer to location in the memory. The callers are expected to update the pointer's position to the start of the
* array that they wish to get prior to {@link #getObject()} call.
*
* Frames read and written using {@link org.apache.druid.frame.write.FrameWriter} and
* {@link org.apache.druid.frame.read.FrameReader} shouldn't worry about this detail, since they automatically update
* and handle the start location
*/
private final ReadableFieldPointer fieldPointer;
/**
* Position last read, for caching the last fetched result
*/
private long currentFieldPosition = -1;
/**
* Value of the row at the location beginning at {@link #currentFieldPosition}
*/
private final List<ElementType> currentRow = new ArrayList<>();
/**
* Nullity of the row at the location beginning at {@link #currentFieldPosition}
*/
private boolean currentRowIsNull;
public NumericArrayFieldSelector(final Memory memory, final ReadableFieldPointer fieldPointer)
{
this.memory = memory;
this.fieldPointer = fieldPointer;
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
// Do nothing
}
@Nullable
@Override
public Object getObject()
{
final List<ElementType> currentArray = computeCurrentArray();
if (currentArray == null) {
return null;
}
return currentArray.toArray();
}
@Override
public Class classOfObject()
{
return Object.class;
}
@Override
public double getDouble()
{
return 0;
}
@Override
public float getFloat()
{
return 0;
}
@Override
public long getLong()
{
return 0;
}
@Override
public boolean isNull()
{
long position = fieldPointer.position();
final byte firstByte = memory.getByte(position);
return firstByte == NumericArrayFieldWriter.NULL_ROW;
}
/**
* Returns the value of the individual element written at the given position
*/
@Nullable
public abstract ElementType getIndividualValueAtMemory(long position);
/**
* Returns the field size that each element in the reader array consumes. It is usually 1 + ElementType.SIZE, to hold
* the element's nullity, and it's representation.
*/
public abstract int getIndividualFieldSize();
@Nullable
private List<ElementType> computeCurrentArray()
{
final long fieldPosition = fieldPointer.position();
if (fieldPosition != currentFieldPosition) {
updateCurrentArray(fieldPosition);
}
this.currentFieldPosition = fieldPosition;
if (currentRowIsNull) {
return null;
}
return currentRow;
}
private void updateCurrentArray(final long fieldPosition)
{
currentRow.clear();
currentRowIsNull = false;
long position = fieldPosition;
long limit = memory.getCapacity();
// Check the first byte, and if it is null, update the current value to null and return
if (isNull()) {
currentRowIsNull = true;
return;
}
// Adding a check here to prevent the position from potentially overflowing
if (position < limit) {
position++;
}
// Sanity check, to make sure that we see the rowTerminator at the end
boolean rowTerminatorSeen = false;
while (position < limit) {
final byte kind = memory.getByte(position);
// Break as soon as we see the ARRAY_TERMINATOR (0x00)
if (kind == NumericArrayFieldWriter.ARRAY_TERMINATOR) {
rowTerminatorSeen = true;
break;
}
// If terminator not seen, then read the field at that location, and increment the position by the element's field
// size to read the next element.
currentRow.add(getIndividualValueAtMemory(position));
position += getIndividualFieldSize();
}
if (!rowTerminatorSeen) {
throw DruidException.defensive("Unexpected end of field");
}
}
}

View File

@ -0,0 +1,249 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Writes the values of the type ARRAY<X> where X is a numeric type to row based frames.
* The format of the array written is as follows:
* <p>
* Format:
* - 1 Byte - {@link #NULL_ROW} or {@link #NON_NULL_ROW} denoting whether the array itself is null
* - If the array is null, then the writer stops here
* - If the array is not null, then it proceeds to the following steps
* <p>
* For each value in the non-null array:
* - 1 Byte - {@link NumericFieldWriter#ARRAY_ELEMENT_NULL_BYTE} or {@link NumericFieldWriter#ARRAY_ELEMENT_NOT_NULL_BYTE}
* denothing whether the proceeding value is null or not.
* - ElementSize Bytes - The encoded value of the element
* <p>
* Once all the values in the non-null arrays are over, writes {@link #ARRAY_TERMINATOR}. This is to aid the byte
* comparison, and also let the reader know that the number of elements in the array are over.
* <p>
* The format doesn't add the number of elements in the array at the beginning, so that the serialization of the arrays
* are byte-by-byte comparable.
* <p>
* Examples:
* 1. null
* | Bytes | Value | Interpretation |
* |--------|-------|-----------------------------|
* | 1 | 0x00 | Denotes that the array null |
* <p>
* 2. [] (empty array)
* | Bytes | Value | Interpretation |
* |--------|----- -|------------------------------------|
* | 1 | 0x01 | Denotes that the array is not null |
* | 2 | 0x00 | End of the array |
* <p>
* 3. [5L, null, 6L]
* | Bytes | Value | Interpretation |
* |---------|--------------|-----------------------------------------------------------------------------------|
* | 1 | 0x01 | Denotes that the array is not null |
* | 2 | 0x02 | Denotes that the next element is not null |
* | 3-10 | transform(5) | Representation of 5 |
* | 11 | 0x01 | Denotes that the next element is null |
* | 12-19 | transform(0) | Representation of 0 (default value, the reader will ignore it if SqlCompatible mode is on |
* | 20 | 0x02 | Denotes that the next element is not null |
* | 21-28 | transform(6) | Representation of 6 |
* | 29 | 0x00 | End of array |
*/
public class NumericArrayFieldWriter implements FieldWriter
{
/**
* Denotes that the array itself is null
*/
public static final byte NULL_ROW = 0x00;
/**
* Denotes that the array is non null
*/
public static final byte NON_NULL_ROW = 0x01;
/**
* Marks the end of the array. Since {@link #NULL_ROW} and {@link #ARRAY_TERMINATOR} will only occur at different
* locations, therefore there is no clash in keeping both's values at 0x00
*/
public static final byte ARRAY_TERMINATOR = 0x00;
private final ColumnValueSelector selector;
private final NumericFieldWriterFactory writerFactory;
/**
* Returns the writer for ARRAY<LONG>
*/
public static NumericArrayFieldWriter getLongArrayFieldWriter(final ColumnValueSelector selector)
{
return new NumericArrayFieldWriter(selector, LongFieldWriter::forArray);
}
/**
* Returns the writer for ARRAY<FLOAT>
*/
public static NumericArrayFieldWriter getFloatArrayFieldWriter(final ColumnValueSelector selector)
{
return new NumericArrayFieldWriter(selector, FloatFieldWriter::forArray);
}
/**
* Returns the writer for ARRAY<DOUBLE>
*/
public static NumericArrayFieldWriter getDoubleArrayFieldWriter(final ColumnValueSelector selector)
{
return new NumericArrayFieldWriter(selector, DoubleFieldWriter::forArray);
}
public NumericArrayFieldWriter(final ColumnValueSelector selector, NumericFieldWriterFactory writerFactory)
{
this.selector = selector;
this.writerFactory = writerFactory;
}
@Override
public long writeTo(WritableMemory memory, long position, long maxSize)
{
Object row = selector.getObject();
if (row == null) {
int requiredSize = Byte.BYTES;
if (requiredSize > maxSize) {
return -1;
}
memory.putByte(position, NULL_ROW);
return requiredSize;
} else {
List<? extends Number> list = FrameWriterUtils.getNumericArrayFromObject(row);
if (list == null) {
int requiredSize = Byte.BYTES;
if (requiredSize > maxSize) {
return -1;
}
memory.putByte(position, NULL_ROW);
return requiredSize;
}
// Create a columnValueSelector to write the individual elements re-using the NumericFieldWriter
AtomicInteger index = new AtomicInteger(0);
ColumnValueSelector<Number> columnValueSelector = new ColumnValueSelector<Number>()
{
@Override
public double getDouble()
{
final Number n = getObject();
assert NullHandling.replaceWithDefault() || n != null;
return n != null ? n.doubleValue() : 0d;
}
@Override
public float getFloat()
{
final Number n = getObject();
assert NullHandling.replaceWithDefault() || n != null;
return n != null ? n.floatValue() : 0f;
}
@Override
public long getLong()
{
final Number n = getObject();
assert NullHandling.replaceWithDefault() || n != null;
return n != null ? n.longValue() : 0L;
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
}
@Override
public boolean isNull()
{
// Arrays preserve the individual element's nullity when they are written and read.
// Therefore, when working with SQL incompatible mode, [7, null] won't change to [7, 0] when written to and
// read from the underlying serialization (as compared with the primitives). Therefore,
// even when NullHandling.replaceWithDefault() is true we need to write null as is, and not convert it to their
// default value when writing the array. Therefore, the check is `getObject() == null` ignoring the value of
// `NullHandling.replaceWithDefaul()`.
return getObject() == null;
}
@Nullable
@Override
public Number getObject()
{
return list.get(index.get());
}
@Override
public Class<? extends Number> classOfObject()
{
return Number.class;
}
};
NumericFieldWriter writer = writerFactory.get(columnValueSelector);
// First byte is reserved for null marker of the array
// Next [(1 + Numeric Size) x Number of elements of array] bytes are reserved for the elements of the array and
// their null markers
// Last byte is reserved for array termination
int requiredSize = Byte.BYTES + (writer.getNumericSizeBytes() + Byte.BYTES) * list.size() + Byte.BYTES;
if (requiredSize > maxSize) {
return -1;
}
long offset = 0;
memory.putByte(position + offset, NON_NULL_ROW);
offset += Byte.BYTES;
for (; index.get() < list.size(); index.incrementAndGet()) {
writer.writeTo(
memory,
position + offset,
maxSize - offset
);
offset += Byte.BYTES + writer.getNumericSizeBytes();
}
memory.putByte(position + offset, ARRAY_TERMINATOR);
return requiredSize;
}
}
@Override
public void close()
{
// Do nothing
}
}

View File

@ -0,0 +1,131 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.column.ValueTypes;
import javax.annotation.Nullable;
/**
* Reads the fields created by the {@link NumericFieldWriter}. See the Javadoc for the writer for format details
*
* @see NumericFieldWriter
*/
public abstract class NumericFieldReader implements FieldReader
{
/**
* The indicator byte which denotes that the following value is null.
*/
private final byte nullIndicatorByte;
public NumericFieldReader(boolean forArray)
{
if (!forArray) {
this.nullIndicatorByte = NumericFieldWriter.NULL_BYTE;
} else {
this.nullIndicatorByte = NumericFieldWriter.ARRAY_ELEMENT_NULL_BYTE;
}
}
@Override
public ColumnValueSelector<?> makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer)
{
return getColumnValueSelector(memory, fieldPointer, nullIndicatorByte);
}
@Override
public DimensionSelector makeDimensionSelector(
Memory memory,
ReadableFieldPointer fieldPointer,
@Nullable ExtractionFn extractionFn
)
{
return ValueTypes.makeNumericWrappingDimensionSelector(
getValueType(),
makeColumnValueSelector(memory, fieldPointer),
extractionFn
);
}
@Override
public boolean isNull(Memory memory, long position)
{
return memory.getByte(position) == nullIndicatorByte;
}
@Override
public boolean isComparable()
{
return true;
}
/**
* Creates a column value selector for the element written at fieldPointer's position in the memory.
* The nullilty check is handled by the nullIndicatorByte
*/
public abstract ColumnValueSelector<?> getColumnValueSelector(
Memory memory,
ReadableFieldPointer fieldPointer,
byte nullIndicatorByte
);
/**
* {@link ValueType} of the inheritor's element
*/
public abstract ValueType getValueType();
/**
* Helper class which allows the inheritors to fetch the nullity of the field located at fieldPointer's position in
* the dataRegion.
*
* The implementations of the column value selectors returned by the {@link #getColumnValueSelector} can inherit this
* class and call {@link #_isNull()} in their {@link ColumnValueSelector#isNull()} to offload the responsibility of
* detecting null elements to this Selector, instead of reworking the null handling
*/
public abstract static class Selector
{
private final Memory dataRegion;
private final ReadableFieldPointer fieldPointer;
private final byte nullIndicatorByte;
public Selector(
final Memory dataRegion,
final ReadableFieldPointer fieldPointer,
final byte nullIndicatorByte
)
{
this.dataRegion = dataRegion;
this.fieldPointer = fieldPointer;
this.nullIndicatorByte = nullIndicatorByte;
}
protected boolean _isNull()
{
return dataRegion.getByte(fieldPointer.position()) == nullIndicatorByte;
}
}
}

View File

@ -0,0 +1,139 @@
/*
* 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.frame.field;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
/**
* FieldWriter for numeric datatypes. The parent class does the null handling for the underlying data, while
* the individual subclasses write the individual element (long, float or double type). This also allows for a clean
* reuse of the readers and writers between the numeric types and also allowing the array writers ({@link NumericArrayFieldWriter})
* to use these methods directly without duplication
*
* Format:
* - 1 byte: Whether the following value is null or not. Take a look at the note on the indicator bytes.
* - X bytes: Encoded value of the selector, or the default value if it is null. X denotes the size of the numeric value
*
* Indicator bytes for denoting whether the element is null or not null changes depending on whether the writer is used
* to write the data for individual value (like LONG) or for an element of an array (like ARRAY<LONG>). This is because
* array support for the numeric types was added later and by then the field writers for individual fields were using
* 0x00 to denote the null byte, which is reserved for denoting the array end when we are writing the elements as part
* of the array instead. (0x00 is used for array end because it helps in preserving the byte comparison property of the
* numeric array field writers).
*
* Therefore, to preserve backward and forward compatibility, the individual element's writers were left unchanged,
* while the array's element's writers used 0x01 and 0x02 to denote null and non-null byte respectively
*
* Values produced by the writer are sortable without decoding
*
* @see NumericArrayFieldWriter for examples of how this class serializes the field for numeric arrays
*/
public abstract class NumericFieldWriter implements FieldWriter
{
/**
* Indicator byte denoting that the numeric value succeeding it is null. This is used in the primitive
* writers. NULL_BYTE < NOT_NULL_BYTE to preserve the ordering while doing byte comparison
*/
public static final byte NULL_BYTE = 0x00;
/**
* Indicator byte denoting that the numeric value succeeding it is not null. This is used in the primitive
* writers
*/
public static final byte NOT_NULL_BYTE = 0x01;
/**
* Indicator byte denoting that the numeric value succeeding it is null. This is used while writing the individual
* elements writers of an array. ARRAY_ELEMENT_NULL_BYTE < ARRAY_ELEMENT_NOT_NULL_BYTE to preserve the ordering
* while doing byte comparison
*/
public static final byte ARRAY_ELEMENT_NULL_BYTE = 0x01;
/**
* Indicator byte denoting that the numeric value succeeding it is not null. This is used while writing the individual
* elements writers of an array
*/
public static final byte ARRAY_ELEMENT_NOT_NULL_BYTE = 0x02;
private final BaseNullableColumnValueSelector selector;
private final byte nullIndicatorByte;
private final byte notNullIndicatorByte;
public NumericFieldWriter(
final BaseNullableColumnValueSelector selector,
final boolean forArray
)
{
this.selector = selector;
if (!forArray) {
this.nullIndicatorByte = NULL_BYTE;
this.notNullIndicatorByte = NOT_NULL_BYTE;
} else {
this.nullIndicatorByte = ARRAY_ELEMENT_NULL_BYTE;
this.notNullIndicatorByte = ARRAY_ELEMENT_NOT_NULL_BYTE;
}
}
@Override
public long writeTo(WritableMemory memory, long position, long maxSize)
{
int size = getNumericSizeBytes() + Byte.BYTES;
if (maxSize < size) {
return -1;
}
// Using isNull() since this is a primitive type
if (selector.isNull()) {
memory.putByte(position, nullIndicatorByte);
writeNullToMemory(memory, position + Byte.BYTES);
} else {
memory.putByte(position, notNullIndicatorByte);
writeSelectorToMemory(memory, position + Byte.BYTES);
}
return size;
}
@Override
public void close()
{
// Nothing to do
}
/**
* @return The size in bytes of the numeric datatype that the implementation of this writer occupies
*/
public abstract int getNumericSizeBytes();
/**
* Writes the value pointed by the selector to memory. The caller should ensure that the selector gives out the
* correct primitive type
*/
public abstract void writeSelectorToMemory(WritableMemory memory, long position);
/**
* Writes the default value for the type to the memory. For long, it is 0L, for double, it is 0.0d etc. Useful mainly
* when the SQL incompatible mode is turned off, and maintains the fact that the size of the numeric field written
* doesn't vary irrespective of whether the value is null
*/
public abstract void writeNullToMemory(WritableMemory memory, long position);
}

View File

@ -0,0 +1,33 @@
/*
* 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.frame.field;
import org.apache.druid.segment.ColumnValueSelector;
/**
* Factory for {@link NumericFieldWriter}
*/
public interface NumericFieldWriterFactory
{
/**
* Constructs an instance of {@link NumericFieldWriter} given the column selector
*/
NumericFieldWriter get(ColumnValueSelector<Number> selector);
}

View File

@ -0,0 +1,40 @@
/*
* 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.frame.field;
/**
* A simple {@link ReadableFieldPointer} that returns the position that was set on its object.
*/
public class SettableFieldPointer implements ReadableFieldPointer
{
long position = 0;
public void setPosition(long position)
{
this.position = position;
}
@Override
public long position()
{
return position;
}
}

View File

@ -0,0 +1,34 @@
/*
* 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.frame.field;
/**
* Reads fields written by {@link StringArrayFieldWriter}
*
* @see StringFieldReader for more details on the format that the reader expects
* @see StringFieldReader#StringFieldReader(boolean) for the selector that the reader returns
*/
public class StringArrayFieldReader extends StringFieldReader
{
StringArrayFieldReader()
{
super(true);
}
}

View File

@ -70,6 +70,11 @@ public class StringFieldReader implements FieldReader
{
private final boolean asArray;
public StringFieldReader()
{
this(false);
}
/**
* Create a string reader.
*
@ -77,7 +82,7 @@ public class StringFieldReader implements FieldReader
* selectors (potentially multi-value ones). If true, selectors from {@link #makeColumnValueSelector}
* behave like string array selectors.
*/
StringFieldReader(final boolean asArray)
protected StringFieldReader(final boolean asArray)
{
this.asArray = asArray;
}

View File

@ -0,0 +1,86 @@
/*
* 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.frame.field;
/**
* Utility methods to map the primitive numeric types into an equi-wide byte representation, such that the
* given byte sequence preserves the ordering of the original type when done byte comparison.
* Checkout {@link org.apache.druid.frame.read.FrameReaderUtils#compareMemoryToByteArrayUnsigned} for how this byte
* comparison is performed.
*/
public class TransformUtils
{
/**
* Transforms a double into a form where it can be compared as unsigned bytes without decoding.
*/
public static long transformFromDouble(final double n)
{
final long bits = Double.doubleToLongBits(n);
final long mask = ((bits & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE;
return Long.reverseBytes(bits ^ mask);
}
/**
* Inverse of {@link #transformFromDouble}.
*/
public static double detransformToDouble(final long bits)
{
final long reversedBits = Long.reverseBytes(bits);
final long mask = (((reversedBits ^ Long.MIN_VALUE) & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE;
return Double.longBitsToDouble(reversedBits ^ mask);
}
/**
* Transforms a long into a form where it can be compared as unsigned bytes without decoding.
*/
public static long transformFromLong(final long n)
{
// Must flip the first (sign) bit so comparison-as-bytes works.
return Long.reverseBytes(n ^ Long.MIN_VALUE);
}
/**
* Reverse the {@link #transformFromLong(long)} function.
*/
public static long detransformToLong(final long bits)
{
return Long.reverseBytes(bits) ^ Long.MIN_VALUE;
}
/**
* Transforms a float into a form where it can be compared as unsigned bytes without decoding.
*/
public static int transformFromFloat(final float n)
{
final int bits = Float.floatToIntBits(n);
final int mask = ((bits & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE;
return Integer.reverseBytes(bits ^ mask);
}
/**
* Inverse of {@link #transformFromFloat(float)}.
*/
public static float detransformToFloat(final int bits)
{
final int reversedBits = Integer.reverseBytes(bits);
final int mask = (((reversedBits ^ Integer.MIN_VALUE) & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE;
return Float.intBitsToFloat(reversedBits ^ mask);
}
}

View File

@ -74,6 +74,7 @@ public class FrameReader
* If the columnType is null, we store the data as {@link ColumnType#NESTED_DATA}. This can be done if we know that
* the data that we receive can be serded generically using the nested data. It is currently used in the brokers to
* store the data with unknown types into frames.
*
* @param signature signature used to generate the reader
*/
public static FrameReader create(final RowSignature signature)
@ -96,8 +97,8 @@ public class FrameReader
signature.getColumnName(columnNumber)
);
columnReaders.add(FrameColumnReaders.create(columnNumber, columnType));
fieldReaders.add(FieldReaders.create(signature.getColumnName(columnNumber), columnType));
columnReaders.add(FrameColumnReaders.create(signature.getColumnName(columnNumber), columnNumber, columnType));
}
return new FrameReader(signature, columnReaders, fieldReaders);
@ -110,7 +111,7 @@ public class FrameReader
/**
* Returns capabilities for a particular column in a particular frame.
*
* <p>
* Preferred over {@link RowSignature#getColumnCapabilities(String)} when reading a particular frame, because this
* method has more insight into what's actually going on with that specific frame (nulls, multivalue, etc). The
* RowSignature version is based solely on type.
@ -151,7 +152,7 @@ public class FrameReader
/**
* Create a {@link FrameComparisonWidget} for the given frame.
*
* <p>
* Only possible for frames of type {@link org.apache.druid.frame.FrameType#ROW_BASED}. The provided
* sortColumns must be a prefix of {@link #signature()}.
*/

View File

@ -19,10 +19,15 @@
package org.apache.druid.frame.read.columnar;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ValueType;
/**
* Creates {@link FrameColumnReader} corresponding to a given column type and number.
*
* Returns a dummy {@link UnsupportedColumnTypeFrameColumnReader} if the column type is not supported or unknown.
* Calling any method of the dummy reader will throw with relevant error message.
*/
public class FrameColumnReaders
{
private FrameColumnReaders()
@ -30,7 +35,11 @@ public class FrameColumnReaders
// No instantiation.
}
public static FrameColumnReader create(final int columnNumber, final ColumnType columnType)
public static FrameColumnReader create(
final String columnName,
final int columnNumber,
final ColumnType columnType
)
{
switch (columnType.getType()) {
case LONG:
@ -51,11 +60,12 @@ public class FrameColumnReaders
case ARRAY:
if (columnType.getElementType().getType() == ValueType.STRING) {
return new StringFrameColumnReader(columnNumber, true);
} else {
return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType);
}
// Fall through to error for other array types
default:
throw new UOE("Unsupported column type [%s]", columnType);
return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType);
}
}
}

View File

@ -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.frame.read.columnar;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
import org.apache.druid.query.rowsandcols.column.Column;
import org.apache.druid.segment.column.ColumnType;
import javax.annotation.Nullable;
/**
* Dummy reader for unsupported types. Throws {@link UnsupportedColumnTypeException} if we try to call any method of the
* frame reader
*/
public class UnsupportedColumnTypeFrameColumnReader implements FrameColumnReader
{
private final String columnName;
@Nullable
private final ColumnType columnType;
UnsupportedColumnTypeFrameColumnReader(String columnName, @Nullable ColumnType columnType)
{
this.columnName = columnName;
this.columnType = columnType;
}
@Override
public Column readRACColumn(Frame frame)
{
throw new UnsupportedColumnTypeException(columnName, columnType);
}
@Override
public ColumnPlus readColumn(Frame frame)
{
throw new UnsupportedColumnTypeException(columnName, columnType);
}
}

View File

@ -33,6 +33,8 @@ import org.apache.druid.segment.DimensionDictionarySelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.data.ComparableIntArray;
import org.apache.druid.segment.data.ComparableList;
import org.apache.druid.segment.data.ComparableStringArray;
import org.apache.druid.segment.data.IndexedInts;
@ -132,7 +134,7 @@ public class FrameWriterUtils
* Retrieves UTF-8 byte buffers from a {@link ColumnValueSelector}, which is expected to be the kind of
* selector you get for an {@code ARRAY<STRING>} column.
*
* Null strings are returned as {@link #NULL_STRING_MARKER_ARRAY}.
* Null strings are returned as {@code null}.
*
* If the entire array returned by {@link BaseObjectColumnValueSelector#getObject()} is null, returns either
* null or {@link #NULL_STRING_MARKER_ARRAY} depending on the value of "useNullArrays".
@ -172,6 +174,48 @@ public class FrameWriterUtils
return retVal;
}
/**
* Retrieves a numeric list from a Java object, given that the object is an instance of something that can be returned
* from {@link ColumnValueSelector#getObject()} of valid numeric array selectors representations
*
* While {@link BaseObjectColumnValueSelector} specifies that only instances of {@code Object[]} can be returned from
* the numeric array selectors, this method also handles a few more cases which can be encountered if the selector is
* directly implemented on top of the group by stuff
*/
@Nullable
public static List<? extends Number> getNumericArrayFromObject(Object row)
{
if (row == null) {
return null;
} else if (row instanceof Number) {
return Collections.singletonList((Number) row);
}
final List<Number> retVal = new ArrayList<>();
if (row instanceof List) {
for (int i = 0; i < ((List<?>) row).size(); i++) {
retVal.add((Number) ((List<?>) row).get(i));
}
} else if (row instanceof Object[]) {
for (Object value : (Object[]) row) {
retVal.add((Number) value);
}
} else if (row instanceof ComparableList) {
for (Object value : ((ComparableList) row).getDelegate()) {
retVal.add((Number) value);
}
} else if (row instanceof ComparableIntArray) {
for (int value : ((ComparableIntArray) row).getDelegate()) {
retVal.add(value);
}
} else {
throw new ISE("Unexpected type %s found", row.getClass().getName());
}
return retVal;
}
/**
* Checks the provided signature for any disallowed field names. Returns any that are found.
*/

View File

@ -70,7 +70,7 @@ public class FrameRowsAndColumns implements RowsAndColumns
.getColumnType(columnIndex)
.orElseThrow(() -> new ISE("just got the id, why is columnType not there?"));
colCache.put(name, FrameColumnReaders.create(columnIndex, columnType).readRACColumn(frame));
colCache.put(name, FrameColumnReaders.create(name, columnIndex, columnType).readRACColumn(frame));
}
}
return colCache.get(name);

View File

@ -57,6 +57,8 @@ public class ColumnTypeFactory implements TypeFactory<ColumnType>
switch (type.getElementType().getType()) {
case LONG:
return ColumnType.LONG_ARRAY;
case FLOAT:
return ColumnType.FLOAT_ARRAY;
case DOUBLE:
return ColumnType.DOUBLE_ARRAY;
case STRING:

View File

@ -298,7 +298,7 @@ public class FrameBasedIndexedTable implements IndexedTable
"Type for column [%s]",
rowSignature.getColumnName(columnNumber)
);
columnReaders.add(FrameColumnReaders.create(columnNumber, columnType));
columnReaders.add(FrameColumnReaders.create(rowSignature.getColumnName(columnNumber), columnNumber, columnType));
}
return columnReaders;

View File

@ -0,0 +1,234 @@
/*
* 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.frame.field;
import com.google.common.collect.ImmutableList;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnit;
import org.mockito.junit.MockitoRule;
import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
public class DoubleArrayFieldReaderTest extends InitializedNullHandlingTest
{
private static final long MEMORY_POSITION = 1;
@Rule
public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
@Mock
public ColumnValueSelector writeSelector;
private WritableMemory memory;
private FieldWriter fieldWriter;
//CHECKSTYLE.OFF: Regexp
private static final Object[] DOUBLES_ARRAY_1 = new Object[]{
Double.MAX_VALUE,
Double.MIN_VALUE,
null,
Double.POSITIVE_INFINITY,
Double.NEGATIVE_INFINITY,
Double.MIN_NORMAL,
null,
0.0d,
1.234234d,
Double.NaN,
-1.344234d,
129123.123123,
-21312213.33,
null,
1111.0,
23.0,
null,
};
private static final Object[] DOUBLES_ARRAY_2 = new Object[]{
null,
Double.MAX_VALUE,
12.234234d,
-21.344234d,
Double.POSITIVE_INFINITY,
null,
Double.MIN_VALUE,
129123.123123,
null,
0.0d,
Double.MIN_NORMAL,
1111.0,
Double.NaN,
Double.NEGATIVE_INFINITY,
null,
-2133.33,
23.0,
null
};
//CHECKSTYLE.ON: Regexp
private static final List<Double> DOUBLES_LIST_1;
private static final List<Double> DOUBLES_LIST_2;
static {
DOUBLES_LIST_1 = Arrays.stream(DOUBLES_ARRAY_1).map(val -> (Double) val).collect(Collectors.toList());
DOUBLES_LIST_2 = Arrays.stream(DOUBLES_ARRAY_2).map(val -> (Double) val).collect(Collectors.toList());
}
@Before
public void setUp()
{
memory = WritableMemory.allocate(1000);
fieldWriter = NumericArrayFieldWriter.getDoubleArrayFieldWriter(writeSelector);
}
@After
public void tearDown()
{
fieldWriter.close();
}
@Test
public void test_isNull_null()
{
writeToMemory(null, MEMORY_POSITION);
Assert.assertTrue(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION);
Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_emptyArray()
{
writeToMemory(new Object[]{}, MEMORY_POSITION);
Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_arrayWithSingleNullElement()
{
writeToMemory(new Object[]{null}, MEMORY_POSITION);
Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_makeColumnValueSelector_null()
{
writeToMemory(null, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertTrue(readSelector.isNull());
}
@Test
public void test_makeColumnValueSelector_aValue()
{
writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(DOUBLES_LIST_1, readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_multipleValues()
{
long sz = writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION);
writeToMemory(DOUBLES_ARRAY_2, MEMORY_POSITION + sz);
IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz));
final ColumnValueSelector<?> readSelector = new DoubleArrayFieldReader().makeColumnValueSelector(memory, pointer);
pointer.setPointer(0);
assertResults(DOUBLES_LIST_1, readSelector.getObject());
pointer.setPointer(1);
assertResults(DOUBLES_LIST_2, readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_emptyArray()
{
writeToMemory(new Object[]{}, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(Collections.emptyList(), readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_arrayWithSingleNullElement()
{
writeToMemory(new Object[]{null}, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(Collections.singletonList(null), readSelector.getObject());
}
private long writeToMemory(final Object value, final long initialPosition)
{
Mockito.when(writeSelector.getObject()).thenReturn(value);
long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition);
if (bytesWritten < 0) {
throw new ISE("Could not write");
}
return bytesWritten;
}
private void assertResults(List<Double> expected, Object actual)
{
if (expected == null) {
Assert.assertNull(actual);
}
Assert.assertTrue(actual instanceof Object[]);
List<Double> actualList = new ArrayList<>();
for (Object val : (Object[]) actual) {
actualList.add((Double) val);
}
Assert.assertEquals(expected, actualList);
}
}

View File

@ -59,7 +59,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
public void setUp()
{
memory = WritableMemory.allocate(1000);
fieldWriter = new DoubleFieldWriter(writeSelector);
fieldWriter = DoubleFieldWriter.forPrimitive(writeSelector);
}
@After
@ -72,14 +72,14 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
public void test_isNull_defaultOrNull()
{
writeToMemory(NullHandling.defaultDoubleValue());
Assert.assertEquals(NullHandling.sqlCompatible(), new DoubleFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertEquals(NullHandling.sqlCompatible(), DoubleFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(5.1d);
Assert.assertFalse(new DoubleFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(DoubleFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION));
}
@Test
@ -88,7 +88,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(NullHandling.defaultDoubleValue());
final ColumnValueSelector<?> readSelector =
new DoubleFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull());
@ -103,7 +103,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(5.1d);
final ColumnValueSelector<?> readSelector =
new DoubleFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(5.1d, readSelector.getObject());
}
@ -114,7 +114,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(NullHandling.defaultDoubleValue());
final DimensionSelector readSelector =
new DoubleFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -148,7 +148,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(5.1d);
final DimensionSelector readSelector =
new DoubleFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -175,7 +175,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(20.5d);
final DimensionSelector readSelector =
new DoubleFieldReader().makeDimensionSelector(
DoubleFieldReader.forPrimitive().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)

View File

@ -0,0 +1,235 @@
/*
* 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.frame.field;
import com.google.common.collect.ImmutableList;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnit;
import org.mockito.junit.MockitoRule;
import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
public class FloatArrayFieldReaderTest extends InitializedNullHandlingTest
{
private static final long MEMORY_POSITION = 1;
@Rule
public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
@Mock
public ColumnValueSelector writeSelector;
private WritableMemory memory;
private FieldWriter fieldWriter;
//CHECKSTYLE.OFF: Regexp
private static final Object[] FLOATS_ARRAY_1 = new Object[]{
null,
Float.MIN_VALUE,
Float.MAX_VALUE,
Float.MIN_NORMAL,
null,
Float.POSITIVE_INFINITY,
Float.NEGATIVE_INFINITY,
null,
0.0f,
1.234234f,
-1.344234f,
Float.NaN,
129123.123123f,
null,
-21312213.33f,
1111.0f,
null,
23.0f
};
private static final Object[] FLOATS_ARRAY_2 = new Object[]{
null,
Float.MAX_VALUE,
null,
Float.POSITIVE_INFINITY,
-0.0f,
0.0f,
-1.234234f,
Float.MIN_VALUE,
1.344234333f,
-129123.123123f,
21312213.33f,
Float.NEGATIVE_INFINITY,
-1111.0f,
1223.0f,
Float.MIN_NORMAL,
null,
Float.NaN,
null
};
//CHECKSTYLE.ON: Regexp
private static final List<Float> FLOATS_LIST_1;
private static final List<Float> FLOATS_LIST_2;
static {
FLOATS_LIST_1 = Arrays.stream(FLOATS_ARRAY_1).map(val -> (Float) val).collect(Collectors.toList());
FLOATS_LIST_2 = Arrays.stream(FLOATS_ARRAY_2).map(val -> (Float) val).collect(Collectors.toList());
}
@Before
public void setUp()
{
memory = WritableMemory.allocate(1000);
fieldWriter = NumericArrayFieldWriter.getFloatArrayFieldWriter(writeSelector);
}
@After
public void tearDown()
{
fieldWriter.close();
}
@Test
public void test_isNull_null()
{
writeToMemory(null, MEMORY_POSITION);
Assert.assertTrue(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION);
Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_emptyArray()
{
writeToMemory(new Object[]{}, MEMORY_POSITION);
Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_arrayWithSingleNullElement()
{
writeToMemory(new Object[]{null}, MEMORY_POSITION);
Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_makeColumnValueSelector_null()
{
writeToMemory(null, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertTrue(readSelector.isNull());
}
@Test
public void test_makeColumnValueSelector_aValue()
{
writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(FLOATS_LIST_1, readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_multipleValues()
{
long sz = writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION);
writeToMemory(FLOATS_ARRAY_2, MEMORY_POSITION + sz);
IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz));
final ColumnValueSelector<?> readSelector = new FloatArrayFieldReader().makeColumnValueSelector(memory, pointer);
pointer.setPointer(0);
assertResults(FLOATS_LIST_1, readSelector.getObject());
pointer.setPointer(1);
assertResults(FLOATS_LIST_2, readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_emptyArray()
{
writeToMemory(new Object[]{}, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(Collections.emptyList(), readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_arrayWithSingleNullElement()
{
writeToMemory(new Object[]{null}, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(Collections.singletonList(null), readSelector.getObject());
}
private long writeToMemory(final Object value, final long initialPosition)
{
Mockito.when(writeSelector.getObject()).thenReturn(value);
long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition);
if (bytesWritten < 0) {
throw new ISE("Could not write");
}
return bytesWritten;
}
private void assertResults(List<Float> expected, Object actual)
{
if (expected == null) {
Assert.assertNull(actual);
}
Assert.assertTrue(actual instanceof Object[]);
List<Float> actualList = new ArrayList<>();
for (Object val : (Object[]) actual) {
actualList.add((Float) val);
}
Assert.assertEquals(expected, actualList);
}
}

View File

@ -59,7 +59,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
public void setUp()
{
memory = WritableMemory.allocate(1000);
fieldWriter = new FloatFieldWriter(writeSelector);
fieldWriter = FloatFieldWriter.forPrimitive(writeSelector);
}
@After
@ -72,14 +72,14 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
public void test_isNull_defaultOrNull()
{
writeToMemory(NullHandling.defaultFloatValue());
Assert.assertEquals(NullHandling.sqlCompatible(), new FloatFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertEquals(NullHandling.sqlCompatible(), FloatFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(5.1f);
Assert.assertFalse(new FloatFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(FloatFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION));
}
@Test
@ -88,7 +88,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(NullHandling.defaultFloatValue());
final ColumnValueSelector<?> readSelector =
new FloatFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull());
@ -103,7 +103,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(5.1f);
final ColumnValueSelector<?> readSelector =
new FloatFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(5.1f, readSelector.getObject());
}
@ -114,7 +114,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(NullHandling.defaultFloatValue());
final DimensionSelector readSelector =
new FloatFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -148,7 +148,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(5.1f);
final DimensionSelector readSelector =
new FloatFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -175,7 +175,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(20.5f);
final DimensionSelector readSelector =
new FloatFieldReader().makeDimensionSelector(
FloatFieldReader.forPrimitive().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)

View File

@ -0,0 +1,56 @@
/*
* 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.frame.field;
import it.unimi.dsi.fastutil.longs.LongArrayList;
import java.util.List;
/**
* Stores the memory locations in an array, and spits out the value pointed to by the memory location by pointer,
* which is settable by the user
*/
public class IndexArrayFieldPointer implements ReadableFieldPointer
{
private final LongArrayList indices;
private int pointer = 0;
public IndexArrayFieldPointer(final List<Long> indices)
{
this.indices = new LongArrayList(indices);
}
private int numIndices()
{
return indices.size();
}
public void setPointer(int newPointer)
{
assert newPointer >= 0 && newPointer < numIndices();
this.pointer = newPointer;
}
@Override
public long position()
{
return indices.getLong(pointer);
}
}

View File

@ -0,0 +1,211 @@
/*
* 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.frame.field;
import com.google.common.collect.ImmutableList;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnit;
import org.mockito.junit.MockitoRule;
import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
public class LongArrayFieldReaderTest extends InitializedNullHandlingTest
{
private static final long MEMORY_POSITION = 1;
@Rule
public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
@Mock
public ColumnValueSelector writeSelector;
private WritableMemory memory;
private FieldWriter fieldWriter;
private static final Object[] LONGS_ARRAY_1 = new Object[]{
Long.MIN_VALUE,
Long.MAX_VALUE,
null,
0L,
123L,
-123L
};
private static final Object[] LONGS_ARRAY_2 = new Object[]{
null,
234L,
Long.MAX_VALUE,
null,
Long.MIN_VALUE,
0L,
-234L,
null
};
private static final List<Long> LONGS_LIST_1;
private static final List<Long> LONGS_LIST_2;
static {
LONGS_LIST_1 = Arrays.stream(LONGS_ARRAY_1).map(val -> (Long) val).collect(Collectors.toList());
LONGS_LIST_2 = Arrays.stream(LONGS_ARRAY_2).map(val -> (Long) val).collect(Collectors.toList());
}
@Before
public void setUp()
{
memory = WritableMemory.allocate(1000);
fieldWriter = NumericArrayFieldWriter.getLongArrayFieldWriter(writeSelector);
}
@After
public void tearDown()
{
fieldWriter.close();
}
@Test
public void test_isNull_null()
{
writeToMemory(null, MEMORY_POSITION);
Assert.assertTrue(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION);
Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_emptyArray()
{
writeToMemory(new Object[]{}, MEMORY_POSITION);
Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_arrayWithSingleNullElement()
{
writeToMemory(new Object[]{null}, MEMORY_POSITION);
Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_makeColumnValueSelector_null()
{
writeToMemory(null, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertTrue(readSelector.isNull());
}
@Test
public void test_makeColumnValueSelector_aValue()
{
writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(LONGS_LIST_1, readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_multipleValues()
{
long sz = writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION);
writeToMemory(LONGS_ARRAY_2, MEMORY_POSITION + sz);
IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz));
final ColumnValueSelector<?> readSelector = new LongArrayFieldReader().makeColumnValueSelector(memory, pointer);
pointer.setPointer(0);
assertResults(LONGS_LIST_1, readSelector.getObject());
pointer.setPointer(1);
assertResults(LONGS_LIST_2, readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_emptyArray()
{
writeToMemory(new Object[]{}, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(Collections.emptyList(), readSelector.getObject());
}
@Test
public void test_makeColumnValueSelector_arrayWithSingleNullElement()
{
writeToMemory(new Object[]{null}, MEMORY_POSITION);
final ColumnValueSelector<?> readSelector =
new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
assertResults(Collections.singletonList(null), readSelector.getObject());
}
private long writeToMemory(final Object value, final long initialPosition)
{
Mockito.when(writeSelector.getObject()).thenReturn(value);
long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition);
if (bytesWritten < 0) {
throw new ISE("Could not write");
}
return bytesWritten;
}
private void assertResults(List<Long> expected, Object actual)
{
if (expected == null) {
Assert.assertNull(actual);
}
Assert.assertTrue(actual instanceof Object[]);
List<Long> actualList = new ArrayList<>();
for (Object val : (Object[]) actual) {
actualList.add((Long) val);
}
Assert.assertEquals(expected, actualList);
}
}

View File

@ -59,7 +59,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
public void setUp()
{
memory = WritableMemory.allocate(1000);
fieldWriter = new LongFieldWriter(writeSelector);
fieldWriter = LongFieldWriter.forPrimitive(writeSelector);
}
@After
@ -72,14 +72,14 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
public void test_isNull_defaultOrNull()
{
writeToMemory(NullHandling.defaultLongValue());
Assert.assertEquals(NullHandling.sqlCompatible(), new LongFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertEquals(NullHandling.sqlCompatible(), LongFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(5L);
Assert.assertFalse(new LongFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(LongFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION));
}
@Test
@ -88,7 +88,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(NullHandling.defaultLongValue());
final ColumnValueSelector<?> readSelector =
new LongFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull());
@ -103,7 +103,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(5L);
final ColumnValueSelector<?> readSelector =
new LongFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(5L, readSelector.getObject());
}
@ -114,7 +114,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(NullHandling.defaultLongValue());
final DimensionSelector readSelector =
new LongFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -148,7 +148,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(5L);
final DimensionSelector readSelector =
new LongFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -175,7 +175,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(25L);
final DimensionSelector readSelector =
new LongFieldReader().makeDimensionSelector(
LongFieldReader.forPrimitive().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)

View File

@ -81,24 +81,24 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
public void test_isNull_nullValue()
{
writeToMemory(Collections.singletonList(null));
Assert.assertTrue(new StringFieldReader(false).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertTrue(new StringFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_twoNullValues()
{
writeToMemory(Arrays.asList(null, null));
Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_nullRow()
{
writeToMemory(null);
Assert.assertTrue(new StringFieldReader(false).isNull(memory, MEMORY_POSITION));
Assert.assertTrue(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertTrue(new StringFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertTrue(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
@ -107,33 +107,33 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(Collections.singletonList(""));
Assert.assertEquals(
NullHandling.replaceWithDefault(),
new StringFieldReader(false).isNull(memory, MEMORY_POSITION)
new StringFieldReader().isNull(memory, MEMORY_POSITION)
);
Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_aValue()
{
writeToMemory(Collections.singletonList("foo"));
Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_multiString()
{
writeToMemory(ImmutableList.of("foo", "bar"));
Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
public void test_isNull_multiStringIncludingNulls()
{
writeToMemory(Arrays.asList(null, "bar"));
Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION));
Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION));
}
@Test
@ -142,9 +142,9 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(Collections.singletonList("foo"));
final ColumnValueSelector<?> readSelector =
new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
final ColumnValueSelector<?> readSelectorAsArray =
new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals("foo", readSelector.getObject());
Assert.assertArrayEquals(new Object[]{"foo"}, (Object[]) readSelectorAsArray.getObject());
@ -156,9 +156,9 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(ImmutableList.of("foo", "bar"));
final ColumnValueSelector<?> readSelector =
new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
final ColumnValueSelector<?> readSelectorAsArray =
new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertEquals(ImmutableList.of("foo", "bar"), readSelector.getObject());
Assert.assertArrayEquals(new Object[]{"foo", "bar"}, (Object[]) readSelectorAsArray.getObject());
@ -170,9 +170,9 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(Collections.singletonList(null));
final ColumnValueSelector<?> readSelector =
new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
final ColumnValueSelector<?> readSelectorAsArray =
new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertNull(readSelector.getObject());
Assert.assertArrayEquals(new Object[]{null}, (Object[]) readSelectorAsArray.getObject());
@ -184,9 +184,9 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(Collections.emptyList());
final ColumnValueSelector<?> readSelector =
new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
final ColumnValueSelector<?> readSelectorAsArray =
new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION));
Assert.assertNull(readSelector.getObject());
Assert.assertArrayEquals(ObjectArrays.EMPTY_ARRAY, (Object[]) readSelectorAsArray.getObject());
@ -199,7 +199,7 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
final IllegalStateException e = Assert.assertThrows(
IllegalStateException.class,
() -> new StringFieldReader(true).makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null)
() -> new StringArrayFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null)
);
MatcherAssert.assertThat(
@ -214,7 +214,7 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(ImmutableList.of("foo", "bar"));
final DimensionSelector readSelector =
new StringFieldReader(false).makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
new StringFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null);
// Data retrieval tests.
final IndexedInts row = readSelector.getRow();
@ -244,7 +244,7 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
writeToMemory(ImmutableList.of("foo", "bar"));
final DimensionSelector readSelector =
new StringFieldReader(false).makeDimensionSelector(
new StringFieldReader().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)

View File

@ -0,0 +1,187 @@
/*
* 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.frame.field;
import com.google.common.collect.ImmutableList;
import org.apache.datasketches.memory.WritableMemory;
import org.junit.Assert;
import org.junit.Test;
import java.util.List;
public class TransformUtilsTest
{
private final WritableMemory lhsMemory = WritableMemory.allocate(10);
private final WritableMemory rhsMemory = WritableMemory.allocate(10);
private static final long MEMORY_LOCATION = 0;
@Test
public void doubleTestWithoutNaN()
{
//CHECKSTYLE.OFF: Regexp
List<Double> values =
ImmutableList.of(
Double.MAX_VALUE,
Double.MIN_VALUE,
Double.POSITIVE_INFINITY,
Double.NEGATIVE_INFINITY,
Double.MIN_NORMAL,
0.0d,
1.234234d,
-1.344234d,
129123.123123,
-21312213.33,
1111.0,
23.0
);
//CHECKSTYLE.ON: Regexp
for (double value : values) {
Assert.assertEquals(
value,
TransformUtils.detransformToDouble(TransformUtils.transformFromDouble(value)),
0.0
);
}
for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) {
for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) {
double lhs = values.get(lhsIndex);
double rhs = values.get(rhsIndex);
lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromDouble(lhs));
rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromDouble(rhs));
int byteCmp = byteComparison(Double.BYTES);
if (byteCmp < 0) {
Assert.assertTrue(lhs < rhs);
} else if (byteCmp == 0) {
Assert.assertEquals(lhs, rhs, 0.0);
} else {
Assert.assertTrue(lhs > rhs);
}
}
}
}
@Test
public void longTest()
{
List<Long> values =
ImmutableList.of(
Long.MIN_VALUE,
Long.MAX_VALUE,
0L,
123L,
-123L
);
for (long value : values) {
Assert.assertEquals(
value,
TransformUtils.detransformToLong(TransformUtils.transformFromLong(value))
);
}
for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) {
for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) {
long lhs = values.get(lhsIndex);
long rhs = values.get(rhsIndex);
lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromLong(lhs));
rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromLong(rhs));
int byteCmp = byteComparison(Long.BYTES);
if (byteCmp < 0) {
Assert.assertTrue(lhs < rhs);
} else if (byteCmp == 0) {
Assert.assertEquals(lhs, rhs);
} else {
Assert.assertTrue(lhs > rhs);
}
}
}
}
@Test
public void floatTestWithoutNaN()
{
//CHECKSTYLE.OFF: Regexp
List<Float> values =
ImmutableList.of(
Float.MIN_VALUE,
Float.MAX_VALUE,
Float.MIN_NORMAL,
Float.POSITIVE_INFINITY,
Float.NEGATIVE_INFINITY,
0.0f,
1.234234f,
-1.344234f,
129123.123123f,
-21312213.33f,
1111.0f,
23.0f
);
//CHECKSTYLE.ON: Regexp
for (float value : values) {
Assert.assertEquals(
value,
TransformUtils.detransformToFloat(TransformUtils.transformFromFloat(value)),
0.0
);
}
for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) {
for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) {
float lhs = values.get(lhsIndex);
float rhs = values.get(rhsIndex);
lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromFloat(lhs));
rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromFloat(rhs));
int byteCmp = byteComparison(Long.BYTES);
if (byteCmp < 0) {
Assert.assertTrue(lhs < rhs);
} else if (byteCmp == 0) {
Assert.assertEquals(lhs, rhs, 0.0);
} else {
Assert.assertTrue(lhs > rhs);
}
}
}
}
private int byteComparison(int numBytes)
{
for (int i = 0; i < numBytes; ++i) {
byte lhsByte = lhsMemory.getByte(MEMORY_LOCATION + i);
byte rhsByte = rhsMemory.getByte(MEMORY_LOCATION + i);
final int cmp = (lhsByte & 0xFF) - (rhsByte & 0xFF);
if (cmp != 0) {
return cmp;
}
}
return 0;
}
}

View File

@ -153,6 +153,33 @@ public class FrameWriterTest extends InitializedNullHandlingTest
testWithDataset(FrameWriterTestData.TEST_LONGS);
}
@Test
public void test_arrayLong()
{
// ARRAY<LONG> can't be read or written for columnar frames, therefore skip the check if it encounters those
// parameters
Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR);
testWithDataset(FrameWriterTestData.TEST_ARRAYS_LONG);
}
@Test
public void test_arrayFloat()
{
// ARRAY<FLOAT> can't be read or written for columnar frames, therefore skip the check if it encounters those
// parameters
Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR);
testWithDataset(FrameWriterTestData.TEST_ARRAYS_FLOAT);
}
@Test
public void test_arrayDouble()
{
// ARRAY<DOUBLE> can't be read or written for columnar frames, therefore skip the check if it encounters those
// parameters
Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR);
testWithDataset(FrameWriterTestData.TEST_ARRAYS_DOUBLE);
}
@Test
public void test_float()
{
@ -226,6 +253,14 @@ public class FrameWriterTest extends InitializedNullHandlingTest
// Test all possible arrangements of two different types.
for (final FrameWriterTestData.Dataset<?> dataset1 : FrameWriterTestData.DATASETS) {
for (final FrameWriterTestData.Dataset<?> dataset2 : FrameWriterTestData.DATASETS) {
if (dataset1.getType().isArray() && dataset1.getType().getElementType().isNumeric()
|| dataset2.getType().isArray() && dataset2.getType().getElementType().isNumeric()) {
if (inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR) {
// Skip the check if any of the dataset is a numerical array and any of the input or the output frame type
// is COLUMNAR.
continue;
}
}
final RowSignature signature = makeSignature(Arrays.asList(dataset1, dataset2));
final Sequence<List<Object>> rowSequence = unsortAndMakeRows(Arrays.asList(dataset1, dataset2));
@ -265,6 +300,7 @@ public class FrameWriterTest extends InitializedNullHandlingTest
public void test_insufficientWriteCapacity()
{
// Test every possible capacity, up to the amount required to write all items from every list.
Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR);
final RowSignature signature = makeSignature(FrameWriterTestData.DATASETS);
final Sequence<List<Object>> rowSequence = unsortAndMakeRows(FrameWriterTestData.DATASETS);
final int totalRows = rowSequence.toList().size();

View File

@ -136,6 +136,24 @@ public class FrameWriterTestData
).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList())
);
public static final Dataset<Object> TEST_ARRAYS_LONG = new Dataset<>(
ColumnType.LONG_ARRAY,
Arrays.asList(
null,
ObjectArrays.EMPTY_ARRAY,
new Object[]{null},
new Object[]{null, 6L, null, 5L, null},
new Object[]{null, 6L, null, 5L, NullHandling.defaultLongValue()},
new Object[]{null, 6L, null, 5L, 0L, -1L},
new Object[]{null, 6L, null, 5L, 0L, -1L, Long.MIN_VALUE},
new Object[]{null, 6L, null, 5L, 0L, -1L, Long.MAX_VALUE},
new Object[]{5L},
new Object[]{5L, 6L},
new Object[]{5L, 6L, null},
new Object[]{Long.MAX_VALUE, Long.MIN_VALUE}
)
);
public static final Dataset<Float> TEST_FLOATS = new Dataset<>(
ColumnType.FLOAT,
Stream.of(
@ -158,6 +176,28 @@ public class FrameWriterTestData
).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList())
);
//CHECKSTYLE.OFF: Regexp
public static final Dataset<Object> TEST_ARRAYS_FLOAT = new Dataset<>(
ColumnType.FLOAT_ARRAY,
Arrays.asList(
null,
ObjectArrays.EMPTY_ARRAY,
new Object[]{null},
new Object[]{null, 6.2f, null, 5.1f, null},
new Object[]{null, 6.2f, null, 5.1f, NullHandling.defaultFloatValue()},
new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f},
new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f, Float.MIN_VALUE},
new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f, Float.MAX_VALUE},
new Object[]{Float.NEGATIVE_INFINITY, Float.MIN_VALUE},
new Object[]{5.7f},
new Object[]{5.7f, 6.2f},
new Object[]{5.7f, 6.2f, null},
new Object[]{Float.MAX_VALUE, Float.MIN_VALUE},
new Object[]{Float.POSITIVE_INFINITY, Float.MIN_VALUE}
)
);
//CHECKSTYLE.ON: Regexp
public static final Dataset<Double> TEST_DOUBLES = new Dataset<>(
ColumnType.DOUBLE,
Stream.of(
@ -180,6 +220,28 @@ public class FrameWriterTestData
).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList())
);
//CHECKSTYLE.OFF: Regexp
public static final Dataset<Object> TEST_ARRAYS_DOUBLE = new Dataset<>(
ColumnType.DOUBLE_ARRAY,
Arrays.asList(
null,
ObjectArrays.EMPTY_ARRAY,
new Object[]{null},
new Object[]{null, 6.2d, null, 5.1d, null},
new Object[]{null, 6.2d, null, 5.1d, NullHandling.defaultDoubleValue()},
new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d},
new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d, Double.MIN_VALUE},
new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d, Double.MAX_VALUE},
new Object[]{Double.NEGATIVE_INFINITY, Double.MIN_VALUE},
new Object[]{5.7d},
new Object[]{5.7d, 6.2d},
new Object[]{5.7d, 6.2d, null},
new Object[]{Double.MAX_VALUE, Double.MIN_VALUE},
new Object[]{Double.POSITIVE_INFINITY, Double.MIN_VALUE}
)
);
//CHECKSTYLE.ON: Regexp
public static final Dataset<HyperLogLogCollector> TEST_COMPLEX = new Dataset<>(
HyperUniquesAggregatorFactory.TYPE,
Arrays.asList(
@ -200,6 +262,9 @@ public class FrameWriterTestData
.add(TEST_STRINGS_SINGLE_VALUE)
.add(TEST_STRINGS_MULTI_VALUE)
.add(TEST_ARRAYS_STRING)
.add(TEST_ARRAYS_LONG)
.add(TEST_ARRAYS_FLOAT)
.add(TEST_ARRAYS_DOUBLE)
.add(TEST_COMPLEX)
.build();

View File

@ -75,42 +75,6 @@ public class FrameWritersTest extends InitializedNullHandlingTest
Assert.assertEquals(ALLOCATOR_CAPACITY, factory.allocatorCapacity());
}
@Test
public void test_rowBased_unsupportedSortingColumnType()
{
final UnsupportedColumnTypeException e = Assert.assertThrows(
UnsupportedColumnTypeException.class,
() -> FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY),
RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(),
Collections.singletonList(new KeyColumn("x", KeyOrder.ASCENDING))
)
);
Assert.assertEquals("x", e.getColumnName());
Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType());
}
@Test
public void test_rowBased_unsupportedNonSortingColumnType()
{
final FrameWriterFactory factory = FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY),
RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(),
Collections.emptyList()
);
final UnsupportedColumnTypeException e = Assert.assertThrows(
UnsupportedColumnTypeException.class,
() -> factory.newFrameWriter(new AllNullColumnSelectorFactory())
);
Assert.assertEquals("x", e.getColumnName());
Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType());
}
@Test
public void test_columnar_unsupportedColumnType()
{

View File

@ -1170,7 +1170,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase
skipVectorize = true;
}
protected void notMsqCompatible()
protected void msqIncompatible()
{
msqCompatible = false;
}

View File

@ -87,7 +87,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testSelectConstantArrayExpressionFromTable()
{
notMsqCompatible();
testQuery(
"SELECT ARRAY[1,2] as arr, dim1 FROM foo LIMIT 1",
ImmutableList.of(
@ -169,7 +168,8 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testSelectNonConstantArrayExpressionFromTableForMultival()
{
notMsqCompatible();
// Produces nested string array, that MSQ can't infer from the selector
msqIncompatible();
final String sql = "SELECT ARRAY[CONCAT(dim3, 'word'),'up'] as arr, dim1 FROM foo LIMIT 5";
final Query<?> scanQuery = newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
@ -209,7 +209,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
// Yes these outputs are strange sometimes, arrays are in a partial state of existence so end up a bit
// stringy for now this is because virtual column selectors are coercing values back to stringish so that
// multi-valued string dimensions can be grouped on.
notMsqCompatible();
List<Object[]> expectedResults;
if (useDefault) {
expectedResults = ImmutableList.of(
@ -389,7 +388,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
// which will still always be stringified to ultimately adhere to the varchar type
// as array support increases in the engine this will likely change since using explict array functions should
// probably kick it into an array
notMsqCompatible();
List<Object[]> expectedResults;
if (useDefault) {
expectedResults = ImmutableList.of(
@ -1021,7 +1019,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayGroupAsLongArray()
{
notMsqCompatible();
// Cannot vectorize as we donot have support in native query subsytem for grouping on arrays
cannotVectorize();
testQuery(
@ -1073,7 +1070,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
{
// Cannot vectorize as we donot have support in native query subsytem for grouping on arrays as keys
cannotVectorize();
notMsqCompatible();
testQuery(
"SELECT ARRAY[d1], SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC",
QUERY_CONTEXT_NO_STRINGIFY_ARRAY,
@ -1121,7 +1117,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayGroupAsFloatArray()
{
notMsqCompatible();
// Cannot vectorize as we donot have support in native query subsytem for grouping on arrays as keys
cannotVectorize();
testQuery(
@ -1612,7 +1607,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayAggNumeric()
{
notMsqCompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_AGG(l1), ARRAY_AGG(DISTINCT l1), ARRAY_AGG(d1), ARRAY_AGG(DISTINCT d1), ARRAY_AGG(f1), ARRAY_AGG(DISTINCT f1) FROM numfoo",
@ -1749,7 +1743,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayAggQuantile()
{
notMsqCompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_QUANTILE(ARRAY_AGG(l1), 0.9) FROM numfoo",
@ -1793,7 +1786,9 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayAggArrays()
{
notMsqCompatible();
// Produces nested array - ARRAY<ARRAY<LONG>>, which frame writers don't support. A way to get this query
// to run would be to use nested columns.
msqIncompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_AGG(ARRAY[l1, l2]), ARRAY_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo",
@ -1890,7 +1885,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayConcatAggArrays()
{
notMsqCompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_CONCAT_AGG(ARRAY[l1, l2]), ARRAY_CONCAT_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo",
@ -2039,7 +2033,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
public void testArrayAggMaxBytes()
{
cannotVectorize();
notMsqCompatible();
testQuery(
"SELECT ARRAY_AGG(l1, 128), ARRAY_AGG(DISTINCT l1, 128) FROM numfoo",
ImmutableList.of(
@ -2239,7 +2232,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayAggGroupByArrayAggOfLongsFromSubquery()
{
notMsqCompatible();
requireMergeBuffers(3);
cannotVectorize();
testQuery(
@ -2379,7 +2371,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayAggGroupByArrayAggOfDoubleFromSubquery()
{
notMsqCompatible();
requireMergeBuffers(3);
cannotVectorize();
testQuery(
@ -2897,7 +2888,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestThriceWithFiltersOnDimAndUnnestCol()
{
notMsqCompatible();
cannotVectorize();
String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n"
+ " ( SELECT * FROM \n"
@ -2996,7 +2986,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestThriceWithFiltersOnDimAndAllUnnestColumns()
{
notMsqCompatible();
cannotVectorize();
String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n"
+ " ( SELECT * FROM \n"

View File

@ -695,7 +695,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
// Like "testFilterAndGroupByLookupUsingJoinOperator", but with the table and lookup reversed.
// MSQ refuses to do RIGHT join with broadcast.
notMsqCompatible();
msqIncompatible();
// Cannot vectorize JOIN operator.
cannotVectorize();
@ -784,7 +784,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testJoinUnionTablesOnLookup(Map<String, Object> queryContext)
{
// MSQ does not support UNION ALL.
notMsqCompatible();
msqIncompatible();
// Cannot vectorize JOIN operator.
cannotVectorize();
@ -1918,7 +1918,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
// Regression test for https://github.com/apache/druid/issues/9646.
// Empty-dataset aggregation queries in MSQ return an empty row, rather than a single row as SQL requires.
notMsqCompatible();
msqIncompatible();
// Cannot vectorize JOIN operator.
cannotVectorize();
@ -1974,7 +1974,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map<String, Object> queryContext)
{
// Empty-dataset aggregation queries in MSQ return an empty row, rather than a single row as SQL requires.
notMsqCompatible();
msqIncompatible();
// Cannot vectorize JOIN operator.
cannotVectorize();
@ -2396,7 +2396,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testSelectOnLookupUsingRightJoinOperator(Map<String, Object> queryContext)
{
// MSQ refuses to do RIGHT join with broadcast.
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT dim1, lookyloo.*\n"
@ -2438,7 +2438,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testSelectOnLookupUsingFullJoinOperator(Map<String, Object> queryContext)
{
// MSQ refuses to do FULL join with broadcast.
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT dim1, m1, cnt, lookyloo.*\n"
@ -3751,7 +3751,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
{
if (sortBasedJoin) {
// Cannot handle the [l1.k = 'abc'] condition.
notMsqCompatible();
msqIncompatible();
}
// Cannot vectorize due to 'concat' expression.
@ -4172,7 +4172,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testJoinOnMultiValuedColumnShouldThrowException(Map<String, Object> queryContext)
{
// MSQ throws a slightly different error than QueryException.
notMsqCompatible();
msqIncompatible();
final String query = "SELECT dim3, l.v from druid.foo f inner join lookup.lookyloo l on f.dim3 = l.k\n";
@ -4189,7 +4189,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testUnionAllTwoQueriesLeftQueryIsJoin(Map<String, Object> queryContext)
{
// MSQ does not support UNION ALL.
notMsqCompatible();
msqIncompatible();
// Fully removing the join allows this query to vectorize.
if (!isRewriteJoinToFilter(queryContext)) {
@ -4233,7 +4233,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testUnionAllTwoQueriesRightQueryIsJoin(Map<String, Object> queryContext)
{
// MSQ does not support UNION ALL.
notMsqCompatible();
msqIncompatible();
// Fully removing the join allows this query to vectorize.
if (!isRewriteJoinToFilter(queryContext)) {
@ -4276,7 +4276,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
public void testUnionAllTwoQueriesBothQueriesAreJoin()
{
// MSQ does not support UNION ALL.
notMsqCompatible();
msqIncompatible();
cannotVectorize();
testQuery(
@ -4982,7 +4982,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
// IndexedTableDimensionSelector. In native, this doesn't happen, because we don't have as much type information,
// and we end up using IndexedTableColumnValueSelector instead. This is really a problem with
// IndexedTableColumnSelectorFactory: it assumes strings are not multi-valued, even though they might be.
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 "
@ -5038,7 +5038,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
// IndexedTableDimensionSelector. In native, this doesn't happen, because we don't have as much type information,
// and we end up using IndexedTableColumnValueSelector instead. This is really a problem with
// IndexedTableColumnSelectorFactory: it assumes strings are not multi-valued, even though they might be.
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN "

View File

@ -141,7 +141,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInformationSchemaSchemata()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT DISTINCT SCHEMA_NAME FROM INFORMATION_SCHEMA.SCHEMATA",
ImmutableList.of(),
@ -158,7 +158,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInformationSchemaTables()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, IS_JOINABLE, IS_BROADCAST\n"
+ "FROM INFORMATION_SCHEMA.TABLES\n"
@ -237,7 +237,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInformationSchemaColumnsOnTable()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n"
+ "FROM INFORMATION_SCHEMA.COLUMNS\n"
@ -259,7 +259,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInformationSchemaColumnsOnForbiddenTable()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n"
+ "FROM INFORMATION_SCHEMA.COLUMNS\n"
@ -290,7 +290,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInformationSchemaColumnsOnView()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n"
+ "FROM INFORMATION_SCHEMA.COLUMNS\n"
@ -305,7 +305,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInformationSchemaColumnsOnAnotherView()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n"
+ "FROM INFORMATION_SCHEMA.COLUMNS\n"
@ -322,7 +322,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCannotInsertWithNativeEngine()
{
notMsqCompatible();
msqIncompatible();
final DruidException e = Assert.assertThrows(
DruidException.class,
() -> testQuery(
@ -341,7 +341,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCannotReplaceWithNativeEngine()
{
notMsqCompatible();
msqIncompatible();
final DruidException e = Assert.assertThrows(
DruidException.class,
() -> testQuery(
@ -360,7 +360,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testAggregatorsOnInformationSchemaColumns()
{
notMsqCompatible();
msqIncompatible();
// Not including COUNT DISTINCT, since it isn't supported by BindableAggregate, and so it can't work.
testQuery(
"SELECT\n"
@ -381,7 +381,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testFilterAggregatorFunctionsOnInformationSchemaRoutines()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ " COUNT(*)\n"
@ -397,7 +397,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testFilterScalarFunctionsOnInformationSchemaRoutines()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ " COUNT(*)\n"
@ -413,7 +413,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testNonExistentSchemaOnInformationSchemaRoutines()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT *\n"
+ "FROM INFORMATION_SCHEMA.ROUTINES\n"
@ -426,7 +426,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTopNLimitWrapping()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@ -631,7 +631,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testEarliestAggregators()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT "
@ -678,7 +678,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testLatestVectorAggregators()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT "
+ "LATEST(cnt), LATEST(cnt + 1), LATEST(m1), LATEST(m1+1) "
@ -712,7 +712,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testLatestAggregators()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT "
@ -857,7 +857,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testLatestVectorAggregatorsOnTimeExpression()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT \n"
+ " LATEST_BY(m1, MILLIS_TO_TIMESTAMP(BITWISE_SHIFT_RIGHT(TIMESTAMP_TO_MILLIS(__time), 3)))\n"
@ -928,7 +928,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testPrimitiveLatestInSubquery()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT SUM(val1), SUM(val2), SUM(val3) FROM (SELECT dim2, LATEST(m1) AS val1, LATEST(cnt) AS val2, LATEST(m2) AS val3 FROM foo GROUP BY dim2)",
ImmutableList.of(
@ -976,7 +976,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testPrimitiveLatestInSubqueryGroupBy()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT dim2, LATEST(m1) AS val1 FROM foo GROUP BY dim2",
ImmutableList.of(
@ -1095,7 +1095,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testPrimitiveEarliestInSubquery()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT SUM(val1), SUM(val2), SUM(val3) FROM (SELECT dim2, EARLIEST(m1) AS val1, EARLIEST(cnt) AS val2, EARLIEST(m2) AS val3 FROM foo GROUP BY dim2)",
@ -1303,7 +1303,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testStringEarliestSingleStringDim()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT dim2, EARLIEST(dim1,10) AS val FROM foo GROUP BY dim2",
ImmutableList.of(
@ -1419,7 +1419,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testEarliestAggregatorsNumericNulls()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT EARLIEST(l1), EARLIEST(d1), EARLIEST(f1) FROM druid.numfoo",
@ -1447,7 +1447,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testLatestAggregatorsNumericNull()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT LATEST(l1), LATEST(d1), LATEST(f1) FROM druid.numfoo",
ImmutableList.of(
@ -1478,7 +1478,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testFirstLatestAggregatorsSkipNulls()
{
notMsqCompatible();
msqIncompatible();
final DimFilter filter;
if (useDefault) {
@ -1592,7 +1592,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testOrderByEarliestFloat()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
@ -1639,7 +1639,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testOrderByEarliestDouble()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
@ -1686,7 +1686,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testOrderByEarliestLong()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
@ -1733,7 +1733,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testOrderByLatestFloat()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@ -1780,7 +1780,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testOrderByLatestDouble()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@ -1826,7 +1826,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testOrderByLatestLong()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@ -1884,7 +1884,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
} else {
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
notMsqCompatible();
msqIncompatible();
expected = ImmutableList.of(
new Object[]{"2", 0.0f},
@ -1935,7 +1935,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
} else {
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
notMsqCompatible();
msqIncompatible();
expected = ImmutableList.of(
new Object[]{"2", 0.0},
@ -1985,7 +1985,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
} else {
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
notMsqCompatible();
msqIncompatible();
expected = ImmutableList.of(
new Object[]{"2", 0L},
@ -2066,7 +2066,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Ignore("Disabled since GROUP BY alias can confuse the validator; see DruidConformance::isGroupByAlias")
public void testGroupByAndOrderByAlias()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT cnt AS theCnt, COUNT(*) FROM druid.foo GROUP BY theCnt ORDER BY theCnt ASC",
ImmutableList.of(
@ -2449,7 +2449,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testExactCountDistinctWithFilter()
{
notMsqCompatible();
msqIncompatible();
final String sqlQuery = "SELECT COUNT(DISTINCT foo.dim1) FILTER(WHERE foo.cnt = 1), SUM(foo.cnt) FROM druid.foo";
// When useApproximateCountDistinct=false and useGroupingSetForExactDistinct=false, planning fails due
// to a bug in the Calcite's rule (AggregateExpandDistinctAggregatesRule)
@ -2814,7 +2814,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllQueries()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo",
ImmutableList.of(
@ -2848,7 +2848,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllQueriesWithLimit()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT * FROM ("
+ "SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo"
@ -2877,7 +2877,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllDifferentTablesWithMapping()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@ -2921,7 +2921,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testJoinUnionAllDifferentTablesWithMapping()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@ -2985,7 +2985,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllTablesColumnTypeMismatchFloatLong()
{
notMsqCompatible();
msqIncompatible();
// "m1" has a different type in foo and foo2 (float vs long), but this query is OK anyway because they can both
// be implicitly cast to double.
@ -3095,7 +3095,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllSameTableTwice()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@ -3139,7 +3139,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllSameTableTwiceWithSameMapping()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@ -3197,7 +3197,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllSameTableThreeTimes()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@ -3302,7 +3302,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnionAllSameTableThreeTimesWithSameMapping()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@ -3673,7 +3673,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
if (NullHandling.sqlCompatible()) {
notMsqCompatible();
msqIncompatible();
}
List<Object[]> expected;
if (useDefault) {
@ -3717,7 +3717,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
if (NullHandling.sqlCompatible()) {
notMsqCompatible();
msqIncompatible();
}
List<Object[]> expected;
if (useDefault) {
@ -3761,7 +3761,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
if (NullHandling.sqlCompatible()) {
notMsqCompatible();
msqIncompatible();
}
List<Object[]> expected;
if (useDefault) {
@ -4199,7 +4199,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingWithNullInFilter()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL)",
ImmutableList.of(
@ -4224,7 +4224,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByNothingWithLiterallyFalseFilter()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE 1 = 0",
ImmutableList.of(
@ -4250,7 +4250,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByNothingWithImpossibleTimeFilter()
{
notMsqCompatible();
msqIncompatible();
// Regression test for https://github.com/apache/druid/issues/7671
testQuery(
@ -4321,7 +4321,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByWithFilterMatchingNothing()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar'",
ImmutableList.of(
@ -4669,7 +4669,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCountStarWithLongColumnFiltersOnFloatLiterals()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM druid.foo WHERE cnt > 1.1 and cnt < 100000001.0",
ImmutableList.of(
@ -5659,7 +5659,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testCountStarWithNotOfDegenerateFilter()
{
notMsqCompatible();
msqIncompatible();
// HashJoinSegmentStorageAdapter is not vectorizable
cannotVectorize();
@ -5689,7 +5689,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testUnplannableQueries()
{
notMsqCompatible();
msqIncompatible();
// All of these queries are unplannable because they rely on features Druid doesn't support.
// This test is here to confirm that we don't fall back to Calcite's interpreter or enumerable implementation.
// It's also here so when we do support these features, we can have "real" tests for these queries.
@ -5798,7 +5798,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testArrayAggQueryOnComplexDatatypes()
{
notMsqCompatible();
msqIncompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_AGG(unique_dim1) FROM druid.foo",
@ -5961,7 +5961,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCountStarWithTimeInIntervalFilterInvalidInterval()
{
notMsqCompatible();
msqIncompatible();
testQueryThrows(
"SELECT COUNT(*) FROM druid.foo "
+ "WHERE TIME_IN_INTERVAL(__time, '2000-01-01/X')",
@ -5978,7 +5978,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCountStarWithTimeInIntervalFilterNonLiteral()
{
notMsqCompatible();
msqIncompatible();
testQueryThrows(
"SELECT COUNT(*) FROM druid.foo "
+ "WHERE TIME_IN_INTERVAL(__time, dim1)",
@ -6559,7 +6559,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testCountDistinct()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT SUM(cnt), COUNT(distinct dim2), COUNT(distinct unique_dim1) FROM druid.foo",
ImmutableList.of(
@ -6684,7 +6684,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
if (NullHandling.sqlCompatible()) {
// Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950
notMsqCompatible();
msqIncompatible();
}
// When HLL is disabled, APPROX_COUNT_DISTINCT is still approximate.
@ -6722,7 +6722,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
if (NullHandling.sqlCompatible()) {
// Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950
notMsqCompatible();
msqIncompatible();
}
testQuery(
@ -6921,7 +6921,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets()
{
notMsqCompatible();
msqIncompatible();
requireMergeBuffers(4);
testQuery(
PLANNER_CONFIG_NO_HLL.withOverrides(
@ -7004,7 +7004,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testApproxCountDistinct()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -7333,7 +7333,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testExactCountDistinctUsingSubqueryOnUnionAllTables()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT\n"
+ " SUM(cnt),\n"
@ -7485,7 +7485,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
// skip in sql compatible mode, this plans to an OR filter with equality filter children...
return;
}
notMsqCompatible();
msqIncompatible();
expectedException.expect(UOE.class);
expectedException.expectMessage(
"The number of values in the IN clause for [dim6] in query exceeds configured maxNumericFilter limit of [2] for INs. Cast [3] values of IN clause to String");
@ -7746,7 +7746,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
if (NullHandling.sqlCompatible()) {
// Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950
notMsqCompatible();
msqIncompatible();
}
testQuery(
@ -8485,7 +8485,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testFilterOnTimeExtractWithVariousTimeUnits()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -8548,7 +8548,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testFilterOnTimeFloorMisaligned()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM druid.foo "
+ "WHERE floor(__time TO month) = TIMESTAMP '2000-01-01 00:00:01'",
@ -8595,7 +8595,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename()
{
notMsqCompatible();
msqIncompatible();
cannotVectorize();
skipVectorize();
requireMergeBuffers(3);
@ -9068,7 +9068,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByExpressionFromLookup()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize direct queries on lookup tables.
cannotVectorize();
@ -9454,7 +9454,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeseriesDontSkipEmptyBuckets()
{
notMsqCompatible();
msqIncompatible();
// Tests that query context parameters are passed through to the underlying query engine.
Long defaultVal = NullHandling.replaceWithDefault() ? 0L : null;
testQuery(
@ -9594,7 +9594,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeseriesEmptyResultsAggregatorDefaultValues()
{
notMsqCompatible();
msqIncompatible();
// timeseries with all granularity have a single group, so should return default results for given aggregators
testQuery(
"SELECT\n"
@ -9700,7 +9700,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized()
{
notMsqCompatible();
msqIncompatible();
cannotVectorize();
skipVectorize();
// timeseries with all granularity have a single group, so should return default results for given aggregators
@ -10016,7 +10016,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByAggregatorDefaultValuesNonVectorized()
{
notMsqCompatible();
msqIncompatible();
cannotVectorize();
skipVectorize();
testQuery(
@ -10641,7 +10641,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSets()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -10706,7 +10706,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingAggregatorDifferentOrder()
{
notMsqCompatible();
msqIncompatible();
requireMergeBuffers(3);
testQuery(
@ -10770,7 +10770,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingAggregatorWithPostAggregator()
{
notMsqCompatible();
msqIncompatible();
List<Object[]> resultList;
if (NullHandling.sqlCompatible()) {
resultList = ImmutableList.of(
@ -10829,7 +10829,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithNumericDimension()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT cnt, COUNT(*)\n"
+ "FROM foo\n"
@ -10860,7 +10860,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByRollup()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -10919,7 +10919,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByRollupDifferentOrder()
{
notMsqCompatible();
msqIncompatible();
// Like "testGroupByRollup", but the ROLLUP exprs are in the reverse order.
testQuery(
"SELECT dim2, gran, SUM(cnt)\n"
@ -10975,7 +10975,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupByCube()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -11037,7 +11037,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithDummyDimension()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -11099,7 +11099,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsNoSuperset()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -11156,7 +11156,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithOrderByDimension()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT dim2, gran, SUM(cnt)\n"
+ "FROM (SELECT FLOOR(__time TO MONTH) AS gran, COALESCE(dim2, '') dim2, cnt FROM druid.foo) AS x\n"
@ -11227,7 +11227,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithOrderByAggregator()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -11296,7 +11296,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithOrderByAggregatorWithLimit()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -11562,7 +11562,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
+ " org.apache.calcite.sql.validate.SqlValidatorException: Column 'dim1' is ambiguous")
public void testProjectAfterSort3()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"select dim1 from (select dim1, dim1, count(*) cnt from druid.foo group by dim1, dim1 order by cnt)",
ImmutableList.of(
@ -11601,7 +11601,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testProjectAfterSort3WithoutAmbiguity()
{
notMsqCompatible();
msqIncompatible();
// This query is equivalent to the one in testProjectAfterSort3 but renames the second grouping column
// to avoid the ambiguous name exception. The inner sort is also optimized out in Calcite 1.21.
testQuery(
@ -12150,7 +12150,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testRequireTimeConditionSimpleQueryNegative()
{
notMsqCompatible();
msqIncompatible();
expectedException.expect(CannotBuildQueryException.class);
expectedException.expectMessage("__time column");
@ -12171,7 +12171,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testRequireTimeConditionSubQueryNegative()
{
notMsqCompatible();
msqIncompatible();
expectedException.expect(CannotBuildQueryException.class);
expectedException.expectMessage("__time column");
@ -12191,7 +12191,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testRequireTimeConditionSemiJoinNegative()
{
notMsqCompatible();
msqIncompatible();
expectedException.expect(CannotBuildQueryException.class);
expectedException.expectMessage("__time column");
@ -12277,7 +12277,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTrigonometricFunction()
{
notMsqCompatible();
msqIncompatible();
testQuery(
PLANNER_CONFIG_DEFAULT,
QUERY_CONTEXT_DEFAULT,
@ -12576,7 +12576,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testQueryContextOuterLimit()
{
notMsqCompatible();
msqIncompatible();
Map<String, Object> outerLimitContext = new HashMap<>(QUERY_CONTEXT_DEFAULT);
outerLimitContext.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 4);
@ -12887,7 +12887,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithLimit()
{
notMsqCompatible();
msqIncompatible();
testQuery(
"SELECT dim2, gran, SUM(cnt)\n"
+ "FROM (SELECT FLOOR(__time TO MONTH) AS gran, COALESCE(dim2, '') dim2, cnt FROM druid.foo) AS x\n"
@ -12952,7 +12952,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGroupingSetsWithLimitOrderByGran()
{
notMsqCompatible();
msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@ -14276,7 +14276,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testGreatestFunctionForStringWithIsNull()
{
notMsqCompatible();
msqIncompatible();
cannotVectorize();
String query = "SELECT l1, LATEST(GREATEST(dim1, dim2)) IS NULL FROM druid.numfoo GROUP BY l1";
@ -14367,7 +14367,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testTimeseriesQueryWithEmptyInlineDatasourceAndGranularity()
{
// TODO(gianm): this test does not actually test the below thing, b/c the timestamp_floor got baked in
notMsqCompatible();
msqIncompatible();
//msqCompatible();
// the SQL query contains an always FALSE filter ('bar' = 'baz'), which optimizes the query to also remove time
@ -14457,7 +14457,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testComplexDecodeAgg()
{
notMsqCompatible();
msqIncompatible();
cannotVectorize();
testQuery(
"SELECT APPROX_COUNT_DISTINCT_BUILTIN(COMPLEX_DECODE_BASE64('hyperUnique',PARSE_JSON(TO_JSON_STRING(unique_dim1)))) from druid.foo",

View File

@ -321,10 +321,15 @@ public class QueryTestRunner
public static class VerifyResults implements QueryVerifyStep
{
protected final BaseExecuteQuery execStep;
protected final boolean verifyRowSignature;
public VerifyResults(BaseExecuteQuery execStep)
public VerifyResults(
BaseExecuteQuery execStep,
boolean verifyRowSignature
)
{
this.execStep = execStep;
this.verifyRowSignature = verifyRowSignature;
}
@Override
@ -346,7 +351,9 @@ public class QueryTestRunner
}
QueryTestBuilder builder = execStep.builder();
builder.expectedResultsVerifier.verifyRowSignature(queryResults.signature);
if (verifyRowSignature) {
builder.expectedResultsVerifier.verifyRowSignature(queryResults.signature);
}
builder.expectedResultsVerifier.verify(builder.sql, results);
}
}
@ -667,7 +674,9 @@ public class QueryTestRunner
verifySteps.add(new VerifyNativeQueries(finalExecStep));
}
if (builder.expectedResultsVerifier != null) {
verifySteps.add(new VerifyResults(finalExecStep));
// Don't verify the row signature when MSQ is running, since the broker receives the task id, and the signature
// would be {TASK:STRING} instead of the expected results signature
verifySteps.add(new VerifyResults(finalExecStep, !config.isRunningMSQ()));
}
if (!builder.customVerifications.isEmpty()) {