You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "gianm (via GitHub)" <gi...@apache.org> on 2023/10/02 18:46:57 UTC

Re: [PR] Field writers for numerical arrays (druid)

gianm commented on code in PR #14900:
URL: https://github.com/apache/druid/pull/14900#discussion_r1341495434


##########
processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java:
##########
@@ -172,6 +174,40 @@ public static List<ByteBuffer> getUtf8ByteBuffersFromStringArraySelector(
     return retVal;
   }
 
+  @Nullable
+  public static List<? extends Number> getNumericArrayFromNumericArray(Object row)

Review Comment:
   This method needs some javadoc, because the name is unclear. ("Numeric array from numeric array"? Sounds like it doesn't do anything)
   
   Also, reading through the impl I think that `numericArrayAsList` would be a better name.



##########
processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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

Review Comment:
   Please include a test case with `Float.NaN`, and a test case with with a `null` mixed in with nonnull doubles. (I see one for `null` by itself, but I don't see a mixture of null and nonnull.) Only include the `null` if `NullHandling.sqlCompatible()` though.



##########
processing/src/main/java/org/apache/druid/frame/read/FrameReader.java:
##########
@@ -96,11 +115,28 @@ public static FrameReader create(final RowSignature signature)
               signature.getColumnName(columnNumber)
           );
 
-      columnReaders.add(FrameColumnReaders.create(columnNumber, columnType));
       fieldReaders.add(FieldReaders.create(signature.getColumnName(columnNumber), columnType));
+
+      // If we encounter a numeric array type, then don't throw the error immediately since the reader can be used to
+      // read only the ROW_BASED frames. Rather, set the optional, and throw the appropriate error message when the reader
+      // tries to read COLUMNAR frame. This should go away once the COLUMNAR frames also start supporting the numeric
+      // array
+      if (columnType.getType() == ValueType.ARRAY

Review Comment:
   nit: the conditional would be more readable if the element type was extracted from the condition.



##########
processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java:
##########
@@ -0,0 +1,223 @@
+/*
+ * 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

Review Comment:
   Please include a test case with `Double.NaN`, and a test case with with a `null` mixed in with nonnull doubles. (I see one for `null` by itself, but I don't see a mixture of null and nonnull.) Only include the `null` if `NullHandling.sqlCompatible()` though.



##########
processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.frame.field;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+
+/**
+ * FieldWriter for numeric datatypes. The parent class does the null handling for the underlying data, while
+ * the individual subclasses write the individual element (long, float or double type). This also allows for a clean
+ * reuse of the readers and writers between the numeric types and also allowing the array writers ({@link NumericArrayFieldWriter})
+ * to use these methods directly without duplication
+ *
+ * Format:
+ *  - 1 byte: Whether the following value is null or not. Take a look at the note on the indicator bytes.
+ *  - X bytes: Encoded value of the selector, or the default value if it is null. X denotes the size of the numeric value
+ *
+ * Indicator bytes for denoting whether the element is null or not null changes depending on whether the writer is used
+ * to write the data for individual value (like LONG) or for an element of an array (like ARRAY<LONG>). This is because
+ * array support for the numeric types was added later and by then the field writers for individual fields were using
+ * 0x00 to denote the null byte, which is reserved for denoting the array end when we are writing the elements as part
+ * of the array instead. (0x00 is used for array end because it helps in preserving the byte comparison property of the
+ * numeric array field writers).
+ *
+ * Therefore, to preserve backward and forward compatibility, the individual element's writers were left unchanged,
+ * while the array's element's writers used 0x01 and 0x02 to denote null and non-null byte respectively
+ *
+ * Values produced by the writer are sortable without decoding
+ */

Review Comment:
   Please include `@see NumericFieldWriter for format details` on `DoubleFieldReader`, `DoubleFieldWriter`, etc, so people can find it easily.



##########
processing/src/main/java/org/apache/druid/frame/read/FrameReader.java:
##########
@@ -57,15 +61,28 @@ public class FrameReader
   // Field readers, for row-based frames.
   private final List<FieldReader> fieldReaders;
 
+  /**
+   * Currently, only ROW_BASED frames support numerical array columns, while the COLUMNAR frames donot. While creating
+   * a FrameReader, for types unsupported by COLUMNAR frames, we populate this field to denote that the FrameReader is
+   * "incomplete" and can't be used to read the columnar frame. However, the FrameReader performs as expected for the
+   * row-based frames.
+   * In short, this is a temporary measure till columnar frames support the numerical array types to punt the unsupported
+   * type check for the numerical arrays (for COLUMNAR frames only) at the usage time, rather than the creation time
+   */
+  private final Optional<Pair<String, ColumnType>> unsupportedColumnAndType;

Review Comment:
   Instead of this structure, IMO it'd keep `FrameReader` cleaner if we create a dummy column reader for the numeric array columns, which throws exceptions on `readColumn` and `readRACColumn`. Then the errors would happen only when we attempt to access those columns.



##########
processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.frame.field;
+
+import org.apache.datasketches.memory.Memory;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.query.extraction.ExtractionFn;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Reader class for the fields written by {@link NumericArrayFieldWriter}. See the Javadoc for the writer for more
+ * information on the format
+ *
+ * The numeric array fields are byte comparable
+ */
+public abstract class NumericArrayFieldReader implements FieldReader
+{
+  @Override
+  public DimensionSelector makeDimensionSelector(
+      Memory memory,
+      ReadableFieldPointer fieldPointer,
+      @Nullable ExtractionFn extractionFn
+  )
+  {
+    throw DruidException.defensive("Cannot call makeDimensionSelector on field of type ARRAY");
+  }
+
+  @Override
+  public boolean isNull(Memory memory, long position)
+  {
+    final byte firstByte = memory.getByte(position);
+    return firstByte == NumericArrayFieldWriter.NULL_ROW;
+  }
+
+  @Override
+  public boolean isComparable()
+  {
+    return true;
+  }
+
+  public abstract static class Selector<T extends Number> implements ColumnValueSelector

Review Comment:
   Please add some javadoc explaining the purpose of this class, including javadoc for the methods `getIndividualValueAtMemory` and `getIndividualFieldSize`. It's pretty twisty-turny to read through the code due to the inheritance stack, so additional javadoc helps the reader figure out what is going on.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org