where X is a numeric type to row based frames.
+ * The format of the array written is as follows:
+ *
+ * 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
+ *
+ * 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
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * Examples:
+ * 1. null
+ * | Bytes | Value | Interpretation |
+ * |--------|-------|-----------------------------|
+ * | 1 | 0x00 | Denotes that the array null |
+ *
+ * 2. [] (empty array)
+ * | Bytes | Value | Interpretation |
+ * |--------|----- -|------------------------------------|
+ * | 1 | 0x01 | Denotes that the array is not null |
+ * | 2 | 0x00 | End of the array |
+ *
+ * 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
+ */
+ public static NumericArrayFieldWriter getLongArrayFieldWriter(final ColumnValueSelector selector)
+ {
+ return new NumericArrayFieldWriter(selector, LongFieldWriter::forArray);
+ }
+
+ /**
+ * Returns the writer for ARRAY
+ */
+ public static NumericArrayFieldWriter getFloatArrayFieldWriter(final ColumnValueSelector selector)
+ {
+ return new NumericArrayFieldWriter(selector, FloatFieldWriter::forArray);
+ }
+
+ /**
+ * Returns the writer for ARRAY
+ */
+ 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 columnValueSelector = new ColumnValueSelector()
+ {
+ @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
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java
new file mode 100644
index 000000000000..cf83c405d092
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java
@@ -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;
+ }
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java
new file mode 100644
index 000000000000..7aa604480c91
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java
@@ -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). 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);
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java
new file mode 100644
index 000000000000..a7ae47c91f4e
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java
@@ -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 selector);
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java
new file mode 100644
index 000000000000..d26f84f251de
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java
@@ -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;
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java
new file mode 100644
index 000000000000..67bed4cb051b
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java
@@ -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);
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java
index 0cce325d2123..6b15804d662a 100644
--- a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java
+++ b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java
@@ -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;
}
diff --git a/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java b/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java
new file mode 100644
index 000000000000..f882f0443568
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java
@@ -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);
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java
index e5cc20746b49..8ddf99325d39 100644
--- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java
+++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java
@@ -74,6 +74,7 @@ private 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 static FrameReader create(final RowSignature signature)
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 RowSignature signature()
/**
* Returns capabilities for a particular column in a particular frame.
- *
+ *
* 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 CursorFactory makeCursorFactory(final Frame frame)
/**
* Create a {@link FrameComparisonWidget} for the given frame.
- *
+ *
* Only possible for frames of type {@link org.apache.druid.frame.FrameType#ROW_BASED}. The provided
* sortColumns must be a prefix of {@link #signature()}.
*/
diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java
index 4caaeaae87b6..98218819ce13 100644
--- a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java
+++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java
@@ -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 @@ private 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 static FrameColumnReader create(final int columnNumber, final ColumnType
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);
}
}
}
diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java
new file mode 100644
index 000000000000..06f356128247
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java
@@ -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);
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java
index 577443cedd92..ac9574d7f052 100644
--- a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java
+++ b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java
@@ -33,6 +33,8 @@
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 static List getUtf8ByteBuffersFromStringSelector(
* Retrieves UTF-8 byte buffers from a {@link ColumnValueSelector}, which is expected to be the kind of
* selector you get for an {@code ARRAY} 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 static List getUtf8ByteBuffersFromStringArraySelector(
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 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.
*/
diff --git a/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java b/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java
index 604ef987394e..d79c517ae3d1 100644
--- a/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java
+++ b/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java
@@ -95,7 +95,7 @@ private static MethodHandle defineClassJava9(MethodHandles.Lookup lookup) throws
}
/**
- * "Compile" a MethodHandle that is equilavent to:
+ * "Compile" a MethodHandle that is equivalent to:
*
* Class> defineClass(Class targetClass, byte[] byteCode, String className) {
* return Unsafe.defineClass(
@@ -147,7 +147,7 @@ private static MethodHandle defineClassJava8(MethodHandles.Lookup lookup) throws
// defineClass(className, byteCode, 0, length, targetClass)
defineClass = MethodHandles.insertArguments(defineClass, 2, (int) 0);
- // JDK8 does not implement MethodHandles.arrayLength so we have to roll our own
+ // JDK8 does not implement MethodHandles.arrayLength, so we have to roll our own
MethodHandle arrayLength = lookup.findStatic(
lookup.lookupClass(),
"getArrayLength",
@@ -171,6 +171,16 @@ private static MethodHandle defineClassJava8(MethodHandles.Lookup lookup) throws
return defineClass;
}
+ /**
+ * This method is referenced in Java 8 using method handle, therefore it is not actually unused, and shouldn't be
+ * removed (till Java 8 is supported)
+ */
+ @SuppressWarnings("unused") // method is referenced and used in defineClassJava8
+ static int getArrayLength(byte[] bytes)
+ {
+ return bytes.length;
+ }
+
public static Class defineClass(
Class> targetClass,
byte[] byteCode,
diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
index c5bb271213e8..b4d06edc77cf 100644
--- a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
+++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java
@@ -20,18 +20,18 @@
package org.apache.druid.query;
import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
-import org.apache.druid.java.util.common.guava.Yielder;
-import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.RowAdapter;
import org.apache.druid.segment.RowBasedCursor;
import org.apache.druid.segment.RowWalker;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
-import java.util.Iterator;
+import java.io.Closeable;
/**
* Helper methods to create cursor from iterable of rows
@@ -43,7 +43,18 @@ public class IterableRowsCursorHelper
* Creates a cursor that iterates over all the rows generated by the iterable. Presence of __time column is not a
* necessity
*/
- public static RowBasedCursor getCursorFromIterable(Iterable rows, RowSignature rowSignature)
+ public static Pair getCursorFromIterable(Iterable rows, RowSignature rowSignature)
+ {
+ return getCursorFromSequence(Sequences.simple(rows), rowSignature);
+ }
+
+ /**
+ * Creates a cursor that iterates over all the rows generated by the sequence. Presence of __time column is not a
+ * necessity.
+ *
+ * Returns a pair of cursor that iterates over the rows and closeable that cleans up the created rowWalker
+ */
+ public static Pair getCursorFromSequence(Sequence rows, RowSignature rowSignature)
{
RowAdapter rowAdapter = columnName -> {
if (rowSignature == null) {
@@ -55,8 +66,10 @@ public static RowBasedCursor getCursorFromIterable(Iterable
}
return row -> row[columnIndex];
};
- RowWalker rowWalker = new RowWalker<>(Sequences.simple(rows), rowAdapter);
- return new RowBasedCursor<>(
+
+ RowWalker rowWalker = new RowWalker<>(rows, rowAdapter);
+
+ Cursor baseCursor = new RowBasedCursor<>(
rowWalker,
rowAdapter,
null,
@@ -66,41 +79,7 @@ public static RowBasedCursor getCursorFromIterable(Iterable
false,
rowSignature != null ? rowSignature : RowSignature.empty()
);
- }
- /**
- * Creates a cursor that iterates over all the rows generated by the sequence. Presence of __time column is not a
- * necessity
- */
- public static RowBasedCursor getCursorFromSequence(Sequence rows, RowSignature rowSignature)
- {
- return getCursorFromIterable(
- new Iterable()
- {
- Yielder yielder = Yielders.each(rows);
-
- @Override
- public Iterator iterator()
- {
- return new Iterator()
- {
- @Override
- public boolean hasNext()
- {
- return !yielder.isDone();
- }
-
- @Override
- public Object[] next()
- {
- Object[] retVal = yielder.get();
- yielder = yielder.next(null);
- return retVal;
- }
- };
- }
- },
- rowSignature
- );
+ return Pair.of(baseCursor, rowWalker);
}
}
diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
index 73205d2b75fa..9c746dd41429 100644
--- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java
@@ -43,6 +43,7 @@
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.MappedSequence;
import org.apache.druid.java.util.common.guava.Sequence;
@@ -72,6 +73,7 @@
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.DateTime;
+import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.BitSet;
@@ -726,12 +728,14 @@ public Optional> resultsAsFrames(
);
- Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
+ Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature
);
+ Cursor cursor = cursorAndCloseable.lhs;
+ Closeable closeble = cursorAndCloseable.rhs;
- Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
+ Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).withBaggage(closeble);
return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
}
diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java
index 97237ef77a2c..0eefa288df2f 100644
--- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java
+++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java
@@ -70,7 +70,7 @@ public Column findColumn(String name)
.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);
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
index b7253d70fe93..4d0885da00d8 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
@@ -36,12 +36,12 @@
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.guava.BaseSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
-import org.apache.druid.java.util.common.guava.Yielder;
-import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.FrameSignaturePair;
import org.apache.druid.query.GenericQueryMetricsFactory;
import org.apache.druid.query.IterableRowsCursorHelper;
@@ -57,6 +57,7 @@
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.utils.CloseableUtils;
+import java.io.Closeable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@@ -220,24 +221,7 @@ public Optional> resultsAsFrames(
)
{
final RowSignature defaultRowSignature = resultArraySignature(query);
- Iterator resultSequenceIterator = new Iterator()
- {
- Yielder yielder = Yielders.each(resultSequence);
-
- @Override
- public boolean hasNext()
- {
- return !yielder.isDone();
- }
-
- @Override
- public ScanResultValue next()
- {
- ScanResultValue scanResultValue = yielder.get();
- yielder = yielder.next(null);
- return scanResultValue;
- }
- };
+ ScanResultValueIterator resultSequenceIterator = new ScanResultValueIterator(resultSequence);
Iterable> retVal = () -> new Iterator>()
{
@@ -280,7 +264,7 @@ public Sequence next()
);
}
};
- return Optional.of(Sequences.concat(retVal));
+ return Optional.of(Sequences.concat(retVal).withBaggage(resultSequenceIterator));
}
private Sequence convertScanResultValuesToFrame(
@@ -294,16 +278,22 @@ private Sequence convertScanResultValuesToFrame(
Preconditions.checkNotNull(rowSignature, "'rowSignature' must be provided");
List cursors = new ArrayList<>();
+ Closer closer = Closer.create();
for (ScanResultValue scanResultValue : batch) {
final List rows = (List) scanResultValue.getEvents();
final Function, Object[]> mapper = getResultFormatMapper(query.getResultFormat(), rowSignature.getColumnNames());
final Iterable formattedRows = Lists.newArrayList(Iterables.transform(rows, (Function) mapper));
- cursors.add(IterableRowsCursorHelper.getCursorFromIterable(
+ Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromIterable(
formattedRows,
rowSignature
- ));
+ );
+ Cursor cursor = cursorAndCloseable.lhs;
+ Closeable closeable = cursorAndCloseable.rhs;
+ cursors.add(cursor);
+ // Cursors created from iterators don't have any resources, therefore this is mostly a defensive check
+ closer.register(closeable);
}
RowSignature modifiedRowSignature = useNestedForUnknownTypes
@@ -323,7 +313,7 @@ private Sequence convertScanResultValuesToFrame(
frameWriterFactory
);
- return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature));
+ return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)).withBaggage(closer);
}
@Override
diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueIterator.java
new file mode 100644
index 000000000000..646c69eaf185
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueIterator.java
@@ -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.query.scan;
+
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.java.util.common.parsers.CloseableIterator;
+
+import java.io.IOException;
+
+/**
+ * Iterates over the scan result sequence and provides an interface to clean up the resources (if any) to close the
+ * underlying sequence. Similar to {@link Yielder}, once close is called on the iterator, the calls to the rest of the
+ * iterator's methods are undefined.
+ */
+public class ScanResultValueIterator implements CloseableIterator
+{
+ Yielder yielder;
+
+ public ScanResultValueIterator(final Sequence resultSequence)
+ {
+ yielder = Yielders.each(resultSequence);
+ }
+
+ @Override
+ public void close() throws IOException
+ {
+ yielder.close();
+ }
+
+ @Override
+ public boolean hasNext()
+ {
+ return !yielder.isDone();
+ }
+
+ @Override
+ public Object next()
+ {
+ ScanResultValue scanResultValue = yielder.get();
+ yielder = yielder.next(null);
+ return scanResultValue;
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
index c16fe29c14de..cd8e553bf512 100644
--- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java
@@ -38,6 +38,7 @@
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.DateTimes;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
@@ -65,6 +66,7 @@
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.DateTime;
+import java.io.Closeable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@@ -474,10 +476,12 @@ public Optional> resultsAsFrames(
)
{
final RowSignature rowSignature = resultArraySignature(query);
- final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
+ final Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature
);
+ final Cursor cursor = cursorAndCloseable.lhs;
+ final Closeable closeable = cursorAndCloseable.rhs;
RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
@@ -489,7 +493,7 @@ public Optional> resultsAsFrames(
new ArrayList<>()
);
- Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
+ Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).withBaggage(closeable);
// All frames are generated with the same signature therefore we can attach the row signature
return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
index 80ffb3e62974..87b50e0e4677 100644
--- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
+++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java
@@ -35,6 +35,7 @@
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
@@ -64,6 +65,7 @@
import org.apache.druid.segment.column.RowSignature;
import org.joda.time.DateTime;
+import java.io.Closeable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
@@ -558,10 +560,12 @@ public Optional> resultsAsFrames(
)
{
final RowSignature rowSignature = resultArraySignature(query);
- final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(
+ final Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence(
resultsAsArrays(query, resultSequence),
rowSignature
);
+ Cursor cursor = cursorAndCloseable.lhs;
+ Closeable closeable = cursorAndCloseable.rhs;
RowSignature modifiedRowSignature = useNestedForUnknownTypes
? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature)
@@ -573,7 +577,7 @@ public Optional> resultsAsFrames(
new ArrayList<>()
);
- Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory);
+ Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).withBaggage(closeable);
return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)));
}
diff --git a/processing/src/main/java/org/apache/druid/segment/RowWalker.java b/processing/src/main/java/org/apache/druid/segment/RowWalker.java
index d6241f197e85..f55245b3bca3 100644
--- a/processing/src/main/java/org/apache/druid/segment/RowWalker.java
+++ b/processing/src/main/java/org/apache/druid/segment/RowWalker.java
@@ -26,14 +26,19 @@
import org.joda.time.DateTime;
import javax.annotation.Nullable;
+import java.io.Closeable;
import java.io.IOException;
import java.util.function.ToLongFunction;
/**
* Used by {@link RowBasedStorageAdapter} and {@link RowBasedCursor} to walk through rows. It allows multiple
* {@link RowBasedCursor} to share the same underlying Iterable.
+ *
+ * The class creates a yielder from the sequence to iterate over the rows. However, it doesn't call the sequence's close
+ * after iterating over it. {@link #close()} should be called by the instantiators of the class to clear the resources
+ * held by the {@link #rowSequence} and the corresponding yielder created to iterate over it.
*/
-public class RowWalker
+public class RowWalker implements Closeable
{
private final Sequence rowSequence;
private final ToLongFunction timestampFunction;
@@ -86,6 +91,7 @@ public void skipToDateTime(final DateTime timestamp, final boolean descending)
}
}
+ @Override
public void close()
{
if (rowYielder != null) {
diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java
index 3eea9193cc37..49793fba4163 100644
--- a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java
+++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java
@@ -57,6 +57,8 @@ public static ColumnType ofType(TypeSignature type)
switch (type.getElementType().getType()) {
case LONG:
return ColumnType.LONG_ARRAY;
+ case FLOAT:
+ return ColumnType.FLOAT_ARRAY;
case DOUBLE:
return ColumnType.DOUBLE_ARRAY;
case STRING:
diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java
index a7de99905c39..3e2b46d5f681 100644
--- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java
+++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java
@@ -298,7 +298,7 @@ private List createColumnReaders(RowSignature rowSignature)
"Type for column [%s]",
rowSignature.getColumnName(columnNumber)
);
- columnReaders.add(FrameColumnReaders.create(columnNumber, columnType));
+ columnReaders.add(FrameColumnReaders.create(rowSignature.getColumnName(columnNumber), columnNumber, columnType));
}
return columnReaders;
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java
index a4fd0907066b..4e46a1a529a0 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java
@@ -24,7 +24,6 @@
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.ByteBufferUtils;
import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
@@ -99,7 +98,7 @@ public int lookupString(@Nullable String value)
// for strings because of this. if other type dictionary writers could potentially use multiple internal files
// in the future, we should transition them to using this approach as well (or build a combination smoosher and
// mapper so that we can have a mutable smoosh)
- File stringSmoosh = FileUtils.createTempDir(name + "__stringTempSmoosh");
+ File stringSmoosh = FileUtils.createTempDir(StringUtils.urlEncode(name) + "__stringTempSmoosh");
final String fileName = NestedCommonFormatColumnSerializer.getInternalFileName(
name,
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
@@ -127,7 +126,7 @@ public int lookupString(@Nullable String value)
final byte[] bytes = StringUtils.toUtf8Nullable(value);
final int index = stringDictionary.indexOf(bytes == null ? null : ByteBuffer.wrap(bytes));
if (index < 0) {
- throw DruidException.defensive("Value not found in string dictionary");
+ throw DruidException.defensive("Value not found in column[%s] string dictionary", name);
}
return index;
}
@@ -135,7 +134,7 @@ public int lookupString(@Nullable String value)
public int lookupLong(@Nullable Long value)
{
if (longDictionary == null) {
- Path longFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME);
+ final Path longFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME);
longBuffer = mapWriter(longFile, longDictionaryWriter);
longDictionary = FixedIndexed.read(longBuffer, TypeStrategies.LONG, ByteOrder.nativeOrder(), Long.BYTES).get();
// reset position
@@ -143,7 +142,7 @@ public int lookupLong(@Nullable Long value)
}
final int index = longDictionary.indexOf(value);
if (index < 0) {
- throw DruidException.defensive("Value not found in long dictionary");
+ throw DruidException.defensive("Value not found in column[%s] long dictionary", name);
}
return index + longOffset();
}
@@ -151,15 +150,20 @@ public int lookupLong(@Nullable Long value)
public int lookupDouble(@Nullable Double value)
{
if (doubleDictionary == null) {
- Path doubleFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME);
+ final Path doubleFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME);
doubleBuffer = mapWriter(doubleFile, doubleDictionaryWriter);
- doubleDictionary = FixedIndexed.read(doubleBuffer, TypeStrategies.DOUBLE, ByteOrder.nativeOrder(), Double.BYTES).get();
+ doubleDictionary = FixedIndexed.read(
+ doubleBuffer,
+ TypeStrategies.DOUBLE,
+ ByteOrder.nativeOrder(),
+ Double.BYTES
+ ).get();
// reset position
doubleBuffer.position(0);
}
final int index = doubleDictionary.indexOf(value);
if (index < 0) {
- throw DruidException.defensive("Value not found in double dictionary");
+ throw DruidException.defensive("Value not found in column[%s] double dictionary", name);
}
return index + doubleOffset();
}
@@ -167,7 +171,7 @@ public int lookupDouble(@Nullable Double value)
public int lookupArray(@Nullable int[] value)
{
if (arrayDictionary == null) {
- Path arrayFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME);
+ final Path arrayFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME);
arrayBuffer = mapWriter(arrayFile, arrayDictionaryWriter);
arrayDictionary = FrontCodedIntArrayIndexed.read(arrayBuffer, ByteOrder.nativeOrder()).get();
// reset position
@@ -175,7 +179,7 @@ public int lookupArray(@Nullable int[] value)
}
final int index = arrayDictionary.indexOf(value);
if (index < 0) {
- throw DruidException.defensive("Value not found in array dictionary");
+ throw DruidException.defensive("Value not found in column[%s] array dictionary", name);
}
return index + arrayOffset();
}
@@ -239,7 +243,7 @@ private int arrayOffset()
private Path makeTempFile(String name)
{
try {
- return Files.createTempFile(name, ".tmp");
+ return Files.createTempFile(StringUtils.urlEncode(name), null);
}
catch (IOException e) {
throw new RuntimeException(e);
@@ -315,7 +319,11 @@ public long write(ByteBuffer[] srcs) throws IOException
public int addToOffset(long numBytesWritten)
{
if (numBytesWritten > bytesLeft()) {
- throw new ISE("Wrote more bytes[%,d] than available[%,d]. Don't do that.", numBytesWritten, bytesLeft());
+ throw DruidException.defensive(
+ "Wrote more bytes[%,d] than available[%,d]. Don't do that.",
+ numBytesWritten,
+ bytesLeft()
+ );
}
currOffset += numBytesWritten;
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java b/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java
index c8b3ab31302e..15691cfc9c4c 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java
@@ -151,35 +151,7 @@ public MutableTypeSet(byte types, boolean hasEmptyArray)
public MutableTypeSet add(ColumnType type)
{
- switch (type.getType()) {
- case STRING:
- types |= STRING_MASK;
- break;
- case LONG:
- types |= LONG_MASK;
- break;
- case DOUBLE:
- types |= DOUBLE_MASK;
- break;
- case ARRAY:
- Preconditions.checkNotNull(type.getElementType(), "ElementType must not be null");
- switch (type.getElementType().getType()) {
- case STRING:
- types |= STRING_ARRAY_MASK;
- break;
- case LONG:
- types |= LONG_ARRAY_MASK;
- break;
- case DOUBLE:
- types |= DOUBLE_ARRAY_MASK;
- break;
- default:
- throw new ISE("Unsupported nested array type: [%s]", type.asTypeString());
- }
- break;
- default:
- throw new ISE("Unsupported nested type: [%s]", type.asTypeString());
- }
+ types = FieldTypeInfo.add(types, type);
return this;
}
@@ -207,7 +179,11 @@ public MutableTypeSet merge(byte other, boolean hasEmptyArray)
@Nullable
public ColumnType getSingleType()
{
- return FieldTypeInfo.getSingleType(types);
+ final ColumnType columnType = FieldTypeInfo.getSingleType(types);
+ if (hasEmptyArray && columnType != null && !columnType.isArray()) {
+ return null;
+ }
+ return columnType;
}
public boolean isEmpty()
@@ -218,6 +194,10 @@ public boolean isEmpty()
public byte getByteValue()
{
+ final ColumnType singleType = FieldTypeInfo.getSingleType(types);
+ if (hasEmptyArray && singleType != null && !singleType.isArray()) {
+ return FieldTypeInfo.add(types, ColumnType.ofArray(singleType));
+ }
return types;
}
@@ -293,6 +273,40 @@ private static ColumnType getSingleType(byte types)
}
}
+ public static byte add(byte types, ColumnType type)
+ {
+ switch (type.getType()) {
+ case STRING:
+ types |= STRING_MASK;
+ break;
+ case LONG:
+ types |= LONG_MASK;
+ break;
+ case DOUBLE:
+ types |= DOUBLE_MASK;
+ break;
+ case ARRAY:
+ Preconditions.checkNotNull(type.getElementType(), "ElementType must not be null");
+ switch (type.getElementType().getType()) {
+ case STRING:
+ types |= STRING_ARRAY_MASK;
+ break;
+ case LONG:
+ types |= LONG_ARRAY_MASK;
+ break;
+ case DOUBLE:
+ types |= DOUBLE_ARRAY_MASK;
+ break;
+ default:
+ throw new ISE("Unsupported nested array type: [%s]", type.asTypeString());
+ }
+ break;
+ default:
+ throw new ISE("Unsupported nested type: [%s]", type.asTypeString());
+ }
+ return types;
+ }
+
public static Set convertToSet(byte types)
{
final Set theTypes = Sets.newHashSetWithExpectedSize(4);
diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java
new file mode 100644
index 000000000000..6381138f62db
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java
@@ -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 DOUBLES_LIST_1;
+ private static final List 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 expected, Object actual)
+ {
+ if (expected == null) {
+ Assert.assertNull(actual);
+ }
+ Assert.assertTrue(actual instanceof Object[]);
+ List actualList = new ArrayList<>();
+ for (Object val : (Object[]) actual) {
+ actualList.add((Double) val);
+ }
+
+ Assert.assertEquals(expected, actualList);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java
index 2393a6a7c3c9..8736c73276cc 100644
--- a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java
@@ -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 void tearDown()
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 void test_makeColumnValueSelector_defaultOrNull()
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 void test_makeColumnValueSelector_aValue()
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 void test_makeDimensionSelector_defaultOrNull()
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 void test_makeDimensionSelector_aValue()
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 void test_makeDimensionSelector_aValue_extractionFn()
writeToMemory(20.5d);
final DimensionSelector readSelector =
- new DoubleFieldReader().makeDimensionSelector(
+ DoubleFieldReader.forPrimitive().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)
diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java
new file mode 100644
index 000000000000..e61e40db1cb1
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java
@@ -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 FLOATS_LIST_1;
+ private static final List 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 expected, Object actual)
+ {
+ if (expected == null) {
+ Assert.assertNull(actual);
+ }
+ Assert.assertTrue(actual instanceof Object[]);
+ List actualList = new ArrayList<>();
+ for (Object val : (Object[]) actual) {
+ actualList.add((Float) val);
+ }
+
+ Assert.assertEquals(expected, actualList);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java
index e85225b99877..441858f0fd67 100644
--- a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java
@@ -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 void tearDown()
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 void test_makeColumnValueSelector_defaultOrNull()
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 void test_makeColumnValueSelector_aValue()
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 void test_makeDimensionSelector_defaultOrNull()
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 void test_makeDimensionSelector_aValue()
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 void test_makeDimensionSelector_aValue_extractionFn()
writeToMemory(20.5f);
final DimensionSelector readSelector =
- new FloatFieldReader().makeDimensionSelector(
+ FloatFieldReader.forPrimitive().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)
diff --git a/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java
new file mode 100644
index 000000000000..1e115f48e3c5
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java
@@ -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 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);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java
new file mode 100644
index 000000000000..aa34cd6afaf3
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java
@@ -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 LONGS_LIST_1;
+ private static final List 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 expected, Object actual)
+ {
+ if (expected == null) {
+ Assert.assertNull(actual);
+ }
+ Assert.assertTrue(actual instanceof Object[]);
+ List actualList = new ArrayList<>();
+ for (Object val : (Object[]) actual) {
+ actualList.add((Long) val);
+ }
+
+ Assert.assertEquals(expected, actualList);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java
index 06e6c42d9e70..7e73a7cfdf03 100644
--- a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java
@@ -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 void tearDown()
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 void test_makeColumnValueSelector_defaultOrNull()
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 void test_makeColumnValueSelector_aValue()
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 void test_makeDimensionSelector_defaultOrNull()
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 void test_makeDimensionSelector_aValue()
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 void test_makeDimensionSelector_aValue_extractionFn()
writeToMemory(25L);
final DimensionSelector readSelector =
- new LongFieldReader().makeDimensionSelector(
+ LongFieldReader.forPrimitive().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)
diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java
index 97370187522e..bde3f77480ee 100644
--- a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java
@@ -81,24 +81,24 @@ public void tearDown()
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 void test_isNull_emptyString()
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 void test_makeColumnValueSelector_singleString_notArray()
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 void test_makeColumnValueSelector_multiString()
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 void test_makeColumnValueSelector_null()
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 void test_makeColumnValueSelector_empty()
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 void test_makeDimensionSelector_multiString_asArray()
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 void test_makeDimensionSelector_multiString()
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 void test_makeDimensionSelector_multiString_withExtractionFn()
writeToMemory(ImmutableList.of("foo", "bar"));
final DimensionSelector readSelector =
- new StringFieldReader(false).makeDimensionSelector(
+ new StringFieldReader().makeDimensionSelector(
memory,
new ConstantFieldPointer(MEMORY_POSITION),
new SubstringDimExtractionFn(1, null)
diff --git a/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java
new file mode 100644
index 000000000000..276e598ad367
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java
@@ -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 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 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 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;
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
index 342ccbefc48c..31b24825f959 100644
--- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java
@@ -153,6 +153,33 @@ public void test_long()
testWithDataset(FrameWriterTestData.TEST_LONGS);
}
+ @Test
+ public void test_arrayLong()
+ {
+ // ARRAY 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 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 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 void test_typePairs()
// 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> rowSequence = unsortAndMakeRows(Arrays.asList(dataset1, dataset2));
@@ -265,6 +300,7 @@ public void test_typePairs()
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> rowSequence = unsortAndMakeRows(FrameWriterTestData.DATASETS);
final int totalRows = rowSequence.toList().size();
diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java
index 3017f5b9ed48..a52c4d5efdd8 100644
--- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java
+++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java
@@ -136,6 +136,24 @@ public class FrameWriterTestData
).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList())
);
+ public static final Dataset 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 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 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 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 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 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();
diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java
index 42ed468ec8c3..9d359eed05e0 100644
--- a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java
+++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java
@@ -75,42 +75,6 @@ public void test_columnar()
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()
{
diff --git a/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java
index fbbc089255b5..e01c9459fa12 100644
--- a/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java
+++ b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java
@@ -32,6 +32,7 @@
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.column.ColumnType;
@@ -40,6 +41,7 @@
import org.junit.Assert;
import org.junit.Test;
+import java.io.Closeable;
import java.util.ArrayList;
public class FrameBasedInlineDataSourceSerializerTest
@@ -124,10 +126,11 @@ private FrameBasedInlineDataSource convertToFrameBasedInlineDataSource(
RowSignature rowSignature
)
{
- Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(
+ Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromIterable(
inlineDataSource.getRows(),
rowSignature
);
+ Cursor cursor = cursorAndCloseable.lhs;
RowSignature modifiedRowSignature = FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature);
Sequence frames = FrameCursorUtils.cursorToFrames(
cursor,
@@ -139,7 +142,7 @@ private FrameBasedInlineDataSource convertToFrameBasedInlineDataSource(
)
);
return new FrameBasedInlineDataSource(
- frames.map(frame -> new FrameSignaturePair(frame, rowSignature)).toList(),
+ frames.map(frame -> new FrameSignaturePair(frame, rowSignature)).withBaggage(cursorAndCloseable.rhs).toList(),
modifiedRowSignature
);
}
diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
index 1acaceabbd60..45f14b80976c 100644
--- a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
+++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java
@@ -48,7 +48,7 @@ public class IterableRowsCursorHelperTest
@Test
public void getCursorFromIterable()
{
- Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(rows, rowSignature);
+ Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(rows, rowSignature).lhs;
testCursorMatchesRowSequence(cursor, rowSignature, rows);
}
@@ -56,7 +56,7 @@ public void getCursorFromIterable()
public void getCursorFromSequence()
{
- Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(Sequences.simple(rows), rowSignature);
+ Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(Sequences.simple(rows), rowSignature).lhs;
testCursorMatchesRowSequence(cursor, rowSignature, rows);
}
diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java
index 0d91e7d5e001..fcf6720311f0 100644
--- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java
+++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java
@@ -787,12 +787,12 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryTypeGauntlet() throw
Assert.assertEquals(resultsRealtime.size(), resultsSegments.size());
if (NullHandling.replaceWithDefault()) {
Assert.assertEquals(
- "[[1672531200000, null, 0, 0.0, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 2, 0.0, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, 0.0, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, 0, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, 0, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]",
+ "[[1672531200000, null, 0, 0.0, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}, v=[]}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 2, 0.0, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}, v=[]}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}, v=[]}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, 0.0, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}, v=[]}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, 0, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}, v=a}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}, v=b}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, 0, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]",
resultsSegments.get(0).getEvents().toString()
);
} else {
Assert.assertEquals(
- "[[1672531200000, null, null, null, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, , 2, null, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, null, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, null, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, null, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]",
+ "[[1672531200000, null, null, null, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}, v=[]}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, , 2, null, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}, v=[]}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}, v=[]}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, null, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}, v=[]}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, null, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}, v=a}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}, v=b}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, null, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]",
resultsSegments.get(0).getEvents().toString()
);
}
diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java
index ed59f5f80652..6a244e9bae97 100644
--- a/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java
@@ -33,6 +33,7 @@
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.frame.write.FrameWriters;
import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.FrameBasedInlineDataSource;
@@ -42,10 +43,12 @@
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.testing.InitializedNullHandlingTest;
+import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
+import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -207,11 +210,13 @@ public class FrameBasedIndexedTableTest extends InitializedNullHandlingTest
private FrameBasedInlineDataSource dataSource;
private FrameBasedIndexedTable frameBasedIndexedTable;
+ private Pair cursorCloseablePair;
@Before
public void setup()
{
- Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE);
+ cursorCloseablePair = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE);
+ Cursor cursor = cursorCloseablePair.lhs;
FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory(
FrameType.COLUMNAR,
new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()),
@@ -226,7 +231,12 @@ public void setup()
);
frameBasedIndexedTable = new FrameBasedIndexedTable(dataSource, KEY_COLUMNS, "test");
+ }
+ @After
+ public void tearDown() throws IOException
+ {
+ cursorCloseablePair.rhs.close();
}
@Test
diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java
index 80daa3549dcf..653b39ff9bf7 100644
--- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java
@@ -171,7 +171,7 @@ public static void staticSetup()
@Before
public void setup() throws IOException
{
- final String fileNameBase = "test";
+ final String fileNameBase = "test/column";
final String arrayFileNameBase = "array";
fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data);
baseBuffer = fileMapper.mapFile(fileNameBase);
diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java
index 0e8d95cc57c9..33df1887ea54 100644
--- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java
@@ -56,6 +56,23 @@ public void testSingleType() throws IOException
}
}
+ @Test
+ public void testSingleTypeWithEmptyArray() throws IOException
+ {
+ List supportedTypes = ImmutableList.of(
+ ColumnType.STRING,
+ ColumnType.LONG,
+ ColumnType.DOUBLE,
+ ColumnType.STRING_ARRAY,
+ ColumnType.LONG_ARRAY,
+ ColumnType.DOUBLE_ARRAY
+ );
+
+ for (ColumnType type : supportedTypes) {
+ testSingleTypeWithEmptyArray(type);
+ }
+ }
+
@Test
public void testMultiType() throws IOException
{
@@ -137,6 +154,51 @@ private void testMultiType(Set columnTypes) throws IOException
Assert.assertEquals(1, BUFFER.position());
}
+ private void testSingleTypeWithEmptyArray(ColumnType columnType) throws IOException
+ {
+ FieldTypeInfo.MutableTypeSet typeSet = new FieldTypeInfo.MutableTypeSet();
+ typeSet.add(columnType);
+ typeSet.addUntypedArray();
+
+ if (columnType.isArray()) {
+ // arrays with empty arrays are still single type
+ Assert.assertEquals(columnType, typeSet.getSingleType());
+ Assert.assertEquals(ImmutableSet.of(columnType), FieldTypeInfo.convertToSet(typeSet.getByteValue()));
+
+ writeTypeSet(typeSet);
+ FieldTypeInfo info = new FieldTypeInfo(BUFFER);
+ Assert.assertEquals(0, BUFFER.position());
+
+ FieldTypeInfo.TypeSet roundTrip = info.getTypes(0);
+ Assert.assertEquals(columnType, roundTrip.getSingleType());
+
+ FieldTypeInfo info2 = FieldTypeInfo.read(BUFFER, 1);
+ Assert.assertEquals(info.getTypes(0), info2.getTypes(0));
+ Assert.assertEquals(1, BUFFER.position());
+ } else {
+ // scalar types become multi-type
+ Set columnTypes = ImmutableSet.of(columnType, ColumnType.ofArray(columnType));
+ FieldTypeInfo.MutableTypeSet merge = new FieldTypeInfo.MutableTypeSet();
+ merge.merge(new FieldTypeInfo.MutableTypeSet().add(columnType).getByteValue(), true);
+
+ Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue());
+ Assert.assertNull(typeSet.getSingleType());
+ Assert.assertEquals(columnTypes, FieldTypeInfo.convertToSet(typeSet.getByteValue()));
+
+ writeTypeSet(typeSet);
+ FieldTypeInfo info = new FieldTypeInfo(BUFFER);
+ Assert.assertEquals(0, BUFFER.position());
+
+ FieldTypeInfo.TypeSet roundTrip = info.getTypes(0);
+ Assert.assertNull(roundTrip.getSingleType());
+ Assert.assertEquals(columnTypes, FieldTypeInfo.convertToSet(roundTrip.getByteValue()));
+
+ FieldTypeInfo info2 = FieldTypeInfo.read(BUFFER, 1);
+ Assert.assertEquals(info.getTypes(0), info2.getTypes(0));
+ Assert.assertEquals(1, BUFFER.position());
+ }
+ }
+
private static void writeTypeSet(FieldTypeInfo.MutableTypeSet typeSet) throws IOException
{
BUFFER.position(0);
diff --git a/processing/src/test/resources/nested-all-types-test-data.json b/processing/src/test/resources/nested-all-types-test-data.json
index 34d92b52ae82..b70c87646019 100644
--- a/processing/src/test/resources/nested-all-types-test-data.json
+++ b/processing/src/test/resources/nested-all-types-test-data.json
@@ -1,7 +1,7 @@
-{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantNumeric": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "variantWithArrays": 1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
-{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantNumeric": 1.1, "variantEmptyObj":"b", "variantEmtpyArray":2, "variantWithArrays": "b", "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
-{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantNumeric": 1.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "variantWithArrays": 3.0, "obj":{"a": 300}, "complexObj":{"x": 4.4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
-{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "variantWithArrays": "1", "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
-{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantNumeric": -1000, "variantEmptyObj":{}, "variantEmtpyArray":[], "variantWithArrays": "hello", "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
-{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantNumeric": 3.33, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
-{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantNumeric": -0.13, "variantEmptyObj":1, "variantEmtpyArray":[], "variantWithArrays": [51, -35], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantNumeric": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "variantWithArrays": 1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}, "v": []}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantNumeric": 1.1, "variantEmptyObj":"b", "variantEmtpyArray":2, "variantWithArrays": "b", "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}, "v": []}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantNumeric": 1.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "variantWithArrays": 3.0, "obj":{"a": 300}, "complexObj":{"x": 4.4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "variantWithArrays": "1", "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}, "v": []}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantNumeric": -1000, "variantEmptyObj":{}, "variantEmtpyArray":[], "variantWithArrays": "hello", "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}, "v": "a"}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantNumeric": 3.33, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}, "v": "b"}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
+{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantNumeric": -0.13, "variantEmptyObj":1, "variantEmtpyArray":[], "variantWithArrays": [51, -35], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}, "v": []}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]}
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
index 16f642701724..3bb20bb3780e 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java
@@ -1170,7 +1170,7 @@ protected void skipVectorize()
skipVectorize = true;
}
- protected void notMsqCompatible()
+ protected void msqIncompatible()
{
msqCompatible = false;
}
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java
index 8fc531d35ab6..e6a669b9c28b 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java
@@ -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 void testSelectNonConstantArrayExpressionFromTable()
@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 void testSomeArrayFunctionsWithScanQuery()
// 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 expectedResults;
if (useDefault) {
expectedResults = ImmutableList.of(
@@ -389,7 +388,6 @@ public void testSomeArrayFunctionsWithScanQueryNoStringify()
// 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 expectedResults;
if (useDefault) {
expectedResults = ImmutableList.of(
@@ -1021,7 +1019,6 @@ public void testArrayOffset()
@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 void testArrayGroupAsDoubleArray()
{
// 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 void testArrayGroupAsDoubleArray()
@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 void testArrayAggMultiValue()
@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 void testArrayAggNumeric()
@Test
public void testArrayAggQuantile()
{
- notMsqCompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_QUANTILE(ARRAY_AGG(l1), 0.9) FROM numfoo",
@@ -1793,7 +1786,9 @@ public void testArrayAggQuantile()
@Test
public void testArrayAggArrays()
{
- notMsqCompatible();
+ // Produces nested array - ARRAY>, 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 void testArrayAggArrays()
@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 void testArrayAggExpression()
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 void testArrayAggGroupByArrayAggFromSubquery()
@Test
public void testArrayAggGroupByArrayAggOfLongsFromSubquery()
{
- notMsqCompatible();
requireMergeBuffers(3);
cannotVectorize();
testQuery(
@@ -2379,7 +2371,6 @@ public void testArrayAggGroupByArrayAggOfStringsFromSubquery()
@Test
public void testArrayAggGroupByArrayAggOfDoubleFromSubquery()
{
- notMsqCompatible();
requireMergeBuffers(3);
cannotVectorize();
testQuery(
@@ -2897,7 +2888,6 @@ public void testUnnestTwiceWithFiltersAndExpressions()
@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 void testUnnestThriceWithFiltersOnDimAndUnnestCol()
@Test
public void testUnnestThriceWithFiltersOnDimAndAllUnnestColumns()
{
- notMsqCompatible();
cannotVectorize();
String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n"
+ " ( SELECT * FROM \n"
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
index bb1660f856a9..d1300ff19b24 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
@@ -695,7 +695,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorBackwards(Map queryContext)
{
// MSQ does not support UNION ALL.
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize JOIN operator.
cannotVectorize();
@@ -1918,7 +1918,7 @@ public void testCommaJoinTableLookupTableMismatchedTypes(Map que
// 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 void testCommaJoinTableLookupTableMismatchedTypes(Map que
public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map 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 void testSelectOnLookupUsingLeftJoinOperator(Map queryCon
public void testSelectOnLookupUsingRightJoinOperator(Map queryContext)
{
// MSQ refuses to do RIGHT join with broadcast.
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT dim1, lookyloo.*\n"
@@ -2438,7 +2438,7 @@ public void testSelectOnLookupUsingRightJoinOperator(Map queryCo
public void testSelectOnLookupUsingFullJoinOperator(Map queryContext)
{
// MSQ refuses to do FULL join with broadcast.
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT dim1, m1, cnt, lookyloo.*\n"
@@ -3751,7 +3751,7 @@ public void testInnerJoinSubqueryWithSelectorFilter(Map queryCon
{
if (sortBasedJoin) {
// Cannot handle the [l1.k = 'abc'] condition.
- notMsqCompatible();
+ msqIncompatible();
}
// Cannot vectorize due to 'concat' expression.
@@ -4172,7 +4172,7 @@ public void testSemiJoinWithOuterTimeExtractAggregateWithOrderBy()
public void testJoinOnMultiValuedColumnShouldThrowException(Map 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 void testJoinOnMultiValuedColumnShouldThrowException(Map
public void testUnionAllTwoQueriesLeftQueryIsJoin(Map 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 void testUnionAllTwoQueriesLeftQueryIsJoin(Map queryConte
public void testUnionAllTwoQueriesRightQueryIsJoin(Map 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 void testUnionAllTwoQueriesRightQueryIsJoin(Map queryCont
public void testUnionAllTwoQueriesBothQueriesAreJoin()
{
// MSQ does not support UNION ALL.
- notMsqCompatible();
+ msqIncompatible();
cannotVectorize();
testQuery(
@@ -4982,7 +4982,7 @@ public void testVirtualColumnOnMVFilterJoinExpression(Map queryC
// 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 void testVirtualColumnOnMVFilterMultiJoinExpression(Map q
// 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 "
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java
index d440a6bb7182..5098343b5389 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java
@@ -5667,7 +5667,7 @@ public void testScanAllTypesAuto()
"1",
"[]",
"[51,-35]",
- "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}",
+ "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]},\"v\":[]}",
"{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}",
null,
"[\"a\",\"b\"]",
@@ -5705,7 +5705,7 @@ public void testScanAllTypesAuto()
"\"b\"",
"2",
"b",
- "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}",
+ "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]},\"v\":[]}",
"{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}",
"[\"a\",\"b\",\"c\"]",
"[null,\"b\"]",
@@ -5743,7 +5743,7 @@ public void testScanAllTypesAuto()
"1",
"1",
"1",
- "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}",
+ "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]},\"v\":[]}",
"{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
"[\"a\",\"b\"]",
"[\"a\",\"b\"]",
@@ -5781,7 +5781,7 @@ public void testScanAllTypesAuto()
"{}",
"4",
"1",
- "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}",
+ "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]},\"v\":[]}",
"{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
"[\"d\",\"e\"]",
"[\"b\",\"b\"]",
@@ -5819,7 +5819,7 @@ public void testScanAllTypesAuto()
"{}",
"[]",
"hello",
- "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}",
+ "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]},\"v\":\"a\"}",
"{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}",
null,
null,
@@ -5857,7 +5857,7 @@ public void testScanAllTypesAuto()
"\"a\"",
"6",
null,
- "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}",
+ "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]},\"v\":\"b\"}",
null,
"[\"a\",\"b\"]",
null,
@@ -5935,7 +5935,7 @@ public void testScanAllTypesAuto()
"1",
"[]",
"[51,-35]",
- "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}",
+ "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]},\"v\":[]}",
"{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}",
null,
"[\"a\",\"b\"]",
@@ -5973,7 +5973,7 @@ public void testScanAllTypesAuto()
"\"b\"",
"2",
"b",
- "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}",
+ "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]},\"v\":[]}",
"{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}",
"[\"a\",\"b\",\"c\"]",
"[null,\"b\"]",
@@ -6011,7 +6011,7 @@ public void testScanAllTypesAuto()
"1",
"1",
"1",
- "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}",
+ "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]},\"v\":[]}",
"{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
"[\"a\",\"b\"]",
"[\"a\",\"b\"]",
@@ -6049,7 +6049,7 @@ public void testScanAllTypesAuto()
"{}",
"4",
"1",
- "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}",
+ "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]},\"v\":[]}",
"{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}",
"[\"d\",\"e\"]",
"[\"b\",\"b\"]",
@@ -6087,7 +6087,7 @@ public void testScanAllTypesAuto()
"{}",
"[]",
"hello",
- "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}",
+ "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]},\"v\":\"a\"}",
"{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}",
null,
null,
@@ -6125,7 +6125,7 @@ public void testScanAllTypesAuto()
"\"a\"",
"6",
null,
- "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}",
+ "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]},\"v\":\"b\"}",
null,
"[\"a\",\"b\"]",
null,
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index 35f85c5fbf2b..d67ed60c5527 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -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 void testInformationSchemaSchemata()
@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 void testInformationSchemaTables()
@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 void testInformationSchemaColumnsOnTable()
@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 void testInformationSchemaColumnsOnForbiddenTable()
@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 void testInformationSchemaColumnsOnView()
@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 void testInformationSchemaColumnsOnAnotherView()
@Test
public void testCannotInsertWithNativeEngine()
{
- notMsqCompatible();
+ msqIncompatible();
final DruidException e = Assert.assertThrows(
DruidException.class,
() -> testQuery(
@@ -341,7 +341,7 @@ public void testCannotInsertWithNativeEngine()
@Test
public void testCannotReplaceWithNativeEngine()
{
- notMsqCompatible();
+ msqIncompatible();
final DruidException e = Assert.assertThrows(
DruidException.class,
() -> testQuery(
@@ -360,7 +360,7 @@ public void testCannotReplaceWithNativeEngine()
@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 void testAggregatorsOnInformationSchemaColumns()
@Test
public void testFilterAggregatorFunctionsOnInformationSchemaRoutines()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ " COUNT(*)\n"
@@ -397,7 +397,7 @@ public void testFilterAggregatorFunctionsOnInformationSchemaRoutines()
@Test
public void testFilterScalarFunctionsOnInformationSchemaRoutines()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ " COUNT(*)\n"
@@ -413,7 +413,7 @@ public void testFilterScalarFunctionsOnInformationSchemaRoutines()
@Test
public void testNonExistentSchemaOnInformationSchemaRoutines()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT *\n"
+ "FROM INFORMATION_SCHEMA.ROUTINES\n"
@@ -426,7 +426,7 @@ public void testNonExistentSchemaOnInformationSchemaRoutines()
@Test
public void testTopNLimitWrapping()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@@ -631,7 +631,7 @@ public void testGroupBySingleColumnDescendingNoTopN()
@Test
public void testEarliestAggregators()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT "
@@ -678,7 +678,7 @@ public void testEarliestAggregators()
@Test
public void testLatestVectorAggregators()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT "
+ "LATEST(cnt), LATEST(cnt + 1), LATEST(m1), LATEST(m1+1) "
@@ -712,7 +712,7 @@ public void testLatestVectorAggregators()
@Test
public void testLatestAggregators()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT "
@@ -857,7 +857,7 @@ public void testAnyAggregatorsOnHeapNumericNulls()
@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 void testAnyAggregatorsOffHeapNumericNulls()
@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 void testPrimitiveLatestInSubquery()
@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 void testStringLatestByGroupByWithAlwaysFalseCondition()
@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 void testPrimitiveAnyInSubquery()
@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 void testStringAnyInSubquery()
@Test
public void testEarliestAggregatorsNumericNulls()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT EARLIEST(l1), EARLIEST(d1), EARLIEST(f1) FROM druid.numfoo",
@@ -1447,7 +1447,7 @@ public void testEarliestAggregatorsNumericNulls()
@Test
public void testLatestAggregatorsNumericNull()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT LATEST(l1), LATEST(d1), LATEST(f1) FROM druid.numfoo",
ImmutableList.of(
@@ -1478,7 +1478,7 @@ public void testLatestAggregatorsNumericNull()
@Test
public void testFirstLatestAggregatorsSkipNulls()
{
- notMsqCompatible();
+ msqIncompatible();
final DimFilter filter;
if (useDefault) {
@@ -1592,7 +1592,7 @@ public void testAnyAggregatorsSkipNullsWithFilter()
@Test
public void testOrderByEarliestFloat()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
@@ -1639,7 +1639,7 @@ public void testOrderByEarliestFloat()
@Test
public void testOrderByEarliestDouble()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
@@ -1686,7 +1686,7 @@ public void testOrderByEarliestDouble()
@Test
public void testOrderByEarliestLong()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
@@ -1733,7 +1733,7 @@ public void testOrderByEarliestLong()
@Test
public void testOrderByLatestFloat()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@@ -1780,7 +1780,7 @@ public void testOrderByLatestFloat()
@Test
public void testOrderByLatestDouble()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@@ -1826,7 +1826,7 @@ public void testOrderByLatestDouble()
@Test
public void testOrderByLatestLong()
{
- notMsqCompatible();
+ msqIncompatible();
List expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
@@ -1884,7 +1884,7 @@ public void testOrderByAnyFloat()
);
} 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 void testOrderByAnyDouble()
);
} 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 void testOrderByAnyLong()
);
} 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 void testGroupByOrdinal()
@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 void testHavingOnExactCountDistinct()
@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 void testTopNWithSelectAndOrderByProjections()
@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 void testUnionAllQueries()
@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 void testUnionAllQueriesWithLimit()
@Test
public void testUnionAllDifferentTablesWithMapping()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@@ -2921,7 +2921,7 @@ public void testUnionAllDifferentTablesWithMapping()
@Test
public void testJoinUnionAllDifferentTablesWithMapping()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@@ -2985,7 +2985,7 @@ public void testUnionAllTablesColumnCountMismatch()
@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 void testUnionAllTablesWhenCastAndMappingIsRequired()
@Test
public void testUnionAllSameTableTwice()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@@ -3139,7 +3139,7 @@ public void testUnionAllSameTableTwice()
@Test
public void testUnionAllSameTableTwiceWithSameMapping()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@@ -3197,7 +3197,7 @@ public void testUnionAllSameTableTwiceWithDifferentMapping()
@Test
public void testUnionAllSameTableThreeTimes()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@@ -3302,7 +3302,7 @@ public void testUnionAllThreeTablesColumnCountMismatch3()
@Test
public void testUnionAllSameTableThreeTimesWithSameMapping()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ "dim1, dim2, SUM(m1), COUNT(*)\n"
@@ -3673,7 +3673,7 @@ public void testNullDoubleTopN()
{
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
if (NullHandling.sqlCompatible()) {
- notMsqCompatible();
+ msqIncompatible();
}
List expected;
if (useDefault) {
@@ -3717,7 +3717,7 @@ public void testNullFloatTopN()
{
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
if (NullHandling.sqlCompatible()) {
- notMsqCompatible();
+ msqIncompatible();
}
List expected;
if (useDefault) {
@@ -3761,7 +3761,7 @@ public void testNullLongTopN()
{
// Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved
if (NullHandling.sqlCompatible()) {
- notMsqCompatible();
+ msqIncompatible();
}
List expected;
if (useDefault) {
@@ -4199,7 +4199,7 @@ public void testColumnIsNull()
@Test
public void testGroupingWithNullInFilter()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL)",
ImmutableList.of(
@@ -4224,7 +4224,7 @@ public void testGroupingWithNullInFilter()
@Test
public void testGroupByNothingWithLiterallyFalseFilter()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE 1 = 0",
ImmutableList.of(
@@ -4250,7 +4250,7 @@ public void testGroupByNothingWithLiterallyFalseFilter()
@Test
public void testGroupByNothingWithImpossibleTimeFilter()
{
- notMsqCompatible();
+ msqIncompatible();
// Regression test for https://github.com/apache/druid/issues/7671
testQuery(
@@ -4321,7 +4321,7 @@ public void testGroupByOneColumnWithLiterallyFalseFilter()
@Test
public void testGroupByWithFilterMatchingNothing()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar'",
ImmutableList.of(
@@ -4669,7 +4669,7 @@ public void testCountStarWithLongColumnFiltersForceRange()
@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 void testCountStarWithDegenerateFilter()
public void testCountStarWithNotOfDegenerateFilter()
{
- notMsqCompatible();
+ msqIncompatible();
// HashJoinSegmentStorageAdapter is not vectorizable
cannotVectorize();
@@ -5689,7 +5689,7 @@ public void testCountStarWithNotOfDegenerateFilter()
@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 void testIsNotDistinctFromLiteral()
@Test
public void testArrayAggQueryOnComplexDatatypes()
{
- notMsqCompatible();
+ msqIncompatible();
cannotVectorize();
testQuery(
"SELECT ARRAY_AGG(unique_dim1) FROM druid.foo",
@@ -5961,7 +5961,7 @@ public void testCountStarWithTimeInIntervalFilterLosAngeles()
@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 void testCountStarWithTimeInIntervalFilterInvalidInterval()
@Test
public void testCountStarWithTimeInIntervalFilterNonLiteral()
{
- notMsqCompatible();
+ msqIncompatible();
testQueryThrows(
"SELECT COUNT(*) FROM druid.foo "
+ "WHERE TIME_IN_INTERVAL(__time, dim1)",
@@ -6559,7 +6559,7 @@ public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp()
@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 void testApproxCountDistinctWhenHllDisabled()
{
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 void testApproxCountDistinctBuiltin()
{
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 void testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJo
@Test
public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets()
{
- notMsqCompatible();
+ msqIncompatible();
requireMergeBuffers(4);
testQuery(
PLANNER_CONFIG_NO_HLL.withOverrides(
@@ -7004,7 +7004,7 @@ public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets()
@Test
public void testApproxCountDistinct()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -7333,7 +7333,7 @@ public void testExactCountDistinctUsingSubquery()
@Test
public void testExactCountDistinctUsingSubqueryOnUnionAllTables()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT\n"
+ " SUM(cnt),\n"
@@ -7485,7 +7485,7 @@ public void testQueryWithMoreThanMaxNumericInFilter()
// 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 void testCountDistinctArithmetic()
{
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 void testFilterOnTimeExtractWithMultipleDays()
@Test
public void testFilterOnTimeExtractWithVariousTimeUnits()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -8548,7 +8548,7 @@ public void testFilterOnTimeExtractWithVariousTimeUnits()
@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 void testGroupByFloor()
@Test
public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename()
{
- notMsqCompatible();
+ msqIncompatible();
cannotVectorize();
skipVectorize();
requireMergeBuffers(3);
@@ -9068,7 +9068,7 @@ public void testCountDistinctOfLookup()
@Test
public void testGroupByExpressionFromLookup()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize direct queries on lookup tables.
cannotVectorize();
@@ -9454,7 +9454,7 @@ public void testTimeseriesLosAngelesUsingTimeFloorConnectionLosAngeles()
@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 void testTimeseriesDescending()
@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 void testTimeseriesEmptyResultsAggregatorDefaultValues()
@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 void testGroupByAggregatorDefaultValues()
@Test
public void testGroupByAggregatorDefaultValuesNonVectorized()
{
- notMsqCompatible();
+ msqIncompatible();
cannotVectorize();
skipVectorize();
testQuery(
@@ -10641,7 +10641,7 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim()
@Test
public void testGroupingSets()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -10706,7 +10706,7 @@ public void testGroupingSets()
@Test
public void testGroupingAggregatorDifferentOrder()
{
- notMsqCompatible();
+ msqIncompatible();
requireMergeBuffers(3);
testQuery(
@@ -10770,7 +10770,7 @@ public void testGroupingAggregatorDifferentOrder()
@Test
public void testGroupingAggregatorWithPostAggregator()
{
- notMsqCompatible();
+ msqIncompatible();
List resultList;
if (NullHandling.sqlCompatible()) {
resultList = ImmutableList.of(
@@ -10829,7 +10829,7 @@ public void testGroupingAggregatorWithPostAggregator()
@Test
public void testGroupingSetsWithNumericDimension()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
"SELECT cnt, COUNT(*)\n"
+ "FROM foo\n"
@@ -10860,7 +10860,7 @@ public void testGroupingSetsWithNumericDimension()
@Test
public void testGroupByRollup()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -10919,7 +10919,7 @@ public void testGroupByRollup()
@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 void testGroupByRollupDifferentOrder()
@Test
public void testGroupByCube()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -11037,7 +11037,7 @@ public void testGroupByCube()
@Test
public void testGroupingSetsWithDummyDimension()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -11099,7 +11099,7 @@ public void testGroupingSetsWithDummyDimension()
@Test
public void testGroupingSetsNoSuperset()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -11156,7 +11156,7 @@ public void testGroupingSetsNoSuperset()
@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 void testGroupingSetsWithOrderByDimension()
@Test
public void testGroupingSetsWithOrderByAggregator()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -11296,7 +11296,7 @@ public void testGroupingSetsWithOrderByAggregator()
@Test
public void testGroupingSetsWithOrderByAggregatorWithLimit()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -11562,7 +11562,7 @@ public void testProjectAfterSort2()
+ " 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 void testProjectAfterSort3()
@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 void testRequireTimeConditionLogicalValuePositive()
@Test
public void testRequireTimeConditionSimpleQueryNegative()
{
- notMsqCompatible();
+ msqIncompatible();
expectedException.expect(CannotBuildQueryException.class);
expectedException.expectMessage("__time column");
@@ -12171,7 +12171,7 @@ public void testRequireTimeConditionSimpleQueryNegative()
@Test
public void testRequireTimeConditionSubQueryNegative()
{
- notMsqCompatible();
+ msqIncompatible();
expectedException.expect(CannotBuildQueryException.class);
expectedException.expectMessage("__time column");
@@ -12191,7 +12191,7 @@ public void testRequireTimeConditionSubQueryNegative()
@Test
public void testRequireTimeConditionSemiJoinNegative()
{
- notMsqCompatible();
+ msqIncompatible();
expectedException.expect(CannotBuildQueryException.class);
expectedException.expectMessage("__time column");
@@ -12277,7 +12277,7 @@ public void testFilterLongDimension()
@Test
public void testTrigonometricFunction()
{
- notMsqCompatible();
+ msqIncompatible();
testQuery(
PLANNER_CONFIG_DEFAULT,
QUERY_CONTEXT_DEFAULT,
@@ -12576,7 +12576,7 @@ public void testLeftRightStringOperators()
@Test
public void testQueryContextOuterLimit()
{
- notMsqCompatible();
+ msqIncompatible();
Map outerLimitContext = new HashMap<>(QUERY_CONTEXT_DEFAULT);
outerLimitContext.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 4);
@@ -12887,7 +12887,7 @@ public void testTimeStampAddConversion()
@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 void testGroupingSetsWithLimit()
@Test
public void testGroupingSetsWithLimitOrderByGran()
{
- notMsqCompatible();
+ msqIncompatible();
// Cannot vectorize due to virtual columns.
cannotVectorize();
@@ -14276,7 +14276,7 @@ public void testGreatestFunctionForNumberWithIsNull()
@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 void testSubqueryTypeMismatchWithLiterals()
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 void testComplexDecode()
@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",
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java
index d39c9bf1388e..2ddc674eadda 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java
@@ -57,7 +57,6 @@
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.joda.time.DateTimeZone;
import org.joda.time.Period;
-import org.junit.Ignore;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@@ -227,7 +226,6 @@ public void testExactCountDistinctOfSemiJoinResult()
);
}
- @Ignore("Merge buffers exceed the prescribed limit when the results are materialized as frames")
@Test
public void testTwoExactCountDistincts()
{
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java
index 963e1e0b23bc..f9d82e71dd7e 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java
@@ -321,10 +321,15 @@ public static Pair> getResults(
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 @@ private void verifyResults(QueryResults queryResults)
}
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 QueryTestRunner(QueryTestBuilder builder)
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()) {