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

[GitHub] [druid] clintropolis opened a new pull request, #13803: nested columns + arrays = array columns!

clintropolis opened a new pull request, #13803:
URL: https://github.com/apache/druid/pull/13803

   ### Description
   _description TBD_
   
   changes:
   * add support for storing nested arrays of string, long, and double values as specialized nested columns instead of breaking them into separate element columns
   * nested column typic mimic behavior means that columns ingested with only root arrays of primitive values will be ARRAY typed columns
   * neat test stuff
   
   
   <img width="1242" alt="Screenshot 2023-02-02 at 6 54 05 PM" src="https://user-images.githubusercontent.com/1577461/218803387-971251c3-879d-4856-9583-bce1f165f680.png">
   
   <img width="995" alt="Screenshot 2023-02-07 at 2 21 33 PM" src="https://user-images.githubusercontent.com/1577461/218803657-af960ae6-c576-492c-8076-a9cd17e7e7e4.png">
   
   <img width="772" alt="Screenshot 2023-02-07 at 5 05 21 PM" src="https://user-images.githubusercontent.com/1577461/218803814-9fd5edaf-2d23-4daa-9e0e-38cf17bd0b30.png">
   
   
   #### Release note
   <!-- Give your best effort to summarize your changes in a couple of sentences aimed toward Druid users. 
   
   If your change doesn't have end user impact, you can skip this section.
   
   For tips about how to write a good release note, see [Release notes](https://github.com/apache/druid/blob/master/CONTRIBUTING.md#release-notes).
   
   -->
   
   
   <hr>
   
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] a release note entry in the PR description.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148808275


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );

Review Comment:
   this shouldn't ever happen, but I guess i can wrap it in an if



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136481488


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        // why not
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+        LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new LiteralFieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(eval);
       }
-      return fieldIndexer.processValue(fieldValue);
+      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedLiteral<?> processArrayOfLiteralsField(
+        ArrayList<NestedPathPart> fieldPath,
+        Object maybeArrayOfLiterals
+    )
+    {
+      final ExprEval<?> maybeLiteralArray = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+      if (maybeLiteralArray.type().isArray() && maybeLiteralArray.type().getElementType().isPrimitive()) {
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new LiteralFieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(maybeLiteralArray);
+      }
+      return null;

Review Comment:
   the contract of `processArrayOfLiteralsField` is supposed to return a `ProcessedLiteral` if and only if the value was an array of literals (it is marked `@Nullable`). `processLiteralField` is not nullable, and must always return a `ProcessedLiteral`.
   
   The `StructuredDataProcessor` code when processing some input and it encounters arrays will first attempt to `processArrayOfLiteralField`, if it returns something, it was an array, else it must instead process the array elements recursively. `processLiteralField` is called on everything that isn't a map or array.
   
   I'll see if i can clarify it better



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141628246


##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -380,9 +551,61 @@ public VectorObjectSelector makeVectorObjectSelector(List<NestedPathPart> path,
     if (fields.indexOf(field) >= 0) {
       BaseColumn col = getColumnHolder(field).getColumn();
       return col.makeVectorObjectSelector(readableOffset);
-    } else {
-      return NilVectorSelector.create(readableOffset);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
+
+        return new VectorObjectSelector()
+        {
+          private final Object[] elements = new Object[arraySelector.getMaxVectorSize()];
+          private int id = ReadableVectorInspector.NULL_ID;
+
+          @Override
+          public Object[] getObjectVector()
+          {
+            if (readableOffset.getId() != id) {
+              final Object[] delegate = arraySelector.getObjectVector();
+              for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {

Review Comment:
   agree this isn't very optimized, i was planning to introduce some form of array selector in a future PR to optimize selecting individual elements



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146925278


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -280,4 +320,122 @@ private <T> boolean allNull(Indexed<T> dimValues)
     }
     return true;
   }
+
+  public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
+  {
+    private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
+        (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek());
+
+    protected final PriorityQueue<PeekingIterator<int[]>> pQueue;
+    protected int counter;
+
+    public ArrayDictionaryMergingIterator(Iterable<Object[]>[] dimValueLookups, GlobalDictionaryIdLookup idLookup)
+    {
+      pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR);
+
+      for (Iterable<Object[]> dimValueLookup : dimValueLookups) {
+        if (dimValueLookup == null) {
+          continue;
+        }
+        final PeekingIterator<int[]> iter = Iterators.peekingIterator(
+            new IdLookupArrayIterator(idLookup, dimValueLookup.iterator())
+        );
+        if (iter.hasNext()) {
+          pQueue.add(iter);
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return !pQueue.isEmpty();
+    }
+
+    @Override
+    public int[] next()
+    {
+      PeekingIterator<int[]> smallest = pQueue.remove();
+      if (smallest == null) {
+        throw new NoSuchElementException();
+      }
+      final int[] value = smallest.next();
+      if (smallest.hasNext()) {
+        pQueue.add(smallest);
+      }
+
+      while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) {
+        PeekingIterator<int[]> same = pQueue.remove();
+        same.next();
+        if (same.hasNext()) {
+          pQueue.add(same);
+        }
+      }
+      counter++;
+
+      return value;
+    }
+
+    public int getCardinality()
+    {
+      return counter;
+    }
+
+    @Override
+    public void remove()
+    {
+      throw new UnsupportedOperationException("remove");
+    }
+  }
+
+  public static class IdLookupArrayIterator implements Iterator<int[]>
+  {
+    private final GlobalDictionaryIdLookup idLookup;
+    private final Iterator<Object[]> delegate;
+
+    public IdLookupArrayIterator(
+        GlobalDictionaryIdLookup idLookup,
+        Iterator<Object[]> delegate
+    )
+    {
+      this.idLookup = idLookup;
+      this.delegate = delegate;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return delegate.hasNext();
+    }
+
+    @Override
+    public int[] next()
+    {
+      final Object[] next = delegate.next();
+      if (next == null) {
+        return null;
+      }
+      final int[] newIdsWhoDis = new int[next.length];
+      for (int i = 0; i < next.length; i++) {
+        if (next[i] == null) {
+          newIdsWhoDis[i] = 0;
+        } else if (next[i] instanceof String) {
+          newIdsWhoDis[i] = idLookup.lookupString((String) next[i]);
+        } else if (next[i] instanceof Long) {
+          newIdsWhoDis[i] = idLookup.lookupLong((Long) next[i]);
+        } else if (next[i] instanceof Double) {
+          newIdsWhoDis[i] = idLookup.lookupDouble((Double) next[i]);
+        } else {
+          newIdsWhoDis[i] = -1;
+        }
+        Preconditions.checkArgument(
+            newIdsWhoDis[i] >= 0,
+            "unknown global id [%s] for value [%s]",
+            newIdsWhoDis[i],
+            next[i]
+        );

Review Comment:
   i think since this stuff is all working i'd like to save further changes/optimizations for a follow-up



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146893401


##########
processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java:
##########
@@ -41,6 +42,18 @@
 public class DelimitedInputFormat extends FlatTextInputFormat
 {
   public static final String TYPE_KEY = "tsv";
+
+  public static DelimitedInputFormat ofColumns(String... columns)
+  {
+    return new DelimitedInputFormat(
+        Arrays.asList(columns),
+        null,
+        null,
+        false,
+        false,
+        0
+    );
+  }

Review Comment:
   removed since there was only one caller



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141568494


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -86,93 +94,123 @@ public NestedDataColumnMerger(
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());

Review Comment:
   `[]` pls



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -86,93 +94,123 @@ public NestedDataColumnMerger(
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+        }
 
-    long dimStartTime = System.currentTimeMillis();
-
-    int numMergeIndex = 0;
-    GlobalDictionarySortedCollector sortedLookup = null;
-    final Indexed[] sortedLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+        boolean allNulls = dimValues == null || allNull(dimValues.getSortedStrings()) &&
+                                                allNull(dimValues.getSortedLongs()) &&
+                                                allNull(dimValues.getSortedDoubles()) &&
+                                                dimValues.getArrayCardinality() == 0;

Review Comment:
   This seems like a nice check to delegate to `GlobalDictionarySortedCollector` instead of implementing here?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -234,7 +272,9 @@ private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNest
     return new GlobalDictionarySortedCollector(
         new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()),
         column.getLongDictionary(),
-        column.getDoubleDictionary()
+        column.getDoubleDictionary(),
+        column.getArraysIterable(),
+        column.getArrayDictionary().size()

Review Comment:
   Why the 2 argument set of `Iterable()` and `size()` instead of a single collection-style object like the others?



##########
processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java:
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * {@link Indexed} specialized for storing int arrays, which must be sorted and unique, using 'front coding'.
+ *
+ * Front coding is a type of delta encoding, where sorted values are grouped into buckets. The first value of the bucket
+ * is written entirely, and remaining values are stored as a pair of an integer which indicates how much of the first
+ * int array of the bucket to use as a prefix, followed by the remaining ints after the prefix to complete the value.
+ *
+ * front coded indexed layout:
+ * | version | bucket size | has null? | number of values | size of "offsets" + "buckets" | "offsets" | "buckets" |
+ * | ------- | ----------- | --------- | ---------------- | ----------------------------- | --------- | --------- |
+ * |    byte |        byte |      byte |        vbyte int |                     vbyte int |     int[] |  bucket[] |
+ *
+ * "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access.
+ *
+ * bucket layout:
+ * | first value | prefix length | fragment | ... | prefix length | fragment |
+ * | ----------- | ------------- | -------- | --- | ------------- | -------- |
+ * |       int[] |     vbyte int |    int[] | ... |     vbyte int |    int[] |
+ *
+ * int array layout:
+ * | length      |  ints |
+ * | ----------- | ----- |
+ * |   vbyte int | int[] |
+ *
+ *
+ * Getting a value first picks the appropriate bucket, finds its offset in the underlying buffer, then scans the bucket
+ * values to seek to the correct position of the value within the bucket in order to reconstruct it using the prefix
+ * length.
+ *
+ * Finding the index of a value involves binary searching the first values of each bucket to find the correct bucket,
+ * then a linear scan within the bucket to find the matching value (or negative insertion point -1 for values that
+ * are not present).
+ *
+ * The value iterator reads an entire bucket at a time, reconstructing the values into an array to iterate within the
+ * bucket before moving onto the next bucket as the iterator is consumed.
+ *
+ * This class is not thread-safe since during operation modifies positions of a shared buffer.
+ */
+public final class FrontCodedIntArrayIndexed implements Indexed<int[]>
+{
+  public static Supplier<FrontCodedIntArrayIndexed> read(ByteBuffer buffer, ByteOrder ordering)
+  {
+    final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering);
+    final byte version = orderedBuffer.get();
+    Preconditions.checkArgument(version == 0, "only V0 exists, encountered " + version);
+    final int bucketSize = Byte.toUnsignedInt(orderedBuffer.get());
+    final boolean hasNull = NullHandling.IS_NULL_BYTE == orderedBuffer.get();
+    final int numValues = VByte.readInt(orderedBuffer);
+    // size of offsets + values
+    final int size = VByte.readInt(orderedBuffer);
+    final int offsetsPosition = orderedBuffer.position();
+    // move position to end of buffer
+    buffer.position(offsetsPosition + size);
+
+    return () -> new FrontCodedIntArrayIndexed(
+        buffer,
+        ordering,
+        bucketSize,
+        numValues,
+        hasNull,
+        offsetsPosition
+    );
+  }
+
+  private final ByteBuffer buffer;
+  private final int adjustedNumValues;
+  private final int adjustIndex;
+  private final int bucketSize;
+  private final int numBuckets;
+  private final int div;
+  private final int rem;
+  private final int offsetsPosition;
+  private final int bucketsPosition;
+  private final boolean hasNull;
+  private final int lastBucketNumValues;
+
+  private FrontCodedIntArrayIndexed(
+      ByteBuffer buffer,
+      ByteOrder order,
+      int bucketSize,
+      int numValues,
+      boolean hasNull,
+      int offsetsPosition
+  )
+  {
+    if (Integer.bitCount(bucketSize) != 1) {
+      throw new ISE("bucketSize must be a power of two but was[%,d]", bucketSize);
+    }
+    this.buffer = buffer.asReadOnlyBuffer().order(order);
+    this.bucketSize = bucketSize;
+    this.hasNull = hasNull;
+
+    this.numBuckets = (int) Math.ceil((double) numValues / (double) bucketSize);
+    this.adjustIndex = hasNull ? 1 : 0;
+    this.adjustedNumValues = numValues + adjustIndex;
+    this.div = Integer.numberOfTrailingZeros(bucketSize);
+    this.rem = bucketSize - 1;
+    this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem;
+    this.offsetsPosition = offsetsPosition;
+    this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES);
+  }
+
+  @Override
+  public int size()
+  {
+    return adjustedNumValues;
+  }
+
+  @Nullable
+  @Override
+  public int[] get(int index)
+  {
+    if (hasNull && index == 0) {
+      return null;
+    }
+    Indexed.checkIndex(index, adjustedNumValues);
+
+    // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust
+    // the index

Review Comment:
   isn't `null` == `0`?  Why does `no negative values` mean that `null` is not stored in the bucket?



##########
processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.segment.nested;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+
+public class ArrayOfLiteralsFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<int[]>
+{
+
+  protected ArrayOfLiteralsFieldColumnWriter(
+      String columnName,
+      String fieldName,
+      SegmentWriteOutMedium segmentWriteOutMedium,
+      IndexSpec indexSpec,
+      GlobalDictionaryIdLookup globalDictionaryIdLookup
+  )
+  {
+    super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup);
+  }
+
+  @Override
+  int[] processValue(int row, Object value)
+  {
+    if (value instanceof Object[]) {

Review Comment:
   Would it be good to have a thing that handles `List<>` as well?



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -327,17 +390,58 @@ public void close()
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
     if (fields.indexOf(field) >= 0) {
       DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);

Review Comment:
   Would be nice to include the elementNumber that was asked for here.



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -394,11 +617,127 @@ public VectorValueSelector makeVectorValueSelector(List<NestedPathPart> path, Re
     if (fields.indexOf(field) >= 0) {
       BaseColumn col = getColumnHolder(field).getColumn();
       return col.makeVectorValueSelector(readableOffset);
-    } else {
-      return NilVectorSelector.create(readableOffset);
     }
-  }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);

Review Comment:
   After seeing this same pre-amble for the 4th time now in the code, I think this code would benefit from doing this pre-amble to make an `ArrayBasedSelectorThingieFactory`, which could then have factory methods for each of the object types (i.e. `makeDimensionSelector()`, `makeColumnValueSelector`, `makeVectoryObjectSelector`, `makeVectorValueSelector`).  This would allow you to have a single method that does this pre-amble and then use that result to return the thing being asked for.



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -380,9 +551,61 @@ public VectorObjectSelector makeVectorObjectSelector(List<NestedPathPart> path,
     if (fields.indexOf(field) >= 0) {
       BaseColumn col = getColumnHolder(field).getColumn();
       return col.makeVectorObjectSelector(readableOffset);
-    } else {
-      return NilVectorSelector.create(readableOffset);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
+
+        return new VectorObjectSelector()
+        {
+          private final Object[] elements = new Object[arraySelector.getMaxVectorSize()];
+          private int id = ReadableVectorInspector.NULL_ID;
+
+          @Override
+          public Object[] getObjectVector()
+          {
+            if (readableOffset.getId() != id) {
+              final Object[] delegate = arraySelector.getObjectVector();
+              for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {

Review Comment:
   If I'm reading this correctly, we are loading up a full `Object[]` only to then throw away everything except the `elementNumber` that we want.  That's quite wasteful, especially given that we know the element that we care about when we are constructing the objects that will do the read.  Let's push down the element number all the way to the thing that's doing the read and only read the minimum amount of stuff into memory.



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -104,6 +108,32 @@ public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList
       }
       return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
     }
+
+    @Nullable
+    @Override
+    public ProcessedLiteral<?> processArrayOfLiteralsField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable Object maybeArrayOfLiterals
+    )
+    {
+      ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
+            NestedPathFinder.toNormalizedJsonPath(fieldPath)
+        );
+        if (writer != null) {
+          try {
+            writer.addValue(rowCount, eval.value());
+            // serializer doesn't use size estimate
+            return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          }
+          catch (IOException e) {
+            throw new RuntimeException(":(");

Review Comment:
   This is not only throwing away the IOException, it's not even attempting to generate a nice message.  `:(` :P



##########
processing/src/test/resources/nested-array-test-data.json:
##########
@@ -0,0 +1,14 @@
+{"timestamp": "2023-01-01T00:00:00", "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],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "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], "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-01T00:00:00", "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],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],   "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],     "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-01T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,              "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "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],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "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], "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-02T00:00:00", "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],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],   "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],     "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-02T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,              "arrayObject":[{"x": 1},{"x":2}]}

Review Comment:
   What's the expectation of how arrays of arrays are handled?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -280,4 +320,122 @@ private <T> boolean allNull(Indexed<T> dimValues)
     }
     return true;
   }
+
+  public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
+  {
+    private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
+        (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek());
+
+    protected final PriorityQueue<PeekingIterator<int[]>> pQueue;
+    protected int counter;
+
+    public ArrayDictionaryMergingIterator(Iterable<Object[]>[] dimValueLookups, GlobalDictionaryIdLookup idLookup)
+    {
+      pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR);
+
+      for (Iterable<Object[]> dimValueLookup : dimValueLookups) {
+        if (dimValueLookup == null) {
+          continue;
+        }
+        final PeekingIterator<int[]> iter = Iterators.peekingIterator(
+            new IdLookupArrayIterator(idLookup, dimValueLookup.iterator())
+        );
+        if (iter.hasNext()) {
+          pQueue.add(iter);
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return !pQueue.isEmpty();
+    }
+
+    @Override
+    public int[] next()
+    {
+      PeekingIterator<int[]> smallest = pQueue.remove();
+      if (smallest == null) {
+        throw new NoSuchElementException();
+      }
+      final int[] value = smallest.next();
+      if (smallest.hasNext()) {
+        pQueue.add(smallest);
+      }
+
+      while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) {
+        PeekingIterator<int[]> same = pQueue.remove();
+        same.next();
+        if (same.hasNext()) {
+          pQueue.add(same);
+        }
+      }
+      counter++;
+
+      return value;
+    }
+
+    public int getCardinality()
+    {
+      return counter;
+    }
+
+    @Override
+    public void remove()
+    {
+      throw new UnsupportedOperationException("remove");
+    }
+  }
+
+  public static class IdLookupArrayIterator implements Iterator<int[]>
+  {
+    private final GlobalDictionaryIdLookup idLookup;
+    private final Iterator<Object[]> delegate;
+
+    public IdLookupArrayIterator(
+        GlobalDictionaryIdLookup idLookup,
+        Iterator<Object[]> delegate
+    )
+    {
+      this.idLookup = idLookup;
+      this.delegate = delegate;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return delegate.hasNext();
+    }
+
+    @Override
+    public int[] next()
+    {
+      final Object[] next = delegate.next();
+      if (next == null) {
+        return null;
+      }
+      final int[] newIdsWhoDis = new int[next.length];
+      for (int i = 0; i < next.length; i++) {
+        if (next[i] == null) {
+          newIdsWhoDis[i] = 0;
+        } else if (next[i] instanceof String) {
+          newIdsWhoDis[i] = idLookup.lookupString((String) next[i]);
+        } else if (next[i] instanceof Long) {
+          newIdsWhoDis[i] = idLookup.lookupLong((Long) next[i]);
+        } else if (next[i] instanceof Double) {
+          newIdsWhoDis[i] = idLookup.lookupDouble((Double) next[i]);
+        } else {
+          newIdsWhoDis[i] = -1;
+        }
+        Preconditions.checkArgument(
+            newIdsWhoDis[i] >= 0,
+            "unknown global id [%s] for value [%s]",
+            newIdsWhoDis[i],
+            next[i]
+        );

Review Comment:
   Given that the global dictionaries, once merged, will be in type-sorted order, do we really need to convert back into the actual values instead of just converting the dictionary id?



##########
processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java:
##########
@@ -101,7 +101,7 @@ public Sequence<Cursor> makeCursors(
           Cursor retVal = cursor;
           ColumnCapabilities capabilities = cursor.getColumnSelectorFactory().getColumnCapabilities(dimensionToUnnest);
           if (capabilities != null) {
-            if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {
+            if (!capabilities.isArray() && capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {

Review Comment:
   This is going to stop the unnest cursor from being able to take advantage of dictionary-based processing for columns that are string arrays and could use the dictionary based processing.  Why can't it use a dimension selector, the column value selectors are, generally speaking, significantly slower for processing compared to when the dimension selector can be used, so continuing to use a dimension selector where possible should be a goal.



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -86,93 +94,123 @@ public NestedDataColumnMerger(
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+        }
 
-    long dimStartTime = System.currentTimeMillis();
-
-    int numMergeIndex = 0;
-    GlobalDictionarySortedCollector sortedLookup = null;
-    final Indexed[] sortedLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+        boolean allNulls = dimValues == null || allNull(dimValues.getSortedStrings()) &&
+                                                allNull(dimValues.getSortedLongs()) &&
+                                                allNull(dimValues.getSortedDoubles()) &&
+                                                dimValues.getArrayCardinality() == 0;
+        sortedLookup = dimValues;
+        if (!allNulls) {
+          sortedLookups[i] = dimValues.getSortedStrings();
+          sortedLongLookups[i] = dimValues.getSortedLongs();
+          sortedDoubleLookups[i] = dimValues.getSortedDoubles();
+          sortedArrayLookups[i] = dimValues.getSortedArrays();
+          numMergeIndex++;
+        }
+      }
 
-    final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+      descriptorBuilder = new ColumnDescriptor.Builder();
 
-    for (int i = 0; i < adapters.size(); i++) {
-      final IndexableAdapter adapter = adapters.get(i);
-      final GlobalDictionarySortedCollector dimValues;
-      if (adapter instanceof IncrementalIndexAdapter) {
-        dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
-      } else if (adapter instanceof QueryableIndexIndexableAdapter) {
-        dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+      final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
+          name,
+          indexSpec,
+          segmentWriteOutMedium,
+          progressIndicator,
+          closer
+      );
+      serializer = defaultSerializer;
+
+      final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
+                                                                     .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
+                                                                     .withDelegate(serializer)
+                                                                     .build();
+      descriptorBuilder.setValueType(ValueType.COMPLEX)
+                       .setHasMultipleValues(false)
+                       .addSerde(partSerde);
+
+      defaultSerializer.open();
+      defaultSerializer.serializeFields(mergedFields);
+
+      int stringCardinality;
+      int longCardinality;
+      int doubleCardinality;
+      int arrayCardinality;
+      if (numMergeIndex == 1) {
+        defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings());
+        defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs());
+        defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles());
+        defaultSerializer.serializeArrayDictionary(() -> new ArrayDictionaryMergingIterator(
+            sortedArrayLookups,
+            defaultSerializer.getGlobalLookup()
+        ));

Review Comment:
   Why can't this one just be `sortedLookup.getSortedArrays()` like the other 3?



##########
processing/src/test/java/org/apache/druid/segment/IndexBuilder.java:
##########
@@ -267,4 +395,44 @@ private static IncrementalIndex buildIncrementalIndexWithRows(
     }
     return incrementalIndex;
   }
+
+  private static IncrementalIndex buildIncrementalIndexWithInputSource(
+      IncrementalIndexSchema schema,
+      InputSource inputSource,
+      InputFormat inputFormat,
+      @Nullable TransformSpec transformSpec,
+      File inputSourceTmpDir,
+      int maxRows
+  )
+  {
+    Preconditions.checkNotNull(schema, "schema");
+    Preconditions.checkNotNull(inputSource, "inputSource");
+    Preconditions.checkNotNull(inputFormat, "inputFormat");
+    Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir");
+
+    final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder()
+        .setIndexSchema(schema)
+        .setMaxRowCount(maxRows)
+        .build();
+    TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE;

Review Comment:
   spell-check



##########
processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.segment.nested;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+
+public class ArrayOfLiteralsFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<int[]>
+{
+
+  protected ArrayOfLiteralsFieldColumnWriter(
+      String columnName,
+      String fieldName,
+      SegmentWriteOutMedium segmentWriteOutMedium,
+      IndexSpec indexSpec,
+      GlobalDictionaryIdLookup globalDictionaryIdLookup
+  )
+  {
+    super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup);
+  }
+
+  @Override
+  int[] processValue(int row, Object value)
+  {
+    if (value instanceof Object[]) {
+      Object[] array = (Object[]) value;
+      final int[] newIdsWhoDis = new int[array.length];
+      for (int i = 0; i < array.length; i++) {
+        if (array[i] == null) {
+          newIdsWhoDis[i] = 0;
+        } else if (array[i] instanceof String) {
+          newIdsWhoDis[i] = globalDictionaryIdLookup.lookupString((String) array[i]);
+        } else if (array[i] instanceof Long) {
+          newIdsWhoDis[i] = globalDictionaryIdLookup.lookupLong((Long) array[i]);
+        } else if (array[i] instanceof Double) {
+          newIdsWhoDis[i] = globalDictionaryIdLookup.lookupDouble((Double) array[i]);
+        } else {
+          newIdsWhoDis[i] = -1;
+        }
+        Preconditions.checkArgument(newIdsWhoDis[i] >= 0, "unknown global id [%s] for value [%s]", newIdsWhoDis[i], array[i]);
+        arrayElements.computeIfAbsent(
+            newIdsWhoDis[i],
+            (id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()
+        ).add(row);
+      }
+      return newIdsWhoDis;
+    }
+    return null;

Review Comment:
   Is `null` the correct thing here?  Is there any way to structure the calling code such that the correct behavior when it got something that wasn't the expected type is to throw an exception?



##########
processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java:
##########
@@ -0,0 +1,455 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.ThreadLocalRandom;
+
+@RunWith(Parameterized.class)
+public class FrontCodedIntArrayIndexedTest
+{
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> constructorFeeder()
+  {
+    return ImmutableList.of(new Object[]{ByteOrder.LITTLE_ENDIAN}, new Object[]{ByteOrder.BIG_ENDIAN});
+  }
+
+  private final ByteOrder order;
+
+  public FrontCodedIntArrayIndexedTest(ByteOrder byteOrder)
+  {
+    this.order = byteOrder;
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexed() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3, 4});
+    values.add(new int[]{1, 2, 1});
+
+    fillBuffer(buffer, values, 4);
+
+    buffer.position(0);
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Iterator<int[]> indexedIterator = codedIndexed.iterator();
+    Iterator<int[]> expectedIterator = values.iterator();
+    int ctr = 0;
+    while (expectedIterator.hasNext() && indexedIterator.hasNext()) {
+      final int[] expectedNext = expectedIterator.next();
+      final int[] next = indexedIterator.next();
+      Assert.assertArrayEquals(expectedNext, next);
+      Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+      ctr++;
+    }
+    Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+  }
+
+
+  @Test
+  public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3, 4});
+    values.add(new int[]{1, 2, 1});
+    fillBuffer(buffer, values, 16);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Iterator<int[]> expectedIterator = values.iterator();
+    Iterator<int[]> indexedIterator = codedIndexed.iterator();
+    int ctr = 0;
+    while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+      final int[] expectedNext = expectedIterator.next();
+      final int[] next = indexedIterator.next();
+      Assert.assertArrayEquals(expectedNext, next);
+      Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+      ctr++;
+    }
+    Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedBigger() throws IOException
+  {
+    final int sizeBase = 10000;
+    final int bucketSize = 16;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 24).order(order);
+    for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) {
+      final TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+      while (values.size() < sizeBase + sizeAdjust) {
+        int length = ThreadLocalRandom.current().nextInt(10);
+        final int[] val = new int[length];
+        for (int j = 0; j < length; j++) {
+          val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+        }
+        values.add(val);
+      }
+      fillBuffer(buffer, values, bucketSize);
+
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> indexedIterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = indexedIterator.next();
+        Assert.assertArrayEquals(expectedNext, next);
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+      Assert.assertEquals(ctr, sizeBase + sizeAdjust);
+    }
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedBiggerWithNulls() throws IOException
+  {
+    final int sizeBase = 10000;
+    final int bucketSize = 16;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order);
+    for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) {
+      TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+      values.add(null);
+      while (values.size() < sizeBase + sizeAdjust + 1) {
+        int length = ThreadLocalRandom.current().nextInt(10);
+        final int[] val = new int[length];
+        for (int j = 0; j < length; j++) {
+          val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+        }
+        values.add(val);
+      }
+      fillBuffer(buffer, values, bucketSize);
+
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> indexedIterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = indexedIterator.next();
+        Assert.assertArrayEquals(expectedNext, next);
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+      Assert.assertEquals(ctr, sizeBase + sizeAdjust + 1);
+    }
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedIndexOf() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 3, 4});
+
+    fillBuffer(buffer, values, 4);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+    Assert.assertEquals(-1, codedIndexed.indexOf(new int[]{1}));
+    Assert.assertEquals(0, codedIndexed.indexOf(new int[]{1, 2}));
+    Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2, 1}));
+    Assert.assertEquals(-3, codedIndexed.indexOf(new int[]{1, 2, 2}));
+    Assert.assertEquals(4, codedIndexed.indexOf(new int[]{1, 3}));
+    Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{1, 4, 4}));
+    Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{9, 1, 1}));
+  }
+
+
+  @Test
+  public void testFrontCodedIntArrayIndexedIndexOfWithNull() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(null);
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 3, 4});
+    fillBuffer(buffer, values, 4);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+    Assert.assertEquals(0, codedIndexed.indexOf(null));
+    Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1}));
+    Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2}));
+    Assert.assertEquals(2, codedIndexed.indexOf(new int[]{1, 2, 1}));
+    Assert.assertEquals(-4, codedIndexed.indexOf(new int[]{1, 2, 2}));
+    Assert.assertEquals(5, codedIndexed.indexOf(new int[]{1, 3}));
+    Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{1, 4, 4}));
+    Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{9, 1, 1}));
+  }
+
+
+  @Test
+  public void testFrontCodedOnlyNull() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    List<int[]> theList = Collections.singletonList(null);
+    fillBuffer(buffer, theList, 4);
+
+    buffer.position(0);
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertNull(codedIndexed.get(0));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(theList.size()));
+
+    Assert.assertEquals(0, codedIndexed.indexOf(null));
+    Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1, 2, 3, 4}));
+
+    Iterator<int[]> iterator = codedIndexed.iterator();
+    Assert.assertTrue(iterator.hasNext());
+    Assert.assertNull(iterator.next());
+    Assert.assertFalse(iterator.hasNext());
+  }
+
+  @Test
+  public void testFrontCodedEmpty() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order);
+    List<int[]> theList = Collections.emptyList();
+    fillBuffer(buffer, theList, 4);
+
+    buffer.position(0);
+    FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertEquals(0, codedUtf8Indexed.size());
+    Throwable t = Assert.assertThrows(IAE.class, () -> codedUtf8Indexed.get(0));
+    Assert.assertEquals("Index[0] >= size[0]", t.getMessage());
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(theList.size()));
+
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(null));
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(StringUtils.toUtf8ByteBuffer("hello")));
+
+    Iterator<ByteBuffer> utf8Iterator = codedUtf8Indexed.iterator();
+    Assert.assertFalse(utf8Iterator.hasNext());
+  }
+
+  @Test
+  public void testBucketSizes() throws IOException
+  {
+    final int numValues = 10000;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order);
+    final int[] bucketSizes = new int[]{
+        1,
+        1 << 1,
+        1 << 2,
+        1 << 3,
+        1 << 4,
+        1 << 5,
+        1 << 6,
+        1 << 7
+    };
+
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(null);
+    while (values.size() < numValues + 1) {
+      int length = ThreadLocalRandom.current().nextInt(10);
+      final int[] val = new int[length];
+      for (int j = 0; j < length; j++) {
+        val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+      }
+      values.add(val);
+    }
+    for (int bucketSize : bucketSizes) {
+      fillBuffer(buffer, values, bucketSize);
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> iterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (iterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = iterator.next();
+        Assert.assertArrayEquals(expectedNext, next);
+
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), iterator.hasNext());
+      Assert.assertEquals(ctr, numValues + 1);
+    }
+  }
+
+  @Test
+  public void testBadBucketSize()
+  {
+    OnHeapMemorySegmentWriteOutMedium medium = new OnHeapMemorySegmentWriteOutMedium();
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            0
+        )
+    );
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            15
+        )
+    );
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            256
+        )
+    );
+  }
+
+  private static long fillBuffer(ByteBuffer buffer, Iterable<int[]> sortedIterable, int bucketSize) throws IOException

Review Comment:
   naming nit: how about `persistToBuffer`.  `fillBuffer` initially made me think that this was just filling the buffer directly, not that it was actually using the "normal" persist logic with the writer.



##########
processing/src/test/java/org/apache/druid/segment/IndexBuilder.java:
##########
@@ -267,4 +395,44 @@ private static IncrementalIndex buildIncrementalIndexWithRows(
     }
     return incrementalIndex;
   }
+
+  private static IncrementalIndex buildIncrementalIndexWithInputSource(
+      IncrementalIndexSchema schema,
+      InputSource inputSource,
+      InputFormat inputFormat,
+      @Nullable TransformSpec transformSpec,
+      File inputSourceTmpDir,
+      int maxRows
+  )
+  {
+    Preconditions.checkNotNull(schema, "schema");
+    Preconditions.checkNotNull(inputSource, "inputSource");
+    Preconditions.checkNotNull(inputFormat, "inputFormat");
+    Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir");
+
+    final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder()
+        .setIndexSchema(schema)
+        .setMaxRowCount(maxRows)
+        .build();
+    TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE;
+    InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null);
+    InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir);
+    InputSourceReader transformingReader = tranformer.decorate(reader);
+    try (CloseableIterator<InputRow> rowIterator = transformingReader.read()) {
+      while (rowIterator.hasNext()) {
+        incrementalIndex.add(rowIterator.next());

Review Comment:
   When `maxRows` is hit, are we expecting an exception?  Generally speaking, setting the `maxRows` on the tests is done as a way to force running queries against multiple segments, so I had expected to see a check for the numRows and incremental persists in anything that takes `maxRows`.



##########
sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java:
##########
@@ -705,6 +705,23 @@ public void testQuery(
         .run();
   }
 
+  public void testQuery(
+      final String sql,
+      final Map<String, Object> queryContext,
+      final List<Query<?>> expectedQueries,
+      final List<Object[]> expectedResults,
+      final RowSignature expectedResultSignature
+  )
+  {
+    testBuilder()
+        .sql(sql)
+        .queryContext(queryContext)
+        .expectedQueries(expectedQueries)
+        .expectedResults(expectedResults)
+        .expectedSignature(expectedResultSignature)
+        .run();
+  }

Review Comment:
   Part of me wonders if, instead of adding more functions with extra arguments to their signatures, we should switch the call-sites to using the builder directly?



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -327,17 +390,58 @@ public void close()
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
     if (fields.indexOf(field) >= 0) {
       DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);

Review Comment:
   Might as well cast and assign to a `NestedPathArrayElement` here.



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -327,17 +390,58 @@ public void close()
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
     if (fields.indexOf(field) >= 0) {
       DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {

Review Comment:
   Looking over the code, we are being very gratuitous with calls to `fields.indexOf()`.  My read is that in the case that this matches and we actually load the column, we will have binary searched for the exact same value a minimum of 3 times.  Once here, then twice in `readNestedFieldColumn`.  Please refactor all of the various methods to take the actual `int` id of what they want to read and push the code towards a singular `indexOf()` where we push around the actual `int fieldId` after that.



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -327,17 +390,58 @@ public void close()
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
     if (fields.indexOf(field) >= 0) {
       DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        ColumnValueSelector<?> arraySelector = col.makeColumnValueSelector(readableOffset);
+        return new BaseSingleValueDimensionSelector()
+        {
+          @Nullable
+          @Override
+          protected String getValue()
+          {
+            Object o = arraySelector.getObject();
+            if (o instanceof Object[]) {
+              Object[] array = (Object[]) o;
+              if (elementNumber < array.length) {
+                Object element = array[elementNumber];
+                if (element == null) {
+                  return null;
+                }
+                return String.valueOf(element);
+              }
+            }
+            return null;
+          }
+
+          @Override
+          public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+          {
+            arraySelector.inspectRuntimeShape(inspector);
+          }
+        };

Review Comment:
   We are being asked for a DimensionSelector here, why convert it to a ColumnValueSelector and pay the cost of grabbing objects when we could very easily just return a DimensionSelector?  If we are being asked for a `DimensionSelector` let's return a proper `DimensionSelector`. I understand that the interface is very String-specific right now, but let's not hobble this implementation just because we want to make a change to that interface at some point in the future.



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -349,9 +453,76 @@ public ColumnValueSelector<?> makeColumnValueSelector(List<NestedPathPart> path,
     if (fields.indexOf(field) >= 0) {
       BaseColumn col = getColumnHolder(field).getColumn();
       return col.makeColumnValueSelector(readableOffset);
-    } else {
-      return NilColumnValueSelector.instance();
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset);

Review Comment:
   Various commentary in the `makeDimensionSelector` implementation applies here too.



##########
processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java:
##########
@@ -42,45 +47,107 @@
   private final ComparatorDimensionDictionary<String> stringDictionary;
   private final ComparatorDimensionDictionary<Long> longDictionary;
   private final ComparatorDimensionDictionary<Double> doubleDictionary;
+  private final Set<Object[]> stringArrays;
+  private final Set<Object[]> longArrays;
+  private final Set<Object[]> doubleArrays;

Review Comment:
   Why this separation by type?  Isn't is just an `array` and the ids are any other scalar thing in the global dictionary?



##########
processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java:
##########
@@ -231,9 +252,16 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException
       @Override
       public long getSerializedSize() throws IOException
       {
+        final long arraySize;
+        if (arrayElements.size() > 0) {
+          arraySize = arrayElementDictionaryWriter.getSerializedSize() + arrayElementIndexWriter.getSerializedSize();
+        } else {
+          arraySize = 0;
+        }

Review Comment:
   Random thought when reading this code: should we keep track of the maximum length of array that we see?  It could be used to quickly filter out things that are accessing elementIds that are larger than anything we store.



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -215,14 +263,24 @@ public void serializeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeS
               indexSpec,
               globalDictionaryIdLookup
           );
-        } else {
+        } else if (Types.is(type, ValueType.DOUBLE)) {
           writer = new DoubleFieldColumnWriter(
               name,
               fieldFileName,
               segmentWriteOutMedium,
               indexSpec,
               globalDictionaryIdLookup
           );
+        } else if (Types.is(type, ValueType.ARRAY)) {
+          writer = new ArrayOfLiteralsFieldColumnWriter(
+              name,
+              fieldFileName,
+              segmentWriteOutMedium,
+              indexSpec,
+              globalDictionaryIdLookup
+          );
+        } else {
+          throw new IllegalArgumentException("wtf");

Review Comment:
   How about make it a `UOE` and include the type that caused us to get here :)



##########
processing/src/test/java/org/apache/druid/segment/IndexBuilder.java:
##########
@@ -157,27 +241,71 @@ public QueryableIndex buildMMappedIndex()
 
   public QueryableIndex buildMMappedMergedIndex()
   {
-    IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory);
     Preconditions.checkNotNull(tmpDir, "tmpDir");
-
     final List<QueryableIndex> persisted = new ArrayList<>();
-    try {
-      for (int i = 0; i < rows.size(); i += ROWS_PER_INDEX_FOR_MERGING) {
+    if (inputSource != null) {
+      Preconditions.checkNotNull(inputSource, "inputSource");
+      Preconditions.checkNotNull(inputFormat, "inputFormat");
+      Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir");
+
+      TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE;

Review Comment:
   spell-check: `transformer`



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -251,33 +313,78 @@ public void serializeStringDictionary(Iterable<String> dictionaryValues) throws
       dictionaryWriter.write(value);
       globalDictionaryIdLookup.addString(value);
     }
+    stringDictionarySerialized = true;
   }
 
   public void serializeLongDictionary(Iterable<Long> dictionaryValues) throws IOException
   {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing long dictionary for column [%s]", name);
+    }
+    if (longDictionarySerialized) {
+      throw new ISE("Long dictionary already serialized for column [%s], cannot serialize again", name);
+    }
     for (Long value : dictionaryValues) {
       if (value == null) {
         continue;
       }
       longDictionaryWriter.write(value);
       globalDictionaryIdLookup.addLong(value);
     }
+    longDictionarySerialized = true;
   }
 
   public void serializeDoubleDictionary(Iterable<Double> dictionaryValues) throws IOException
   {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing double dictionary for column [%s]", name);
+    }
+    if (!longDictionarySerialized) {
+      throw new ISE("Must serialize long value dictionary before serializing double dictionary for column [%s]", name);
+    }
+    if (doubleDictionarySerialized) {
+      throw new ISE("Double dictionary already serialized for column [%s], cannot serialize again", name);
+    }
     for (Double value : dictionaryValues) {
       if (value == null) {
         continue;
       }
       doubleDictionaryWriter.write(value);
       globalDictionaryIdLookup.addDouble(value);
     }
+    doubleDictionarySerialized = true;
+  }
+
+  public void serializeArrayDictionary(Iterable<int[]> dictionaryValues) throws IOException
+  {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (!longDictionarySerialized) {
+      throw new ISE("Must serialize long value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (!doubleDictionarySerialized) {
+      throw new ISE("Must serialize double value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (arrayDictionarySerialized) {
+      throw new ISE("Array dictionary already serialized for column [%s], cannot serialize again", name);
+    }

Review Comment:
   It seems weird to me to have a bunch of public methods that throw a bunch of exceptions if they aren't called in the right order.  Why do each of them need to be exposed as public individually if they can only be called once in a specific order?  Why not have a single `serializeDictionaries` method that is called publicly which does the things in the correct order?



##########
sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java:
##########
@@ -95,7 +95,7 @@ public DimFilter toDruidFilter(
     final DruidExpression leftExpr = druidExpressions.get(0);
     final DruidExpression rightExpr = druidExpressions.get(1);
 
-    if (leftExpr.isSimpleExtraction()) {
+    if (leftExpr.isSimpleExtraction() && !(leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) {

Review Comment:
   What if it's an `array_contains()` over just a normal single-valued `String` column?  Shouldn't that also match the filter, pretending that each row contains an array of size 1?



##########
processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java:
##########
@@ -509,50 +507,6 @@ public void createIndex(
     }
   }
 
-  public void createIndex(
-      InputStream inputDataStream,
-      String parserJson,
-      String transformSpecJson,
-      String aggregators,
-      File outDir,
-      long minTimestamp,
-      Granularity gran,
-      int maxRowCount,
-      boolean rollup
-  ) throws Exception
-  {
-    try {
-      StringInputRowParser parser = mapper.readValue(parserJson, StringInputRowParser.class);
-      TransformSpec transformSpec;
-      if (transformSpecJson != null) {
-        transformSpec = mapper.readValue(transformSpecJson, TransformSpec.class);
-        parser = new TransformingStringInputRowParser(parser.getParseSpec(), parser.getEncoding(), transformSpec);
-      }
-
-      LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8");
-      List<AggregatorFactory> aggregatorSpecs = mapper.readValue(
-          aggregators,
-          new TypeReference<List<AggregatorFactory>>()
-          {
-          }
-      );
-
-      createIndex(
-          iter,
-          parser,
-          aggregatorSpecs.toArray(new AggregatorFactory[0]),
-          outDir,
-          minTimestamp,
-          gran,
-          true,
-          maxRowCount,
-          rollup
-      );
-    }
-    finally {
-      Closeables.close(inputDataStream, true);
-    }
-  }

Review Comment:
   Removing this method could quite easily break extensions.  IIRC, the `AggregationTestHelper` exists primarily to aid in building tests for AggregatorFactory extensions.  Did the same logic move somewhere else? If so, can we keep this method (maybe deprecated?) and have it call that other place?



##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -426,6 +765,12 @@ public ColumnIndexSupplier getColumnIndexSupplier(List<NestedPathPart> path)
   {
     final String field = getField(path);
     if (fields.indexOf(field) < 0) {
+      if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+        final String arrayField = getField(path.subList(0, path.size() - 1));
+        if (fields.indexOf(arrayField) >= 0) {
+          return NoIndexesColumnIndexSupplier.getInstance();
+        }

Review Comment:
   I'm assuming this is no indexes because we cannot force it to also have a value matcher along with using a bitmap index?  If so, it's sad that we cannot use indexes in this case.  We have indexes that are extremely useful here.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141629817


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -215,14 +263,24 @@ public void serializeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeS
               indexSpec,
               globalDictionaryIdLookup
           );
-        } else {
+        } else if (Types.is(type, ValueType.DOUBLE)) {
           writer = new DoubleFieldColumnWriter(
               name,
               fieldFileName,
               segmentWriteOutMedium,
               indexSpec,
               globalDictionaryIdLookup
           );
+        } else if (Types.is(type, ValueType.ARRAY)) {
+          writer = new ArrayOfLiteralsFieldColumnWriter(
+              name,
+              fieldFileName,
+              segmentWriteOutMedium,
+              indexSpec,
+              globalDictionaryIdLookup
+          );
+        } else {
+          throw new IllegalArgumentException("wtf");

Review Comment:
   oops this one was also a placeholder



-- 
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


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1106168713


##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -349,9 +450,73 @@
     if (fields.indexOf(field) >= 0) {
       BaseColumn col = getColumnHolder(field).getColumn();
       return col.makeColumnValueSelector(readableOffset);
-    } else {
-      return NilColumnValueSelector.instance();
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset);
+        return new ColumnValueSelector<Object>()
+        {
+          @Override
+          public boolean isNull()
+          {
+            Object o = getObject();
+            return !(o instanceof Number);
+          }
+
+          @Override
+          public long getLong()
+          {
+            Object o = getObject();
+            return o instanceof Number ? ((Number) o).longValue() : 0L;
+          }
+
+          @Override
+          public float getFloat()
+          {
+            Object o = getObject();
+            return o instanceof Number ? ((Number) o).floatValue() : 0f;
+          }
+
+          @Override
+          public double getDouble()
+          {
+            Object o = getObject();
+            return o instanceof Number ? ((Number) o).doubleValue() : 0.0;
+          }
+
+          @Override
+          public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+          {
+            arraySelector.inspectRuntimeShape(inspector);
+          }
+
+          @Nullable
+          @Override
+          public Object getObject()
+          {
+            Object o = arraySelector.getObject();
+            if (o instanceof Object[]) {
+              Object[] array = (Object[]) o;
+              if (elementNumber < array.length) {
+                return array[elementNumber];

Review Comment:
   ## Improper validation of user-provided array index
   
   This index depends on a [user-provided value](1) which can cause an ArrayIndexOutOfBoundsException.
   This index depends on a [user-provided value](2) which can cause an ArrayIndexOutOfBoundsException.
   This index depends on a [user-provided value](3) which can cause an ArrayIndexOutOfBoundsException.
   This index depends on a [user-provided value](4) which can cause an ArrayIndexOutOfBoundsException.
   This index depends on a [user-provided value](5) which can cause an ArrayIndexOutOfBoundsException.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4268)



##########
processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.primitives.Ints;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.io.Channels;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+import org.apache.druid.segment.writeout.WriteOutBytes;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+import java.util.Arrays;
+import java.util.Comparator;
+
+public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter<int[]>
+{
+  private static final int MAX_LOG_BUFFER_SIZE = 26;
+
+  public static final Comparator<int[]> ARRAY_COMPARATOR = (o1, o2) -> {
+    //noinspection ArrayEquality
+    if (o1 == o2) {
+      return 0;
+    }
+    if (o1 == null) {
+      return -1;
+    }
+    if (o2 == null) {
+      return 1;
+    }
+    final int iter = Math.min(o1.length, o2.length);
+    for (int i = 0; i < iter; i++) {
+      final int cmp = Integer.compare(o1[i], o2[i]);
+      if (cmp == 0) {
+        continue;
+      }
+      return cmp;
+    }
+    return Integer.compare(o1.length, o2.length);
+  };
+
+  private final SegmentWriteOutMedium segmentWriteOutMedium;
+  private final int bucketSize;
+  private final ByteOrder byteOrder;
+  private final int[][] bucketBuffer;
+  private final ByteBuffer getOffsetBuffer;
+  private final int div;
+
+  @Nullable
+  private int[] prevObject = null;
+  @Nullable
+  private WriteOutBytes headerOut = null;
+  @Nullable
+  private WriteOutBytes valuesOut = null;
+  private int numWritten = 0;
+  private ByteBuffer scratch;
+  private int logScratchSize = 10;
+  private boolean isClosed = false;
+  private boolean hasNulls = false;
+
+  public FrontCodedIntArrayIndexedWriter(
+      SegmentWriteOutMedium segmentWriteOutMedium,
+      ByteOrder byteOrder,
+      int bucketSize
+  )
+  {
+    if (Integer.bitCount(bucketSize) != 1 || bucketSize < 1 || bucketSize > 128) {
+      throw new IAE("bucketSize must be a power of two (from 1 up to 128) but was[%,d]", bucketSize);
+    }
+    this.segmentWriteOutMedium = segmentWriteOutMedium;
+    this.scratch = ByteBuffer.allocate(1 << logScratchSize).order(byteOrder);
+    this.bucketSize = bucketSize;
+    this.byteOrder = byteOrder;
+    this.bucketBuffer = new int[bucketSize][];
+    this.getOffsetBuffer = ByteBuffer.allocate(Integer.BYTES).order(byteOrder);
+    this.div = Integer.numberOfTrailingZeros(bucketSize);
+  }
+
+  @Override
+  public void open() throws IOException
+  {
+    headerOut = segmentWriteOutMedium.makeWriteOutBytes();
+    valuesOut = segmentWriteOutMedium.makeWriteOutBytes();
+  }
+
+  @Override
+  public void write(@Nullable int[] value) throws IOException
+  {
+
+    if (prevObject != null && ARRAY_COMPARATOR.compare(prevObject, value) >= 0) {
+      throw new ISE(
+          "Values must be sorted and unique. Element [%s] with value [%s] is before or equivalent to [%s]",
+          numWritten,
+          value == null ? null : Arrays.toString(value),
+          Arrays.toString(prevObject)
+      );
+    }
+
+    if (value == null) {
+      hasNulls = true;
+      return;
+    }
+
+    // if the bucket buffer is full, write the bucket
+    if (numWritten > 0 && (numWritten % bucketSize) == 0) {
+      resetScratch();
+      int written;
+      // write the bucket, growing scratch buffer as necessary
+      do {
+        written = writeBucket(scratch, bucketBuffer, bucketSize);
+        if (written < 0) {
+          growScratch();
+        }
+      } while (written < 0);
+      scratch.flip();
+      Channels.writeFully(valuesOut, scratch);
+
+      resetScratch();
+      // write end offset for current value
+      scratch.putInt((int) valuesOut.size());
+      scratch.flip();
+      Channels.writeFully(headerOut, scratch);
+    }
+
+    bucketBuffer[numWritten % bucketSize] = value;
+
+    ++numWritten;
+    prevObject = value;
+  }
+
+
+  @Override
+  public long getSerializedSize() throws IOException
+  {
+    if (!isClosed) {
+      flush();
+    }
+    int headerAndValues = Ints.checkedCast(headerOut.size() + valuesOut.size());
+    return Byte.BYTES +
+           Byte.BYTES +
+           Byte.BYTES +
+           VByte.computeIntSize(numWritten) +
+           VByte.computeIntSize(headerAndValues) +
+           headerAndValues;
+  }
+
+  @Override
+  public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
+  {
+    if (!isClosed) {
+      flush();
+    }
+    resetScratch();
+    // version 0
+    scratch.put((byte) 0);
+    scratch.put((byte) bucketSize);
+    scratch.put(hasNulls ? NullHandling.IS_NULL_BYTE : NullHandling.IS_NOT_NULL_BYTE);
+    VByte.writeInt(scratch, numWritten);
+    VByte.writeInt(scratch, Ints.checkedCast(headerOut.size() + valuesOut.size()));
+    scratch.flip();
+    Channels.writeFully(channel, scratch);
+    headerOut.writeTo(channel);
+    valuesOut.writeTo(channel);
+  }
+
+  @Override
+  public boolean isSorted()
+  {
+    return true;
+  }
+
+  @Nullable
+  @Override
+  public int[] get(int index) throws IOException
+  {
+    if (index == 0 && hasNulls) {
+      return null;
+    }
+    final int adjustedIndex = hasNulls ? index - 1 : index;
+    final int relativeIndex = adjustedIndex % bucketSize;
+    // check for current page
+    if (adjustedIndex >= numWritten - bucketSize) {
+      return bucketBuffer[relativeIndex];
+    } else {
+      final int bucket = adjustedIndex >> div;
+      long startOffset;
+      if (bucket == 0) {
+        startOffset = 0;
+      } else {
+        startOffset = getBucketOffset(bucket - 1);
+      }
+      long endOffset = getBucketOffset(bucket);
+      int bucketSize = Ints.checkedCast(endOffset - startOffset);

Review Comment:
   ## Possible confusion of local and field
   
   Confusing name: method [get](1) also refers to field [bucketSize](2) (without qualifying it with 'this').
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4267)



##########
processing/src/test/java/org/apache/druid/query/NestedDataTestUtils.java:
##########
@@ -19,432 +19,444 @@
 
 package org.apache.druid.query;
 
-import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.io.Files;
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.io.LineIterator;
-import org.apache.druid.data.input.impl.StringInputRowParser;
+import org.apache.druid.data.input.InputFormat;
+import org.apache.druid.data.input.InputRowSchema;
+import org.apache.druid.data.input.InputSource;
+import org.apache.druid.data.input.ResourceInputSource;
+import org.apache.druid.data.input.impl.DelimitedInputFormat;
+import org.apache.druid.data.input.impl.DimensionsSpec;
+import org.apache.druid.data.input.impl.JsonInputFormat;
+import org.apache.druid.data.input.impl.LocalInputSource;
+import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.guice.NestedDataModule;
 import org.apache.druid.java.util.common.StringUtils;
 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.nary.TrinaryFn;
 import org.apache.druid.java.util.common.io.Closer;
-import org.apache.druid.query.aggregation.AggregationTestHelper;
 import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.segment.IncrementalIndexSegment;
+import org.apache.druid.segment.IndexBuilder;
+import org.apache.druid.segment.NestedDataDimensionSchema;
 import org.apache.druid.segment.QueryableIndexSegment;
 import org.apache.druid.segment.Segment;
 import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.segment.incremental.IncrementalIndexSchema;
+import org.apache.druid.segment.transform.ExpressionTransform;
+import org.apache.druid.segment.transform.TransformSpec;
 import org.apache.druid.timeline.SegmentId;
 import org.junit.rules.TemporaryFolder;
 
 import java.io.ByteArrayInputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.io.SequenceInputStream;
-import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.function.BiFunction;
 
 public class NestedDataTestUtils
 {
-  public static final String SIMPLE_DATA_FILE = "simple-nested-test-data.json";
-  public static final String SIMPLE_PARSER_FILE = "simple-nested-test-data-parser.json";
-  public static final String SIMPLE_DATA_TSV_FILE = "simple-nested-test-data.tsv";
-  public static final String SIMPLE_PARSER_TSV_FILE = "simple-nested-test-data-tsv-parser.json";
-  public static final String SIMPLE_PARSER_TSV_TRANSFORM_FILE = "simple-nested-test-data-tsv-transform.json";
-  public static final String SIMPLE_AGG_FILE = "simple-nested-test-data-aggs.json";
-
-  public static final String TYPES_DATA_FILE = "types-test-data.json";
-  public static final String TYPES_PARSER_FILE = "types-test-data-parser.json";
-
-  public static final String NUMERIC_DATA_FILE = "numeric-nested-test-data.json";
-  public static final String NUMERIC_PARSER_FILE = "numeric-nested-test-data-parser.json";
+  public static final String SIMPLE_DATA_FILE = "nested-simple-test-data.json";
+  public static final String SIMPLE_DATA_TSV_FILE = "nested-simple-test-data.tsv";
+  public static final String NUMERIC_DATA_FILE = "nested-numeric-test-data.json";
+  public static final String TYPES_DATA_FILE = "nested-types-test-data.json";
+  public static final String ARRAY_TYPES_DATA_FILE = "nested-array-test-data.json";
 
   public static final ObjectMapper JSON_MAPPER;
 
+  public static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("timestamp", null, null);
+
+  public static final DimensionsSpec AUTO_DISCOVERY =
+      DimensionsSpec.builder()
+                    .setUseNestedColumnIndexerForSchemaDiscovery(true)
+                    .build();
+
+  public static final DimensionsSpec TSV_SCHEMA =
+      DimensionsSpec.builder()
+                    .setDimensions(
+                        Arrays.asList(
+                            new StringDimensionSchema("dim"),
+                            new NestedDataDimensionSchema("nest_json"),
+                            new NestedDataDimensionSchema("nester_json"),
+                            new NestedDataDimensionSchema("variant_json"),
+                            new NestedDataDimensionSchema("list_json")
+                        )
+                    )
+                    .build();
+  public static final InputRowSchema AUTO_SCHEMA = new InputRowSchema(
+      TIMESTAMP_SPEC,
+      AUTO_DISCOVERY,
+      null
+  );
+
+  public static final InputRowSchema SIMPLE_DATA_TSV_SCHEMA = new InputRowSchema(
+      TIMESTAMP_SPEC,
+      TSV_SCHEMA,
+      null
+  );
+
+  public static DelimitedInputFormat SIMPLE_DATA_TSV_INPUT_FORMAT = DelimitedInputFormat.ofColumns(
+      "timestamp",
+      "dim",
+      "nest",
+      "nester",
+      "variant",
+      "list"
+  );
+  public static final TransformSpec SIMPLE_DATA_TSV_TRANSFORM = new TransformSpec(
+      null,
+      Arrays.asList(
+          new ExpressionTransform("nest_json", "parse_json(nest)", TestExprMacroTable.INSTANCE),
+          new ExpressionTransform("nester_json", "parse_json(nester)", TestExprMacroTable.INSTANCE),
+          new ExpressionTransform("variant_json", "parse_json(variant)", TestExprMacroTable.INSTANCE),
+          new ExpressionTransform("list_json", "parse_json(list)", TestExprMacroTable.INSTANCE)
+      )
+  );
+
+  public static final AggregatorFactory[] COUNT = new AggregatorFactory[]{
+      new CountAggregatorFactory("count")
+  };
+
   static {
     JSON_MAPPER = TestHelper.makeJsonMapper();
     JSON_MAPPER.registerModules(NestedDataModule.getJacksonModulesList());
   }
 
-  public static List<Segment> createSegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSimpleSegmentsTsv(
       TemporaryFolder tempFolder,
-      Closer closer,
-      Granularity granularity,
-      boolean rollup,
-      int maxRowCount
-  ) throws Exception
+      Closer closer
+  )
+      throws Exception
   {
-    return createSegments(
-        helper,
+    return createSimpleNestedTestDataTsvSegments(
         tempFolder,
         closer,
-        SIMPLE_DATA_FILE,
-        SIMPLE_PARSER_FILE,
-        SIMPLE_AGG_FILE,
-        granularity,
-        rollup,
-        maxRowCount
+        Granularities.NONE,
+        true
     );
   }
 
-  public static List<Segment> createTsvSegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSimpleNestedTestDataTsvSegments(
       TemporaryFolder tempFolder,
       Closer closer,
       Granularity granularity,
-      boolean rollup,
-      int maxRowCount
+      boolean rollup
   ) throws Exception
   {
     return createSegments(
-        helper,
         tempFolder,
         closer,
         SIMPLE_DATA_TSV_FILE,
-        SIMPLE_PARSER_TSV_FILE,
-        SIMPLE_PARSER_TSV_TRANSFORM_FILE,
-        SIMPLE_AGG_FILE,
+        SIMPLE_DATA_TSV_INPUT_FORMAT,
+        TIMESTAMP_SPEC,
+        SIMPLE_DATA_TSV_SCHEMA.getDimensionsSpec(),
+        SIMPLE_DATA_TSV_TRANSFORM,
+        COUNT,
         granularity,
-        rollup,
-        maxRowCount
+        rollup
     );
   }
 
-  public static Segment createIncrementalIndex(
-      Granularity granularity,
-      boolean rollup,
-      boolean deserializeComplexMetrics,
-      int maxRowCount
-  )
-      throws Exception
+  public static Segment createSimpleNestedTestDataIncrementalIndex(TemporaryFolder tempFolder) throws Exception
   {
-    return createIncrementalIndex(
+    return createIncrementalIndexForJsonInput(
+        tempFolder,
         SIMPLE_DATA_FILE,
-        SIMPLE_PARSER_FILE,
-        SIMPLE_AGG_FILE,
-        granularity,
-        rollup,
-        deserializeComplexMetrics,
-        maxRowCount
+        Granularities.NONE,
+        true,
+        1000
     );
   }
 
-  public static List<Segment> createSegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSimpleNestedTestDataSegments(
       TemporaryFolder tempFolder,
-      Closer closer,
-      String inputFileName,
-      String parserJsonFileName,
-      String aggJsonFileName,
-      Granularity granularity,
-      boolean rollup,
-      int maxRowCount
-  ) throws Exception
+      Closer closer
+  )
+      throws Exception
   {
-    File segmentDir = tempFolder.newFolder();
-    File inputFile = readFileFromClasspath(inputFileName);
-    FileInputStream inputDataStream = new FileInputStream(inputFile);
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-
-    helper.createIndex(
-        inputDataStream,
-        parserJson,
-        aggJson,
-        segmentDir,
-        0,
-        granularity,
-        maxRowCount,
-        rollup
-    );
-    inputDataStream.close();
-
-    final List<Segment> segments = Lists.transform(
-        ImmutableList.of(segmentDir),
-        dir -> {
-          try {
-            return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
-          }
-          catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        }
+    return createSegmentsForJsonInput(
+        tempFolder,
+        closer,
+        SIMPLE_DATA_FILE,
+        Granularities.NONE,
+        true
     );
+  }
 
-    return segments;
+  public static Segment createIncrementalIndexForJsonInput(TemporaryFolder tempFolder, String fileName)
+      throws Exception
+  {
+    return createIncrementalIndexForJsonInput(
+        tempFolder,
+        fileName,
+        Granularities.NONE,
+        true,
+        1000
+    );
   }
 
-  public static List<Segment> createSegments(
-      AggregationTestHelper helper,
+  public static Segment createIncrementalIndexForJsonInput(
       TemporaryFolder tempFolder,
-      Closer closer,
-      String inputFileName,
-      String parserJsonFileName,
-      String transformSpecJsonFileName,
-      String aggJsonFileName,
+      String file,
       Granularity granularity,
       boolean rollup,
       int maxRowCount
-  ) throws Exception
+  )
+      throws Exception
   {
-    File segmentDir = tempFolder.newFolder();
-    File inputFile = readFileFromClasspath(inputFileName);
-    FileInputStream inputDataStream = new FileInputStream(inputFile);
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String transformSpecJson = readFileFromClasspathAsString(transformSpecJsonFileName);
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-
-    helper.createIndex(
-        inputDataStream,
-        parserJson,
-        transformSpecJson,
-        aggJson,
-        segmentDir,
-        0,
+    return createIncrementalIndex(
+        tempFolder,
+        file,
+        JsonInputFormat.DEFAULT,
+        TIMESTAMP_SPEC,
+        AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        COUNT,
         granularity,
-        maxRowCount,
-        rollup
-    );
-    inputDataStream.close();
-
-    final List<Segment> segments = Lists.transform(
-        ImmutableList.of(segmentDir),
-        dir -> {
-          try {
-            return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
-          }
-          catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        }
+        rollup,
+        maxRowCount
     );
-
-    return segments;
   }
 
-  public static List<Segment> createSegmentsWithConcatenatedInput(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegmentsForJsonInput(
       TemporaryFolder tempFolder,
       Closer closer,
+      String inputFile,
       Granularity granularity,
-      boolean rollup,
-      int maxRowCount,
-      int numCopies,
-      int numSegments
+      boolean rollup
   ) throws Exception
   {
-    return createSegmentsWithConcatenatedInput(
-        helper,
+    return createSegments(
         tempFolder,
         closer,
-        SIMPLE_DATA_FILE,
-        SIMPLE_PARSER_FILE,
-        null,
-        SIMPLE_AGG_FILE,
+        inputFile,
+        JsonInputFormat.DEFAULT,
+        TIMESTAMP_SPEC,
+        AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        COUNT,
         granularity,
-        rollup,
-        maxRowCount,
-        numCopies,
-        numSegments
+        rollup
     );
   }
 
-  /**
-   * turn small test data into bigger test data by duplicating itself into a bigger stream
-   */
-  public static List<Segment> createSegmentsWithConcatenatedInput(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegmentsWithConcatenatedJsonInput(
       TemporaryFolder tempFolder,
       Closer closer,
-      String inputFileName,
-      String parserJsonFileName,
-      String transformSpecJsonFileName,
-      String aggJsonFileName,
+      String inputFile,
       Granularity granularity,
       boolean rollup,
-      int maxRowCount,
       int numCopies,
       int numSegments
   ) throws Exception
   {
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String transformSpecJson = transformSpecJsonFileName != null ? readFileFromClasspathAsString(transformSpecJsonFileName) : null;
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-
-    List<File> segmentDirs = Lists.newArrayListWithCapacity(numSegments);
+    List<InputSource> inputFiles = Lists.newArrayListWithCapacity(numSegments);
     for (int i = 0; i < numSegments; i++) {
-      List<InputStream> inputStreams = Lists.newArrayListWithCapacity(numCopies);
-      for (int j = 0; j < numCopies; j++) {
-        inputStreams.add(new FileInputStream(readFileFromClasspath(inputFileName)));
-        if (j + 1 < numCopies) {
-          inputStreams.add(new ByteArrayInputStream(StringUtils.toUtf8("\n")));
-        }
-      }
-      SequenceInputStream inputDataStream = new SequenceInputStream(Collections.enumeration(inputStreams));
-      File segmentDir = tempFolder.newFolder();
-      helper.createIndex(
-          inputDataStream,
-          parserJson,
-          transformSpecJson,
-          aggJson,
-          segmentDir,
-          0,
-          granularity,
-          maxRowCount,
-          rollup
-      );
-      inputDataStream.close();
-      segmentDirs.add(segmentDir);
+      File file = selfConcatenateResourceFile(tempFolder, inputFile, numCopies);
+      inputFiles.add(new LocalInputSource(file.getParentFile(), file.getName()));
     }
-
-    final List<Segment> segments = Lists.transform(
-        segmentDirs,
-        dir -> {
-          try {
-            return closer.register(new QueryableIndexSegment(helper.getIndexIO().loadIndex(dir), SegmentId.dummy("")));
-          }
-          catch (IOException ex) {
-            throw new RuntimeException(ex);
-          }
-        }
-    );
-
-    return segments;
-  }
-
-  public static Segment createIncrementalIndex(
-      String inputFileName,
-      String parserJsonFileName,
-      String aggJsonFileName,
-      Granularity granularity,
-      boolean rollup,
-      boolean deserializeComplexMetrics,
-      int maxRowCount
-  )
-      throws Exception
-  {
-    File inputFile = readFileFromClasspath(inputFileName);
-    FileInputStream inputDataStream = new FileInputStream(inputFile);
-    String parserJson = readFileFromClasspathAsString(parserJsonFileName);
-    String aggJson = readFileFromClasspathAsString(aggJsonFileName);
-    StringInputRowParser parser = JSON_MAPPER.readValue(parserJson, StringInputRowParser.class);
-
-    LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8");
-    List<AggregatorFactory> aggregatorSpecs = JSON_MAPPER.readValue(
-        aggJson,
-        new TypeReference<List<AggregatorFactory>>()
-        {
-        }
-    );
-    IncrementalIndex index = AggregationTestHelper.createIncrementalIndex(
-        iter,
-        parser,
-        parser.getParseSpec().getDimensionsSpec().getDimensions(),
-        aggregatorSpecs.toArray(new AggregatorFactory[0]),
-        0,
+    return createSegments(
+        tempFolder,
+        closer,
+        inputFiles,
+        JsonInputFormat.DEFAULT,
+        TIMESTAMP_SPEC,
+        AUTO_DISCOVERY,
+        TransformSpec.NONE,
+        COUNT,
         granularity,
-        deserializeComplexMetrics,
-        maxRowCount,
         rollup
     );
-    inputDataStream.close();
-    return new IncrementalIndexSegment(index, SegmentId.dummy("test_datasource"));
   }
 
-  public static Segment createDefaultHourlyIncrementalIndex() throws Exception
-  {
-    return createIncrementalIndex(Granularities.HOUR, true, true, 1000);
-  }
-
-  public static Segment createDefaultDailyIncrementalIndex() throws Exception
-  {
-    return createIncrementalIndex(Granularities.DAY, true, true, 1000);
-  }
-
-  public static List<Segment> createDefaultHourlySegments(
-      AggregationTestHelper helper,
+  public static List<Segment> createSegmentsForJsonInput(
       TemporaryFolder tempFolder,
-      Closer closer
+      Closer closer,
+      String inputFile
   )
       throws Exception
   {
-    return createSegments(
-        helper,
+    return createSegmentsForJsonInput(
         tempFolder,
         closer,
-        Granularities.HOUR,
-        true,
-        1000
+        inputFile,
+        Granularities.NONE,
+        true
     );
   }
 
-  public static List<Segment> createDefaultHourlySegmentsTsv(
-      AggregationTestHelper helper,
+  public static Segment createIncrementalIndex(
       TemporaryFolder tempFolder,
-      Closer closer
+      String inputFileName,
+      InputFormat inputFormat,
+      TimestampSpec timestampSpec,
+      DimensionsSpec dimensionsSpec,
+      TransformSpec transformSpec,
+      AggregatorFactory[] aggregators,
+      Granularity queryGranularity,
+      boolean rollup,
+      int maxRowCount

Review Comment:
   ## Useless parameter
   
   The parameter 'maxRowCount' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4266)



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141626580


##########
processing/src/main/java/org/apache/druid/segment/nested/ArrayOfLiteralsFieldColumnWriter.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.segment.nested;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
+import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import java.io.IOException;
+import java.nio.channels.WritableByteChannel;
+
+public class ArrayOfLiteralsFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<int[]>
+{
+
+  protected ArrayOfLiteralsFieldColumnWriter(
+      String columnName,
+      String fieldName,
+      SegmentWriteOutMedium segmentWriteOutMedium,
+      IndexSpec indexSpec,
+      GlobalDictionaryIdLookup globalDictionaryIdLookup
+  )
+  {
+    super(columnName, fieldName, segmentWriteOutMedium, indexSpec, globalDictionaryIdLookup);
+  }
+
+  @Override
+  int[] processValue(int row, Object value)
+  {
+    if (value instanceof Object[]) {

Review Comment:
   it doesn't have to because `ExprEval` homogenizes the array types to `Object[]`



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146904796


##########
processing/src/test/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedTest.java:
##########
@@ -0,0 +1,455 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.ThreadLocalRandom;
+
+@RunWith(Parameterized.class)
+public class FrontCodedIntArrayIndexedTest
+{
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<Object[]> constructorFeeder()
+  {
+    return ImmutableList.of(new Object[]{ByteOrder.LITTLE_ENDIAN}, new Object[]{ByteOrder.BIG_ENDIAN});
+  }
+
+  private final ByteOrder order;
+
+  public FrontCodedIntArrayIndexedTest(ByteOrder byteOrder)
+  {
+    this.order = byteOrder;
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexed() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3, 4});
+    values.add(new int[]{1, 2, 1});
+
+    fillBuffer(buffer, values, 4);
+
+    buffer.position(0);
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Iterator<int[]> indexedIterator = codedIndexed.iterator();
+    Iterator<int[]> expectedIterator = values.iterator();
+    int ctr = 0;
+    while (expectedIterator.hasNext() && indexedIterator.hasNext()) {
+      final int[] expectedNext = expectedIterator.next();
+      final int[] next = indexedIterator.next();
+      Assert.assertArrayEquals(expectedNext, next);
+      Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+      ctr++;
+    }
+    Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+  }
+
+
+  @Test
+  public void testFrontCodedIntArrayIndexedSingleBucket() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3, 4});
+    values.add(new int[]{1, 2, 1});
+    fillBuffer(buffer, values, 16);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Iterator<int[]> expectedIterator = values.iterator();
+    Iterator<int[]> indexedIterator = codedIndexed.iterator();
+    int ctr = 0;
+    while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+      final int[] expectedNext = expectedIterator.next();
+      final int[] next = indexedIterator.next();
+      Assert.assertArrayEquals(expectedNext, next);
+      Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+      ctr++;
+    }
+    Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedBigger() throws IOException
+  {
+    final int sizeBase = 10000;
+    final int bucketSize = 16;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 24).order(order);
+    for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) {
+      final TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+      while (values.size() < sizeBase + sizeAdjust) {
+        int length = ThreadLocalRandom.current().nextInt(10);
+        final int[] val = new int[length];
+        for (int j = 0; j < length; j++) {
+          val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+        }
+        values.add(val);
+      }
+      fillBuffer(buffer, values, bucketSize);
+
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> indexedIterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = indexedIterator.next();
+        Assert.assertArrayEquals(expectedNext, next);
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+      Assert.assertEquals(ctr, sizeBase + sizeAdjust);
+    }
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedBiggerWithNulls() throws IOException
+  {
+    final int sizeBase = 10000;
+    final int bucketSize = 16;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order);
+    for (int sizeAdjust = 0; sizeAdjust < bucketSize; sizeAdjust++) {
+      TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+      values.add(null);
+      while (values.size() < sizeBase + sizeAdjust + 1) {
+        int length = ThreadLocalRandom.current().nextInt(10);
+        final int[] val = new int[length];
+        for (int j = 0; j < length; j++) {
+          val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+        }
+        values.add(val);
+      }
+      fillBuffer(buffer, values, bucketSize);
+
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> indexedIterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (indexedIterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = indexedIterator.next();
+        Assert.assertArrayEquals(expectedNext, next);
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), indexedIterator.hasNext());
+      Assert.assertEquals(ctr, sizeBase + sizeAdjust + 1);
+    }
+  }
+
+  @Test
+  public void testFrontCodedIntArrayIndexedIndexOf() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 3, 4});
+
+    fillBuffer(buffer, values, 4);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+    Assert.assertEquals(-1, codedIndexed.indexOf(new int[]{1}));
+    Assert.assertEquals(0, codedIndexed.indexOf(new int[]{1, 2}));
+    Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2, 1}));
+    Assert.assertEquals(-3, codedIndexed.indexOf(new int[]{1, 2, 2}));
+    Assert.assertEquals(4, codedIndexed.indexOf(new int[]{1, 3}));
+    Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{1, 4, 4}));
+    Assert.assertEquals(-7, codedIndexed.indexOf(new int[]{9, 1, 1}));
+  }
+
+
+  @Test
+  public void testFrontCodedIntArrayIndexedIndexOfWithNull() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(null);
+    values.add(new int[]{1, 2});
+    values.add(new int[]{1, 2, 1});
+    values.add(new int[]{1, 2, 3});
+    values.add(new int[]{1, 2, 4});
+    values.add(new int[]{1, 3});
+    values.add(new int[]{1, 3, 4});
+    fillBuffer(buffer, values, 4);
+
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+    Assert.assertEquals(0, codedIndexed.indexOf(null));
+    Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1}));
+    Assert.assertEquals(1, codedIndexed.indexOf(new int[]{1, 2}));
+    Assert.assertEquals(2, codedIndexed.indexOf(new int[]{1, 2, 1}));
+    Assert.assertEquals(-4, codedIndexed.indexOf(new int[]{1, 2, 2}));
+    Assert.assertEquals(5, codedIndexed.indexOf(new int[]{1, 3}));
+    Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{1, 4, 4}));
+    Assert.assertEquals(-8, codedIndexed.indexOf(new int[]{9, 1, 1}));
+  }
+
+
+  @Test
+  public void testFrontCodedOnlyNull() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 12).order(order);
+    List<int[]> theList = Collections.singletonList(null);
+    fillBuffer(buffer, theList, 4);
+
+    buffer.position(0);
+    FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertNull(codedIndexed.get(0));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedIndexed.get(theList.size()));
+
+    Assert.assertEquals(0, codedIndexed.indexOf(null));
+    Assert.assertEquals(-2, codedIndexed.indexOf(new int[]{1, 2, 3, 4}));
+
+    Iterator<int[]> iterator = codedIndexed.iterator();
+    Assert.assertTrue(iterator.hasNext());
+    Assert.assertNull(iterator.next());
+    Assert.assertFalse(iterator.hasNext());
+  }
+
+  @Test
+  public void testFrontCodedEmpty() throws IOException
+  {
+    ByteBuffer buffer = ByteBuffer.allocate(1 << 6).order(order);
+    List<int[]> theList = Collections.emptyList();
+    fillBuffer(buffer, theList, 4);
+
+    buffer.position(0);
+    FrontCodedIndexed codedUtf8Indexed = FrontCodedIndexed.read(
+        buffer,
+        buffer.order()
+    ).get();
+
+    Assert.assertEquals(0, codedUtf8Indexed.size());
+    Throwable t = Assert.assertThrows(IAE.class, () -> codedUtf8Indexed.get(0));
+    Assert.assertEquals("Index[0] >= size[0]", t.getMessage());
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(-1));
+    Assert.assertThrows(IllegalArgumentException.class, () -> codedUtf8Indexed.get(theList.size()));
+
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(null));
+    Assert.assertEquals(-1, codedUtf8Indexed.indexOf(StringUtils.toUtf8ByteBuffer("hello")));
+
+    Iterator<ByteBuffer> utf8Iterator = codedUtf8Indexed.iterator();
+    Assert.assertFalse(utf8Iterator.hasNext());
+  }
+
+  @Test
+  public void testBucketSizes() throws IOException
+  {
+    final int numValues = 10000;
+    final ByteBuffer buffer = ByteBuffer.allocate(1 << 25).order(order);
+    final int[] bucketSizes = new int[]{
+        1,
+        1 << 1,
+        1 << 2,
+        1 << 3,
+        1 << 4,
+        1 << 5,
+        1 << 6,
+        1 << 7
+    };
+
+    TreeSet<int[]> values = new TreeSet<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
+    values.add(null);
+    while (values.size() < numValues + 1) {
+      int length = ThreadLocalRandom.current().nextInt(10);
+      final int[] val = new int[length];
+      for (int j = 0; j < length; j++) {
+        val[j] = ThreadLocalRandom.current().nextInt(0, 10_000);
+      }
+      values.add(val);
+    }
+    for (int bucketSize : bucketSizes) {
+      fillBuffer(buffer, values, bucketSize);
+      FrontCodedIntArrayIndexed codedIndexed = FrontCodedIntArrayIndexed.read(
+          buffer,
+          buffer.order()
+      ).get();
+
+      Iterator<int[]> expectedIterator = values.iterator();
+      Iterator<int[]> iterator = codedIndexed.iterator();
+      int ctr = 0;
+      while (iterator.hasNext() && expectedIterator.hasNext()) {
+        final int[] expectedNext = expectedIterator.next();
+        final int[] next = iterator.next();
+        Assert.assertArrayEquals(expectedNext, next);
+
+        Assert.assertEquals(ctr, codedIndexed.indexOf(next));
+        ctr++;
+      }
+      Assert.assertEquals(expectedIterator.hasNext(), iterator.hasNext());
+      Assert.assertEquals(ctr, numValues + 1);
+    }
+  }
+
+  @Test
+  public void testBadBucketSize()
+  {
+    OnHeapMemorySegmentWriteOutMedium medium = new OnHeapMemorySegmentWriteOutMedium();
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            0
+        )
+    );
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            15
+        )
+    );
+
+    Assert.assertThrows(
+        IAE.class,
+        () -> new FrontCodedIntArrayIndexedWriter(
+            medium,
+            ByteOrder.nativeOrder(),
+            256
+        )
+    );
+  }
+
+  private static long fillBuffer(ByteBuffer buffer, Iterable<int[]> sortedIterable, int bucketSize) throws IOException

Review Comment:
   changed (and also in `FrontCodedIndexedTest`)



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148807692


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );
+          switch (columnType.getElementType().getType()) {
+            case LONG:
+              typeSet.add(ColumnType.LONG_ARRAY);
+              final Object[] longArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+              return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
+            case DOUBLE:
+              typeSet.add(ColumnType.DOUBLE_ARRAY);
+              final Object[] doubleArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray);
+              return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
+            case STRING:
+              final Object[] stringArray = eval.asArray();
+              // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
+              // the type set
+              if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
+                typeSet.add(ColumnType.STRING_ARRAY);
+              }
+              sizeEstimate = globalDimensionDictionary.addStringArray(stringArray);
+              return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
+            default:
+              throw new IAE("Unhandled type: %s", columnType);
+          }
+        case STRING:
+        default:

Review Comment:
   I don't think we should hit the default case, i'm not entirely sure why i wrote it like this originally



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148806371


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -81,46 +87,75 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
   private final StructuredDataProcessor fieldProcessor = new StructuredDataProcessor()
   {
     @Override
-    public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
       final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
           NestedPathFinder.toNormalizedJsonPath(fieldPath)
       );
       if (writer != null) {
         try {
-          ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+          final ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
           if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) {
             writer.addValue(rowCount, eval.value());
           } else {
             // behave consistently with nested column indexer, which defaults to string
             writer.addValue(rowCount, eval.asString());
           }
           // serializer doesn't use size estimate
-          return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          return ProcessedValue.NULL_LITERAL;
         }
         catch (IOException e) {
-          throw new RuntimeException(":(");
+          throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, fieldValue);
         }
       }
-      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+      return ProcessedValue.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedValue<?> processArrayField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable List<?> array
+    )
+    {
+      final ExprEval<?> eval = ExprEval.bestEffortArray(array);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
+            NestedPathFinder.toNormalizedJsonPath(fieldPath)
+        );
+        if (writer != null) {
+          try {
+            writer.addValue(rowCount, eval.value());
+            // serializer doesn't use size estimate
+            return ProcessedValue.NULL_LITERAL;
+          }
+          catch (IOException e) {
+            throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, array);

Review Comment:
   yeah, that's fair, i can change it, was thinking in terms of helping myself debug what wasn't handled correctly but it is leaky



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146902679


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -86,93 +94,123 @@ public NestedDataColumnMerger(
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+        }
 
-    long dimStartTime = System.currentTimeMillis();
-
-    int numMergeIndex = 0;
-    GlobalDictionarySortedCollector sortedLookup = null;
-    final Indexed[] sortedLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+        boolean allNulls = dimValues == null || allNull(dimValues.getSortedStrings()) &&
+                                                allNull(dimValues.getSortedLongs()) &&
+                                                allNull(dimValues.getSortedDoubles()) &&
+                                                dimValues.getArrayCardinality() == 0;

Review Comment:
   moved to `GlobalDictionarySortedCollector`



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146914518


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -234,7 +272,9 @@ private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNest
     return new GlobalDictionarySortedCollector(
         new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()),
         column.getLongDictionary(),
-        column.getDoubleDictionary()
+        column.getDoubleDictionary(),
+        column.getArraysIterable(),
+        column.getArrayDictionary().size()

Review Comment:
   it felt strange to try to wire this iterable up to wrap in an `Indexed` when all we need it for is to iterate over



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146896054


##########
processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java:
##########
@@ -480,11 +480,25 @@ public Expr apply(List<Expr> args)
       final StructuredDataProcessor processor = new StructuredDataProcessor()
       {
         @Override
-        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
         {
           // do nothing, we only want the list of fields returned by this processor
           return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
         }
+
+        @Nullable
+        @Override
+        public ProcessedLiteral<?> processArrayOfLiteralsField(
+            ArrayList<NestedPathPart> fieldPath,
+            @Nullable Object maybeArrayOfLiterals
+        )
+        {
+          ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);

Review Comment:
   i split out `bestEffortArray(@Nullable List<?> theList)` and changed `StructuredDataProcessor` method to always pass a `List` when attempting to process arrays



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148812313


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );

Review Comment:
   Just for clarity, I was suggestion that instead of using `Preconditions` you use an if statement instead.



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );

Review Comment:
   Just for clarity, I was suggesting that instead of using `Preconditions` you use an if statement instead.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148805906


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );
+          switch (columnType.getElementType().getType()) {
+            case LONG:
+              typeSet.add(ColumnType.LONG_ARRAY);
+              final Object[] longArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+              return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
+            case DOUBLE:
+              typeSet.add(ColumnType.DOUBLE_ARRAY);
+              final Object[] doubleArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray);
+              return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
+            case STRING:
+              final Object[] stringArray = eval.asArray();
+              // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
+              // the type set
+              if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
+                typeSet.add(ColumnType.STRING_ARRAY);
+              }
+              sizeEstimate = globalDimensionDictionary.addStringArray(stringArray);
+              return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
+            default:
+              throw new IAE("Unhandled type: %s", columnType);

Review Comment:
   We don't really expect it to happen because the `StructuredDataProcessor` should have done its job correctly and only defined fields for primitive and array of primitive, so this is more of a sanity check



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148813190


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );
+          switch (columnType.getElementType().getType()) {
+            case LONG:
+              typeSet.add(ColumnType.LONG_ARRAY);
+              final Object[] longArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+              return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
+            case DOUBLE:
+              typeSet.add(ColumnType.DOUBLE_ARRAY);
+              final Object[] doubleArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray);
+              return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
+            case STRING:
+              final Object[] stringArray = eval.asArray();
+              // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
+              // the type set
+              if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
+                typeSet.add(ColumnType.STRING_ARRAY);
+              }
+              sizeEstimate = globalDimensionDictionary.addStringArray(stringArray);
+              return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
+            default:
+              throw new IAE("Unhandled type: %s", columnType);

Review Comment:
   Ah, so `array-of-arrays` should be handled in `StructuredDataProcessor` instead?
   
   Even after this PR, the handling of `array-of-arrays` and `array-of-objects` is still a bit muddy, right?  (it's falling back to the "an array is just an object with numbers for field names" behavior?)



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141642926


##########
processing/src/test/resources/nested-array-test-data.json:
##########
@@ -0,0 +1,14 @@
+{"timestamp": "2023-01-01T00:00:00", "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],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "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], "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-01T00:00:00", "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],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],   "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],     "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-01T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,              "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "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],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "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], "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-02T00:00:00", "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],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],   "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],     "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-02T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,              "arrayObject":[{"x": 1},{"x":2}]}

Review Comment:
   i'll add some tests for this, but assuming something like ARRAY<ARRAY<STRING>> there would be an array column for each outer array element



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141643257


##########
processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java:
##########
@@ -42,45 +47,107 @@
   private final ComparatorDimensionDictionary<String> stringDictionary;
   private final ComparatorDimensionDictionary<Long> longDictionary;
   private final ComparatorDimensionDictionary<Double> doubleDictionary;
+  private final Set<Object[]> stringArrays;
+  private final Set<Object[]> longArrays;
+  private final Set<Object[]> doubleArrays;

Review Comment:
   oh maybe this isn't used for merging segments... let me get back to you on this one



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146904360


##########
processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java:
##########
@@ -101,7 +101,7 @@ public Sequence<Cursor> makeCursors(
           Cursor retVal = cursor;
           ColumnCapabilities capabilities = cursor.getColumnSelectorFactory().getColumnCapabilities(dimensionToUnnest);
           if (capabilities != null) {
-            if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {
+            if (!capabilities.isArray() && capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {

Review Comment:
   i confirmed that this doesn't impact `MV_TO_ARRAY` when used in SQL because the planner unwraps it leaving just a direct column access for the underlying multi-value column, so the capabilities are `STRING` typed with `hasMultipleValues` set to true.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136481973


##########
processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java:
##########
@@ -134,6 +134,11 @@ public Map<String, ColumnAnalysis> analyze(Segment segment)
             analysis = analyzeStringColumn(capabilities, storageAdapter, columnName);
           }
           break;
+        case ARRAY:
+          // todo (clint): this is wack, but works for now because arrays are always nested complex columns...

Review Comment:
   yep, i'm planning on replacing this todo with a 'what we need to do in the future', just haven't got to it yet



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141643536


##########
processing/src/main/java/org/apache/druid/segment/nested/GlobalDictionaryEncodedFieldColumnWriter.java:
##########
@@ -231,9 +252,16 @@ public void writeTo(int finalRowCount, FileSmoosher smoosher) throws IOException
       @Override
       public long getSerializedSize() throws IOException
       {
+        final long arraySize;
+        if (arrayElements.size() > 0) {
+          arraySize = arrayElementDictionaryWriter.getSerializedSize() + arrayElementIndexWriter.getSerializedSize();
+        } else {
+          arraySize = 0;
+        }

Review Comment:
   yeah i've got a whole pile of stats i'd like to track but haven't got to yet, will add this one to the list.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146907393


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -104,6 +108,32 @@ public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList
       }
       return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
     }
+
+    @Nullable
+    @Override
+    public ProcessedLiteral<?> processArrayOfLiteralsField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable Object maybeArrayOfLiterals
+    )
+    {
+      ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
+            NestedPathFinder.toNormalizedJsonPath(fieldPath)
+        );
+        if (writer != null) {
+          try {
+            writer.addValue(rowCount, eval.value());
+            // serializer doesn't use size estimate
+            return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          }
+          catch (IOException e) {
+            throw new RuntimeException(":(");

Review Comment:
   fixed



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -215,14 +263,24 @@ public void serializeFields(SortedMap<String, NestedLiteralTypeInfo.MutableTypeS
               indexSpec,
               globalDictionaryIdLookup
           );
-        } else {
+        } else if (Types.is(type, ValueType.DOUBLE)) {
           writer = new DoubleFieldColumnWriter(
               name,
               fieldFileName,
               segmentWriteOutMedium,
               indexSpec,
               globalDictionaryIdLookup
           );
+        } else if (Types.is(type, ValueType.ARRAY)) {
+          writer = new ArrayOfLiteralsFieldColumnWriter(
+              name,
+              fieldFileName,
+              segmentWriteOutMedium,
+              indexSpec,
+              globalDictionaryIdLookup
+          );
+        } else {
+          throw new IllegalArgumentException("wtf");

Review Comment:
   fixed



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141630196


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -251,33 +313,78 @@ public void serializeStringDictionary(Iterable<String> dictionaryValues) throws
       dictionaryWriter.write(value);
       globalDictionaryIdLookup.addString(value);
     }
+    stringDictionarySerialized = true;
   }
 
   public void serializeLongDictionary(Iterable<Long> dictionaryValues) throws IOException
   {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing long dictionary for column [%s]", name);
+    }
+    if (longDictionarySerialized) {
+      throw new ISE("Long dictionary already serialized for column [%s], cannot serialize again", name);
+    }
     for (Long value : dictionaryValues) {
       if (value == null) {
         continue;
       }
       longDictionaryWriter.write(value);
       globalDictionaryIdLookup.addLong(value);
     }
+    longDictionarySerialized = true;
   }
 
   public void serializeDoubleDictionary(Iterable<Double> dictionaryValues) throws IOException
   {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing double dictionary for column [%s]", name);
+    }
+    if (!longDictionarySerialized) {
+      throw new ISE("Must serialize long value dictionary before serializing double dictionary for column [%s]", name);
+    }
+    if (doubleDictionarySerialized) {
+      throw new ISE("Double dictionary already serialized for column [%s], cannot serialize again", name);
+    }
     for (Double value : dictionaryValues) {
       if (value == null) {
         continue;
       }
       doubleDictionaryWriter.write(value);
       globalDictionaryIdLookup.addDouble(value);
     }
+    doubleDictionarySerialized = true;
+  }
+
+  public void serializeArrayDictionary(Iterable<int[]> dictionaryValues) throws IOException
+  {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (!longDictionarySerialized) {
+      throw new ISE("Must serialize long value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (!doubleDictionarySerialized) {
+      throw new ISE("Must serialize double value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (arrayDictionarySerialized) {
+      throw new ISE("Array dictionary already serialized for column [%s], cannot serialize again", name);
+    }

Review Comment:
   yeah i agree it could use refactored, i added the exceptions to at least enforce this is done correctly because it was previously true as well just nothing was checking it, will look into making some adjustments



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141633256


##########
sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java:
##########
@@ -95,7 +95,7 @@ public DimFilter toDruidFilter(
     final DruidExpression leftExpr = druidExpressions.get(0);
     final DruidExpression rightExpr = druidExpressions.get(1);
 
-    if (leftExpr.isSimpleExtraction()) {
+    if (leftExpr.isSimpleExtraction() && !(leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) {

Review Comment:
   the type shouldn't show up as array here if it isn't an array column



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136487243


##########
extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java:
##########
@@ -181,7 +181,7 @@ public void testNestedColumnSchemalessNestedTestFileNoNested() throws IOExceptio
     );
 
     List<InputRow> rows = readAllRows(reader);
-    Assert.assertEquals(ImmutableList.of("dim1", "metric1", "timestamp"), rows.get(0).getDimensions());
+    Assert.assertEquals(ImmutableList.of("dim1", "metric1"), rows.get(0).getDimensions());

Review Comment:
   this is related to the change in dimension filter in `MapBasedInputRow` to always filter out the timestamp spec column.
   
   In 'normal' production code the timestamp spec is added to dimension exclusions so the code in `MapBasedInputRow` that computes dimensions to ensure time would not be in the dimensions list. However, in test code, especially in processing, which doesn't have access to the methods that take a dataschema and transform it into an input row schema, its pretty easy to not explicitly add timestamp column to the dimensions exclusion list. So as a result of not manually adding timestamp column to exclusions, it would end up in the dimensions list in schema discovery modes, as a string (or nested column, depending on config), which when doing rollup tests means it ends up as part of the rollup key, and so on. (Again this doesn't happen in current production code because it goes through that translator utility method).
   
   I made the change there to always filter the timestamp spec column from the dimensions list to make it easier to not write wrong tests for schema discovery mode, which caused the change here and other places.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136487914


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -145,6 +175,10 @@ public DimensionSelector makeDimensionSelector(
     final int dimIndex = desc.getIndex();
     final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
     if (rootLiteralSelector != null) {
+      final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+      if (root.getTypes().getSingleType().isArray()) {
+        throw new UnsupportedOperationException("Not supported");
+      }

Review Comment:
   heh, you are reading it correctly, currently the 'single typed root' dimension selector should only be used for scalar string columns, everything else should use the column value selector instead, will try to clarify with comments



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136488128


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -411,7 +445,7 @@ public Object getObject()
         if (0 <= dimIndex && dimIndex < dims.length) {
           final StructuredData data = (StructuredData) dims[dimIndex];
           if (data != null) {
-            return data.getValue();
+            return ExprEval.bestEffortOf(data.getValue()).value();

Review Comment:
   yeah, i did this to make it consistent with the value that is stored in the dictionary (since the selector is used for merging/persisting)



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148769259


##########
processing/src/main/java/org/apache/druid/math/expr/ExprEval.java:
##########
@@ -370,6 +370,18 @@ public static ExprEval ofComplex(ExpressionType outputType, @Nullable Object val
     return new ComplexExprEval(outputType, value);
   }
 
+  public static ExprEval bestEffortArray(@Nullable List<?> theList)
+  {
+    // do not convert empty lists to arrays with a single null element here, because that should have been done
+    // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately
+    // empty
+    NonnullPair<ExpressionType, Object[]> coerced = coerceListToArray(theList, false);
+    if (coerced == null) {
+      return bestEffortOf(null);

Review Comment:
   Calling `bestEffortOf(null)` looks to me like it's going to go through like 15 if statements, failing them all before just returning `new StringExprEval(null)`.  Why not just return the good thing here given that we already know what it should be and avoid the potential branches?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );
+          switch (columnType.getElementType().getType()) {
+            case LONG:
+              typeSet.add(ColumnType.LONG_ARRAY);
+              final Object[] longArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+              return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
+            case DOUBLE:
+              typeSet.add(ColumnType.DOUBLE_ARRAY);
+              final Object[] doubleArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray);
+              return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
+            case STRING:
+              final Object[] stringArray = eval.asArray();
+              // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
+              // the type set
+              if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
+                typeSet.add(ColumnType.STRING_ARRAY);
+              }
+              sizeEstimate = globalDimensionDictionary.addStringArray(stringArray);
+              return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
+            default:
+              throw new IAE("Unhandled type: %s", columnType);
+          }
+        case STRING:
+        default:

Review Comment:
   What's an example of when we hit the `default` case?  I'm legitimately asking because I do not know the answer and find myself wondering if believing that it's a String is really the right thing to do versus generating a parsing error or something of that nature.



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java:
##########
@@ -31,29 +32,36 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.WritableByteChannel;
+import java.util.Objects;
 import java.util.Set;
 
 /**
  * Binary serialization for nested field type info, translated into this compact format for storage in segments.
  * The index of the type info here is the same as the field index in {@link CompressedNestedDataComplexColumn#fields}
  */
-public class NestedLiteralTypeInfo
+public class NestedFieldTypeInfo
 {
   private static final byte STRING_MASK = 1;
   private static final byte LONG_MASK = 1 << 2;
   private static final byte DOUBLE_MASK = 1 << 3;
 
-  public static NestedLiteralTypeInfo read(ByteBuffer buffer, int length)
+  private static final byte STRING_ARRAY_MASK = 1 << 4;
+
+  private static final byte LONG_ARRAY_MASK = 1 << 5;
+
+  private static final byte DOUBLE_ARRAY_MASK = 1 << 6;

Review Comment:
   I dunno if I'm being stingy, but couldn't we have an `array` mask that is applied and then we check the type mask after that?  Would mean that we can add array support to this and only consume 1 more bit instead of consuming 3 more bits.



##########
processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java:
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * {@link Indexed} specialized for storing int arrays, which must be sorted and unique, using 'front coding'.
+ *
+ * Front coding is a type of delta encoding, where sorted values are grouped into buckets. The first value of the bucket
+ * is written entirely, and remaining values are stored as a pair of an integer which indicates how much of the first
+ * int array of the bucket to use as a prefix, followed by the remaining ints after the prefix to complete the value.
+ *
+ * front coded indexed layout:
+ * | version | bucket size | has null? | number of values | size of "offsets" + "buckets" | "offsets" | "buckets" |
+ * | ------- | ----------- | --------- | ---------------- | ----------------------------- | --------- | --------- |
+ * |    byte |        byte |      byte |        vbyte int |                     vbyte int |     int[] |  bucket[] |
+ *
+ * "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access.
+ *
+ * bucket layout:
+ * | first value | prefix length | fragment | ... | prefix length | fragment |
+ * | ----------- | ------------- | -------- | --- | ------------- | -------- |
+ * |       int[] |     vbyte int |    int[] | ... |     vbyte int |    int[] |
+ *
+ * int array layout:
+ * | length      |  ints |
+ * | ----------- | ----- |
+ * |   vbyte int | int[] |
+ *
+ *
+ * Getting a value first picks the appropriate bucket, finds its offset in the underlying buffer, then scans the bucket
+ * values to seek to the correct position of the value within the bucket in order to reconstruct it using the prefix
+ * length.
+ *
+ * Finding the index of a value involves binary searching the first values of each bucket to find the correct bucket,
+ * then a linear scan within the bucket to find the matching value (or negative insertion point -1 for values that
+ * are not present).
+ *
+ * The value iterator reads an entire bucket at a time, reconstructing the values into an array to iterate within the
+ * bucket before moving onto the next bucket as the iterator is consumed.
+ *
+ * This class is not thread-safe since during operation modifies positions of a shared buffer.
+ */
+public final class FrontCodedIntArrayIndexed implements Indexed<int[]>
+{
+  public static Supplier<FrontCodedIntArrayIndexed> read(ByteBuffer buffer, ByteOrder ordering)
+  {
+    final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering);
+    final byte version = orderedBuffer.get();
+    Preconditions.checkArgument(version == 0, "only V0 exists, encountered " + version);
+    final int bucketSize = Byte.toUnsignedInt(orderedBuffer.get());
+    final boolean hasNull = NullHandling.IS_NULL_BYTE == orderedBuffer.get();
+    final int numValues = VByte.readInt(orderedBuffer);
+    // size of offsets + values
+    final int size = VByte.readInt(orderedBuffer);
+    final int offsetsPosition = orderedBuffer.position();
+    // move position to end of buffer
+    buffer.position(offsetsPosition + size);
+
+    return () -> new FrontCodedIntArrayIndexed(
+        buffer,
+        ordering,
+        bucketSize,
+        numValues,
+        hasNull,
+        offsetsPosition
+    );
+  }
+
+  private final ByteBuffer buffer;
+  private final int adjustedNumValues;
+  private final int adjustIndex;
+  private final int bucketSize;
+  private final int numBuckets;
+  private final int div;
+  private final int rem;
+  private final int offsetsPosition;
+  private final int bucketsPosition;
+  private final boolean hasNull;
+  private final int lastBucketNumValues;
+
+  private FrontCodedIntArrayIndexed(
+      ByteBuffer buffer,
+      ByteOrder order,
+      int bucketSize,
+      int numValues,
+      boolean hasNull,
+      int offsetsPosition
+  )
+  {
+    if (Integer.bitCount(bucketSize) != 1) {
+      throw new ISE("bucketSize must be a power of two but was[%,d]", bucketSize);
+    }
+    this.buffer = buffer.asReadOnlyBuffer().order(order);
+    this.bucketSize = bucketSize;
+    this.hasNull = hasNull;
+
+    this.numBuckets = (int) Math.ceil((double) numValues / (double) bucketSize);
+    this.adjustIndex = hasNull ? 1 : 0;
+    this.adjustedNumValues = numValues + adjustIndex;
+    this.div = Integer.numberOfTrailingZeros(bucketSize);
+    this.rem = bucketSize - 1;
+    this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem;
+    this.offsetsPosition = offsetsPosition;
+    this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES);
+  }
+
+  @Override
+  public int size()
+  {
+    return adjustedNumValues;
+  }
+
+  @Nullable
+  @Override
+  public int[] get(int index)
+  {
+    if (hasNull && index == 0) {
+      return null;
+    }
+    Indexed.checkIndex(index, adjustedNumValues);
+
+    // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust
+    // the index

Review Comment:
   If I understood correctly, can I suggest updating the comment to be
   
   ```
   // For arrays, there is a conundrum of how to encode null different from the empty array.  In this code, 
   // we take the following approach:
   //   1) the 0 entry in our dictionary is assumed to be indicative of a completely null entry
   //   2) The 1 value in our dictionary is assumed to be the empty array
   //   3) Instead of storing the 0 value in this dictionary, we employ an indexing-shifting technique, where the
   //       dictionary never stores null, and starts with the empty array (index 1), but shifted by 1 such that what
   //       is persisted on disk is actually 0 instead of 1.
   // adjustIndex represents whether the null existed and thus whether we actually need to adjust the value
   ```
   



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -81,46 +87,75 @@ public class NestedDataColumnSerializer implements GenericColumnSerializer<Struc
   private final StructuredDataProcessor fieldProcessor = new StructuredDataProcessor()
   {
     @Override
-    public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedValue<?> processField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
       final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
           NestedPathFinder.toNormalizedJsonPath(fieldPath)
       );
       if (writer != null) {
         try {
-          ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+          final ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
           if (eval.type().isPrimitive() || (eval.type().isArray() && eval.type().getElementType().isPrimitive())) {
             writer.addValue(rowCount, eval.value());
           } else {
             // behave consistently with nested column indexer, which defaults to string
             writer.addValue(rowCount, eval.asString());
           }
           // serializer doesn't use size estimate
-          return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          return ProcessedValue.NULL_LITERAL;
         }
         catch (IOException e) {
-          throw new RuntimeException(":(");
+          throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, fieldValue);
         }
       }
-      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+      return ProcessedValue.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedValue<?> processArrayField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable List<?> array
+    )
+    {
+      final ExprEval<?> eval = ExprEval.bestEffortArray(array);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
+            NestedPathFinder.toNormalizedJsonPath(fieldPath)
+        );
+        if (writer != null) {
+          try {
+            writer.addValue(rowCount, eval.value());
+            // serializer doesn't use size estimate
+            return ProcessedValue.NULL_LITERAL;
+          }
+          catch (IOException e) {
+            throw new RE(e, "Failed to write field [%s] value [%s]", fieldPath, array);

Review Comment:
   This exception leaks the content of the data if it ever gets built.  I think the "best" we can do here is mention which field it was trying to write.



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -234,7 +272,9 @@ private GlobalDictionarySortedCollector getSortedIndexFromV1QueryableAdapterNest
     return new GlobalDictionarySortedCollector(
         new StringEncodingStrategies.Utf8ToStringIndexed(column.getStringDictionary()),
         column.getLongDictionary(),
-        column.getDoubleDictionary()
+        column.getDoubleDictionary(),
+        column.getArraysIterable(),
+        column.getArrayDictionary().size()

Review Comment:
   Sure, doesn't have to be an `Indexed`, just also seemed weird to be passing in fully encapsulated objects above and then suddenly start passing in 2 arguments to wrap a new thing that seems similar to the ones above...



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );

Review Comment:
   Nit: I think this would be better as an `if` statement.  If this ever gets thrown out, someone is gonna want to attach a debugger to the point that this gets thrown from and they are gonna need to convert it to an if statement to be able to do that without setting some conditions and stuff on their debug breakpoint.



##########
sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java:
##########
@@ -190,6 +190,7 @@ public static void setupNullValues()
 
   public static final Map<String, Object> QUERY_CONTEXT_NO_STRINGIFY_ARRAY =
       DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false)
+                                   .put(PlannerContext.CTX_ENABLE_UNNEST, true)

Review Comment:
   Not sure how much I care about this, but I expected this to be put on the Unnest/Array tests instead of across all of the tests in this base class.



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -426,55 +471,71 @@ public Class<?> classOfObject()
     };
   }
 
-  static class LiteralFieldIndexer
+  static class FieldIndexer
   {
     private final GlobalDimensionDictionary globalDimensionDictionary;
-    private final NestedLiteralTypeInfo.MutableTypeSet typeSet;
+    private final NestedFieldTypeInfo.MutableTypeSet typeSet;
 
-    LiteralFieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
+    FieldIndexer(GlobalDimensionDictionary globalDimensionDictionary)
     {
       this.globalDimensionDictionary = globalDimensionDictionary;
-      this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
+      this.typeSet = new NestedFieldTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedValue<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // sanity check, this should never happen
+          Preconditions.checkNotNull(
+              columnType.getElementType(),
+              "Array type [%s] for value [%s] missing element type, how did this possibly happen?",
+              eval.type(),
+              eval.valueOrDefault()
+          );
+          switch (columnType.getElementType().getType()) {
+            case LONG:
+              typeSet.add(ColumnType.LONG_ARRAY);
+              final Object[] longArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addLongArray(longArray);
+              return new StructuredDataProcessor.ProcessedValue<>(longArray, sizeEstimate);
+            case DOUBLE:
+              typeSet.add(ColumnType.DOUBLE_ARRAY);
+              final Object[] doubleArray = eval.asArray();
+              sizeEstimate = globalDimensionDictionary.addDoubleArray(doubleArray);
+              return new StructuredDataProcessor.ProcessedValue<>(doubleArray, sizeEstimate);
+            case STRING:
+              final Object[] stringArray = eval.asArray();
+              // empty arrays and arrays with all nulls are detected as string arrays, but dont count them as part of
+              // the type set
+              if (stringArray.length > 0 && !Arrays.stream(stringArray).allMatch(Objects::isNull)) {
+                typeSet.add(ColumnType.STRING_ARRAY);
+              }
+              sizeEstimate = globalDimensionDictionary.addStringArray(stringArray);
+              return new StructuredDataProcessor.ProcessedValue<>(stringArray, sizeEstimate);
+            default:
+              throw new IAE("Unhandled type: %s", columnType);

Review Comment:
   Just double checking, is this where we expect an array-of-arrays or an array-of-objects to end up?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -86,99 +94,129 @@ public NestedDataColumnMerger(
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter [%s]", adapter.getClass());
+        }
 
-    long dimStartTime = System.currentTimeMillis();
-
-    int numMergeIndex = 0;
-    GlobalDictionarySortedCollector sortedLookup = null;
-    final Indexed[] sortedLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+        boolean allNulls = dimValues == null || dimValues.allNull();
+        sortedLookup = dimValues;
+        if (!allNulls) {
+          sortedLookups[i] = dimValues.getSortedStrings();
+          sortedLongLookups[i] = dimValues.getSortedLongs();
+          sortedDoubleLookups[i] = dimValues.getSortedDoubles();
+          sortedArrayLookups[i] = dimValues.getSortedArrays();
+          numMergeIndex++;
+        }
+      }
 
-    final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+      descriptorBuilder = new ColumnDescriptor.Builder();
 
-    for (int i = 0; i < adapters.size(); i++) {
-      final IndexableAdapter adapter = adapters.get(i);
-      final GlobalDictionarySortedCollector dimValues;
-      if (adapter instanceof IncrementalIndexAdapter) {
-        dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
-      } else if (adapter instanceof QueryableIndexIndexableAdapter) {
-        dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+      final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
+          name,
+          indexSpec,
+          segmentWriteOutMedium,
+          progressIndicator,
+          closer
+      );
+      serializer = defaultSerializer;
+
+      final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
+                                                                     .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
+                                                                     .withDelegate(serializer)
+                                                                     .build();
+      descriptorBuilder.setValueType(ValueType.COMPLEX)
+                       .setHasMultipleValues(false)
+                       .addSerde(partSerde);
+
+      defaultSerializer.open();
+      defaultSerializer.serializeFields(mergedFields);
+
+      int stringCardinality;
+      int longCardinality;
+      int doubleCardinality;
+      int arrayCardinality;
+      if (numMergeIndex == 1) {
+        defaultSerializer.serializeDictionaries(
+            sortedLookup.getSortedStrings(),
+            sortedLookup.getSortedLongs(),
+            sortedLookup.getSortedDoubles(),
+            () -> new ArrayDictionaryMergingIterator(
+                sortedArrayLookups,
+                defaultSerializer.getGlobalLookup()
+            )
+        );
+        stringCardinality = sortedLookup.getStringCardinality();
+        longCardinality = sortedLookup.getLongCardinality();
+        doubleCardinality = sortedLookup.getDoubleCardinality();
+        arrayCardinality = sortedLookup.getArrayCardinality();
       } else {
-        throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+        final SimpleDictionaryMergingIterator<String> stringIterator = new SimpleDictionaryMergingIterator<>(
+            sortedLookups,
+            STRING_MERGING_COMPARATOR
+        );
+        final SimpleDictionaryMergingIterator<Long> longIterator = new SimpleDictionaryMergingIterator<>(
+            sortedLongLookups,
+            LONG_MERGING_COMPARATOR
+        );
+        final SimpleDictionaryMergingIterator<Double> doubleIterator = new SimpleDictionaryMergingIterator<>(
+            sortedDoubleLookups,
+            DOUBLE_MERGING_COMPARATOR
+        );
+        final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
+            sortedArrayLookups,
+            defaultSerializer.getGlobalLookup()
+        );
+        defaultSerializer.serializeDictionaries(
+            () -> stringIterator,
+            () -> longIterator,
+            () -> doubleIterator,
+            () -> arrayIterator
+        );
+        stringCardinality = stringIterator.getCardinality();
+        longCardinality = longIterator.getCardinality();
+        doubleCardinality = doubleIterator.getCardinality();
+        arrayCardinality = arrayIterator.getCardinality();
       }
 
-      boolean allNulls = allNull(dimValues.getSortedStrings()) &&
-                         allNull(dimValues.getSortedLongs()) &&
-                         allNull(dimValues.getSortedDoubles());
-      sortedLookup = dimValues;
-      if (!allNulls) {
-        sortedLookups[i] = dimValues.getSortedStrings();
-        sortedLongLookups[i] = dimValues.getSortedLongs();
-        sortedDoubleLookups[i] = dimValues.getSortedDoubles();
-        numMergeIndex++;
-      }
-    }
-
-    int cardinality = 0;
-    descriptorBuilder = new ColumnDescriptor.Builder();
-
-    final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
-        name,
-        indexSpec,
-        segmentWriteOutMedium,
-        progressIndicator,
-        closer
-    );
-    serializer = defaultSerializer;
-
-    final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
-                                                                   .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
-                                                                   .withDelegate(serializer)
-                                                                   .build();
-    descriptorBuilder.setValueType(ValueType.COMPLEX)
-                     .setHasMultipleValues(false)
-                     .addSerde(partSerde);
-
-    defaultSerializer.open();
-    defaultSerializer.serializeFields(mergedFields);
-
-    if (numMergeIndex > 1) {
-      SimpleDictionaryMergingIterator<String> dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedLookups,
-          STRING_MERGING_COMPARATOR
+      log.debug(
+          "Completed dim[%s] conversions with string cardinality[%,d], long cardinality[%,d], double cardinality[%,d], array cardinality[%,d] in %,d millis.",
+          name,
+          stringCardinality,
+          longCardinality,
+          doubleCardinality,
+          arrayCardinality,
+          System.currentTimeMillis() - dimStartTime
       );
-      SimpleDictionaryMergingIterator<Long> longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedLongLookups,
-          LONG_MERGING_COMPARATOR
-      );
-      SimpleDictionaryMergingIterator<Double> doubleDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedDoubleLookups,
-          DOUBLE_MERGING_COMPARATOR
-      );
-      defaultSerializer.serializeStringDictionary(() -> dictionaryMergeIterator);
-      defaultSerializer.serializeLongDictionary(() -> longDictionaryMergeIterator);
-      defaultSerializer.serializeDoubleDictionary(() -> doubleDictionaryMergeIterator);
-      cardinality = dictionaryMergeIterator.getCardinality();
-    } else if (numMergeIndex == 1) {
-      defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings());
-      defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs());
-      defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles());
-      cardinality = sortedLookup.size();
     }
-
-    log.debug(
-        "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.",
-        name,
-        cardinality,
-        System.currentTimeMillis() - dimStartTime
-    );
+    catch (Throwable ioe) {

Review Comment:
   Catching `Throwable` is dangerous, why cast such a wide net?



-- 
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


[GitHub] [druid] clintropolis merged pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis merged PR #13803:
URL: https://github.com/apache/druid/pull/13803


-- 
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


[GitHub] [druid] clintropolis commented on pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on PR #13803:
URL: https://github.com/apache/druid/pull/13803#issuecomment-1481862585

   I've fixed the behavior of `UNNEST` with regards to how it handles `null` and `[]` values for `ARRAY` typed inputs. Per experiments with postgres, `null` and empty arrays `[]` are supposed to output nothing when fed into `UNNEST`:
   
   <img width="1259" alt="Screenshot 2023-03-23 at 2 02 11 AM" src="https://user-images.githubusercontent.com/1577461/227349056-a3d6794c-dcba-4222-86ea-ecd9ceec65f5.png">
   
   <img width="1015" alt="Screenshot 2023-03-23 at 2 04 02 AM" src="https://user-images.githubusercontent.com/1577461/227349099-04e9c280-7ff4-4388-9e37-0708dad62cf4.png">
   
   <img width="1270" alt="Screenshot 2023-03-23 at 2 04 57 AM" src="https://user-images.githubusercontent.com/1577461/227349136-a7b5a2d5-ce81-4ef4-bfb2-42d6bb552687.png">
   
    but currently we are treating them as `[null]`, which is probably unfortunately what we have to keep doing for multi-value `STRING` types because we have trouble distinguishing `null`, `[]` and `[null]`, but no need to do this with actual arrays. So, I have fixed `UnnestColumnValueSelectorCursor` to consume `null` and `[]` values, fixing the behavior and added some tests.
   
   


-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141642926


##########
processing/src/test/resources/nested-array-test-data.json:
##########
@@ -0,0 +1,14 @@
+{"timestamp": "2023-01-01T00:00:00", "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],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "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], "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-01T00:00:00", "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],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],   "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-01T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],     "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-01T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,              "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "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],     "arrayObject":[{"x": 1},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "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], "arrayObject":[{"x": 3},{"x":4}]}
+{"timestamp": "2023-01-02T00:00:00", "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],      "arrayObject":[null,{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["d", "e"],       "arrayStringNulls": ["b", "b"],         "arrayLong":[1, 4],       "arrayLongNulls":[1],           "arrayDouble":[2.2, 3.3, 4.0],                                        "arrayVariant":["a", "b", "c"],   "arrayObject":[{"x": null},{"x":2}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": null,                                                     "arrayLong":[1, 2, 3],    "arrayLongNulls":null,          "arrayDouble":[1.1, 2.2, 3.3],  "arrayDoubleNulls":null,                                                "arrayObject":[{"x": 1000},{"y":2000}]}
+{"timestamp": "2023-01-02T00:00:00", "arrayString": ["a", "b"],       "arrayStringNulls": null,                                         "arrayLongNulls":[null, 2, 9],  "arrayDouble":null,             "arrayDoubleNulls":[999, 5.5, null],  "arrayVariant":["a", 1, 2.2],     "arrayObject":[{"a": 1},{"b":2}]}
+{"timestamp": "2023-01-02T00:00:00",                                  "arrayStringNulls": ["a", "b"],         "arrayLong":null,         "arrayLongNulls":[2, 3],                                        "arrayDoubleNulls":[null, 1.1],       "arrayVariant":null,              "arrayObject":[{"x": 1},{"x":2}]}

Review Comment:
   i'll add some tests for this, but assuming something like `ARRAY<ARRAY<STRING>>` there would be an array column for each outer array element



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148812179


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -86,99 +94,129 @@ public NestedDataColumnMerger(
   @Override
   public void writeMergedValueDictionary(List<IndexableAdapter> adapters) throws IOException
   {
+    try {
+      long dimStartTime = System.currentTimeMillis();
+
+      int numMergeIndex = 0;
+      GlobalDictionarySortedCollector sortedLookup = null;
+      final Indexed[] sortedLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
+      final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+      final Iterable<Object[]>[] sortedArrayLookups = new Iterable[adapters.size()];
+
+      final SortedMap<String, NestedFieldTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+
+      for (int i = 0; i < adapters.size(); i++) {
+        final IndexableAdapter adapter = adapters.get(i);
+        final GlobalDictionarySortedCollector dimValues;
+        if (adapter instanceof IncrementalIndexAdapter) {
+          dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
+        } else if (adapter instanceof QueryableIndexIndexableAdapter) {
+          dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+        } else {
+          throw new ISE("Unable to merge columns of unsupported adapter [%s]", adapter.getClass());
+        }
 
-    long dimStartTime = System.currentTimeMillis();
-
-    int numMergeIndex = 0;
-    GlobalDictionarySortedCollector sortedLookup = null;
-    final Indexed[] sortedLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedLongLookups = new Indexed[adapters.size()];
-    final Indexed[] sortedDoubleLookups = new Indexed[adapters.size()];
+        boolean allNulls = dimValues == null || dimValues.allNull();
+        sortedLookup = dimValues;
+        if (!allNulls) {
+          sortedLookups[i] = dimValues.getSortedStrings();
+          sortedLongLookups[i] = dimValues.getSortedLongs();
+          sortedDoubleLookups[i] = dimValues.getSortedDoubles();
+          sortedArrayLookups[i] = dimValues.getSortedArrays();
+          numMergeIndex++;
+        }
+      }
 
-    final SortedMap<String, NestedLiteralTypeInfo.MutableTypeSet> mergedFields = new TreeMap<>();
+      descriptorBuilder = new ColumnDescriptor.Builder();
 
-    for (int i = 0; i < adapters.size(); i++) {
-      final IndexableAdapter adapter = adapters.get(i);
-      final GlobalDictionarySortedCollector dimValues;
-      if (adapter instanceof IncrementalIndexAdapter) {
-        dimValues = getSortedIndexFromIncrementalAdapter((IncrementalIndexAdapter) adapter, mergedFields);
-      } else if (adapter instanceof QueryableIndexIndexableAdapter) {
-        dimValues = getSortedIndexesFromQueryableAdapter((QueryableIndexIndexableAdapter) adapter, mergedFields);
+      final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
+          name,
+          indexSpec,
+          segmentWriteOutMedium,
+          progressIndicator,
+          closer
+      );
+      serializer = defaultSerializer;
+
+      final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
+                                                                     .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
+                                                                     .withDelegate(serializer)
+                                                                     .build();
+      descriptorBuilder.setValueType(ValueType.COMPLEX)
+                       .setHasMultipleValues(false)
+                       .addSerde(partSerde);
+
+      defaultSerializer.open();
+      defaultSerializer.serializeFields(mergedFields);
+
+      int stringCardinality;
+      int longCardinality;
+      int doubleCardinality;
+      int arrayCardinality;
+      if (numMergeIndex == 1) {
+        defaultSerializer.serializeDictionaries(
+            sortedLookup.getSortedStrings(),
+            sortedLookup.getSortedLongs(),
+            sortedLookup.getSortedDoubles(),
+            () -> new ArrayDictionaryMergingIterator(
+                sortedArrayLookups,
+                defaultSerializer.getGlobalLookup()
+            )
+        );
+        stringCardinality = sortedLookup.getStringCardinality();
+        longCardinality = sortedLookup.getLongCardinality();
+        doubleCardinality = sortedLookup.getDoubleCardinality();
+        arrayCardinality = sortedLookup.getArrayCardinality();
       } else {
-        throw new ISE("Unable to merge columns of unsupported adapter %s", adapter.getClass());
+        final SimpleDictionaryMergingIterator<String> stringIterator = new SimpleDictionaryMergingIterator<>(
+            sortedLookups,
+            STRING_MERGING_COMPARATOR
+        );
+        final SimpleDictionaryMergingIterator<Long> longIterator = new SimpleDictionaryMergingIterator<>(
+            sortedLongLookups,
+            LONG_MERGING_COMPARATOR
+        );
+        final SimpleDictionaryMergingIterator<Double> doubleIterator = new SimpleDictionaryMergingIterator<>(
+            sortedDoubleLookups,
+            DOUBLE_MERGING_COMPARATOR
+        );
+        final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
+            sortedArrayLookups,
+            defaultSerializer.getGlobalLookup()
+        );
+        defaultSerializer.serializeDictionaries(
+            () -> stringIterator,
+            () -> longIterator,
+            () -> doubleIterator,
+            () -> arrayIterator
+        );
+        stringCardinality = stringIterator.getCardinality();
+        longCardinality = longIterator.getCardinality();
+        doubleCardinality = doubleIterator.getCardinality();
+        arrayCardinality = arrayIterator.getCardinality();
       }
 
-      boolean allNulls = allNull(dimValues.getSortedStrings()) &&
-                         allNull(dimValues.getSortedLongs()) &&
-                         allNull(dimValues.getSortedDoubles());
-      sortedLookup = dimValues;
-      if (!allNulls) {
-        sortedLookups[i] = dimValues.getSortedStrings();
-        sortedLongLookups[i] = dimValues.getSortedLongs();
-        sortedDoubleLookups[i] = dimValues.getSortedDoubles();
-        numMergeIndex++;
-      }
-    }
-
-    int cardinality = 0;
-    descriptorBuilder = new ColumnDescriptor.Builder();
-
-    final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
-        name,
-        indexSpec,
-        segmentWriteOutMedium,
-        progressIndicator,
-        closer
-    );
-    serializer = defaultSerializer;
-
-    final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
-                                                                   .withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
-                                                                   .withDelegate(serializer)
-                                                                   .build();
-    descriptorBuilder.setValueType(ValueType.COMPLEX)
-                     .setHasMultipleValues(false)
-                     .addSerde(partSerde);
-
-    defaultSerializer.open();
-    defaultSerializer.serializeFields(mergedFields);
-
-    if (numMergeIndex > 1) {
-      SimpleDictionaryMergingIterator<String> dictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedLookups,
-          STRING_MERGING_COMPARATOR
+      log.debug(
+          "Completed dim[%s] conversions with string cardinality[%,d], long cardinality[%,d], double cardinality[%,d], array cardinality[%,d] in %,d millis.",
+          name,
+          stringCardinality,
+          longCardinality,
+          doubleCardinality,
+          arrayCardinality,
+          System.currentTimeMillis() - dimStartTime
       );
-      SimpleDictionaryMergingIterator<Long> longDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedLongLookups,
-          LONG_MERGING_COMPARATOR
-      );
-      SimpleDictionaryMergingIterator<Double> doubleDictionaryMergeIterator = new SimpleDictionaryMergingIterator<>(
-          sortedDoubleLookups,
-          DOUBLE_MERGING_COMPARATOR
-      );
-      defaultSerializer.serializeStringDictionary(() -> dictionaryMergeIterator);
-      defaultSerializer.serializeLongDictionary(() -> longDictionaryMergeIterator);
-      defaultSerializer.serializeDoubleDictionary(() -> doubleDictionaryMergeIterator);
-      cardinality = dictionaryMergeIterator.getCardinality();
-    } else if (numMergeIndex == 1) {
-      defaultSerializer.serializeStringDictionary(sortedLookup.getSortedStrings());
-      defaultSerializer.serializeLongDictionary(sortedLookup.getSortedLongs());
-      defaultSerializer.serializeDoubleDictionary(sortedLookup.getSortedDoubles());
-      cardinality = sortedLookup.size();
     }
-
-    log.debug(
-        "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.",
-        name,
-        cardinality,
-        System.currentTimeMillis() - dimStartTime
-    );
+    catch (Throwable ioe) {

Review Comment:
   hmm, I don't remember why I added this, I think I was debugging something and needed an easy to place to catch to tell me what was failing for which column, and catching here seemed the best way to tell me what was messed up, but it could probably narrow it down to `IOException` and just rethrow it



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148809855


##########
processing/src/main/java/org/apache/druid/math/expr/ExprEval.java:
##########
@@ -370,6 +370,18 @@ public static ExprEval ofComplex(ExpressionType outputType, @Nullable Object val
     return new ComplexExprEval(outputType, value);
   }
 
+  public static ExprEval bestEffortArray(@Nullable List<?> theList)
+  {
+    // do not convert empty lists to arrays with a single null element here, because that should have been done
+    // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately
+    // empty
+    NonnullPair<ExpressionType, Object[]> coerced = coerceListToArray(theList, false);
+    if (coerced == null) {
+      return bestEffortOf(null);

Review Comment:
   I think it would probably be best to have null handled first in `bestEffortOf` so that if we ever decide to represent `null` as something more sensible (like introduce a null type), we won't be making a 'string' here, not to mention the saving of not running through a bunch of checks for a null value.



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148812003


##########
processing/src/main/java/org/apache/druid/math/expr/ExprEval.java:
##########
@@ -370,6 +370,18 @@ public static ExprEval ofComplex(ExpressionType outputType, @Nullable Object val
     return new ComplexExprEval(outputType, value);
   }
 
+  public static ExprEval bestEffortArray(@Nullable List<?> theList)
+  {
+    // do not convert empty lists to arrays with a single null element here, because that should have been done
+    // by the selectors preparing their ObjectBindings if necessary. If we get to this point it was legitimately
+    // empty
+    NonnullPair<ExpressionType, Object[]> coerced = coerceListToArray(theList, false);
+    if (coerced == null) {
+      return bestEffortOf(null);

Review Comment:
   Sure, I expected you to actually to create a method for `makeNullNode()` or something in `ExprEval` instead of newing it up directly here.  Just moving around the ordering of the statements is 🤷 .  Though, checking for `null` first is probably better than last.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136484273


##########
processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java:
##########
@@ -77,22 +76,36 @@ public static InputRow parse(InputRowSchema inputRowSchema, Map<String, Object>
    * 3) If isIncludeAllDimensions is not set and {@link DimensionsSpec#getDimensionNames()} is empty,
    *    the dimensions in the given map is returned.
    *
-   * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}.
+   * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}
+   * or {@link TimestampSpec#getTimestampColumn()}.
    */
   private static List<String> findDimensions(
+      TimestampSpec timestampSpec,
       DimensionsSpec dimensionsSpec,
       Map<String, Object> rawInputRow
   )
   {
     if (dimensionsSpec.isIncludeAllDimensions()) {
       LinkedHashSet<String> dimensions = new LinkedHashSet<>(dimensionsSpec.getDimensionNames());
-      dimensions.addAll(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions()));
+      for (String field : rawInputRow.keySet()) {
+        if (timestampSpec.getTimestampColumn().equals(field) || dimensionsSpec.getDimensionExclusions().contains(field)) {

Review Comment:
   yeah, can adjust, though after an earlier discussion i just want to remove `getDimensions` from `InputRow`, but haven't decided if in this PR or some later change. 



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136481675


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        // why not

Review Comment:
   oops, this needs removed



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146907717


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -251,33 +313,78 @@ public void serializeStringDictionary(Iterable<String> dictionaryValues) throws
       dictionaryWriter.write(value);
       globalDictionaryIdLookup.addString(value);
     }
+    stringDictionarySerialized = true;
   }
 
   public void serializeLongDictionary(Iterable<Long> dictionaryValues) throws IOException
   {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing long dictionary for column [%s]", name);
+    }
+    if (longDictionarySerialized) {
+      throw new ISE("Long dictionary already serialized for column [%s], cannot serialize again", name);
+    }
     for (Long value : dictionaryValues) {
       if (value == null) {
         continue;
       }
       longDictionaryWriter.write(value);
       globalDictionaryIdLookup.addLong(value);
     }
+    longDictionarySerialized = true;
   }
 
   public void serializeDoubleDictionary(Iterable<Double> dictionaryValues) throws IOException
   {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing double dictionary for column [%s]", name);
+    }
+    if (!longDictionarySerialized) {
+      throw new ISE("Must serialize long value dictionary before serializing double dictionary for column [%s]", name);
+    }
+    if (doubleDictionarySerialized) {
+      throw new ISE("Double dictionary already serialized for column [%s], cannot serialize again", name);
+    }
     for (Double value : dictionaryValues) {
       if (value == null) {
         continue;
       }
       doubleDictionaryWriter.write(value);
       globalDictionaryIdLookup.addDouble(value);
     }
+    doubleDictionarySerialized = true;
+  }
+
+  public void serializeArrayDictionary(Iterable<int[]> dictionaryValues) throws IOException
+  {
+    if (!stringDictionarySerialized) {
+      throw new ISE("Must serialize string value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (!longDictionarySerialized) {
+      throw new ISE("Must serialize long value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (!doubleDictionarySerialized) {
+      throw new ISE("Must serialize double value dictionary before serializing array dictionary for column [%s]", name);
+    }
+    if (arrayDictionarySerialized) {
+      throw new ISE("Array dictionary already serialized for column [%s], cannot serialize again", name);
+    }

Review Comment:
   have added `serializedDictionaries`



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146920835


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnMerger.java:
##########
@@ -280,4 +320,122 @@ private <T> boolean allNull(Indexed<T> dimValues)
     }
     return true;
   }
+
+  public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
+  {
+    private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =
+        (lhs, rhs) -> FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR.compare(lhs.peek(), rhs.peek());
+
+    protected final PriorityQueue<PeekingIterator<int[]>> pQueue;
+    protected int counter;
+
+    public ArrayDictionaryMergingIterator(Iterable<Object[]>[] dimValueLookups, GlobalDictionaryIdLookup idLookup)
+    {
+      pQueue = new PriorityQueue<>(PEEKING_ITERATOR_COMPARATOR);
+
+      for (Iterable<Object[]> dimValueLookup : dimValueLookups) {
+        if (dimValueLookup == null) {
+          continue;
+        }
+        final PeekingIterator<int[]> iter = Iterators.peekingIterator(
+            new IdLookupArrayIterator(idLookup, dimValueLookup.iterator())
+        );
+        if (iter.hasNext()) {
+          pQueue.add(iter);
+        }
+      }
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return !pQueue.isEmpty();
+    }
+
+    @Override
+    public int[] next()
+    {
+      PeekingIterator<int[]> smallest = pQueue.remove();
+      if (smallest == null) {
+        throw new NoSuchElementException();
+      }
+      final int[] value = smallest.next();
+      if (smallest.hasNext()) {
+        pQueue.add(smallest);
+      }
+
+      while (!pQueue.isEmpty() && Arrays.equals(value, pQueue.peek().peek())) {
+        PeekingIterator<int[]> same = pQueue.remove();
+        same.next();
+        if (same.hasNext()) {
+          pQueue.add(same);
+        }
+      }
+      counter++;
+
+      return value;
+    }
+
+    public int getCardinality()
+    {
+      return counter;
+    }
+
+    @Override
+    public void remove()
+    {
+      throw new UnsupportedOperationException("remove");
+    }
+  }
+
+  public static class IdLookupArrayIterator implements Iterator<int[]>
+  {
+    private final GlobalDictionaryIdLookup idLookup;
+    private final Iterator<Object[]> delegate;
+
+    public IdLookupArrayIterator(
+        GlobalDictionaryIdLookup idLookup,
+        Iterator<Object[]> delegate
+    )
+    {
+      this.idLookup = idLookup;
+      this.delegate = delegate;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+      return delegate.hasNext();
+    }
+
+    @Override
+    public int[] next()
+    {
+      final Object[] next = delegate.next();
+      if (next == null) {
+        return null;
+      }
+      final int[] newIdsWhoDis = new int[next.length];
+      for (int i = 0; i < next.length; i++) {
+        if (next[i] == null) {
+          newIdsWhoDis[i] = 0;
+        } else if (next[i] instanceof String) {
+          newIdsWhoDis[i] = idLookup.lookupString((String) next[i]);
+        } else if (next[i] instanceof Long) {
+          newIdsWhoDis[i] = idLookup.lookupLong((Long) next[i]);
+        } else if (next[i] instanceof Double) {
+          newIdsWhoDis[i] = idLookup.lookupDouble((Double) next[i]);
+        } else {
+          newIdsWhoDis[i] = -1;
+        }
+        Preconditions.checkArgument(
+            newIdsWhoDis[i] >= 0,
+            "unknown global id [%s] for value [%s]",
+            newIdsWhoDis[i],
+            next[i]
+        );

Review Comment:
   So, we are dealing with them here like this so we can lookup the new global id from the newly merged lower scalar value dictionaries. Otherwise we would need the mappings of old ids to new ids, which we don't currently have anywhere, and its a lot more complicated since its per segment. This way we just lookup the old values and after the lower dictionaries are merged, just lookup the array elements for the newly sorted values



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146926137


##########
processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java:
##########
@@ -134,6 +134,11 @@ public Map<String, ColumnAnalysis> analyze(Segment segment)
             analysis = analyzeStringColumn(capabilities, storageAdapter, columnName);
           }
           break;
+        case ARRAY:
+          // todo (clint): this is wack, but works for now because arrays are always nested complex columns...

Review Comment:
   split out into an `analyzeArrayColumn` that is a mostly empty stub



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141628246


##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -380,9 +551,61 @@ public VectorObjectSelector makeVectorObjectSelector(List<NestedPathPart> path,
     if (fields.indexOf(field) >= 0) {
       BaseColumn col = getColumnHolder(field).getColumn();
       return col.makeVectorObjectSelector(readableOffset);
-    } else {
-      return NilVectorSelector.create(readableOffset);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset);
+
+        return new VectorObjectSelector()
+        {
+          private final Object[] elements = new Object[arraySelector.getMaxVectorSize()];
+          private int id = ReadableVectorInspector.NULL_ID;
+
+          @Override
+          public Object[] getObjectVector()
+          {
+            if (readableOffset.getId() != id) {
+              final Object[] delegate = arraySelector.getObjectVector();
+              for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {

Review Comment:
   agree this isn't very optimized, i was planning to introduce some form of array selector in a future PR to optimize selecting individual elements, for now i was focusing on getting things functional.



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148814742


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java:
##########
@@ -31,29 +32,36 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.WritableByteChannel;
+import java.util.Objects;
 import java.util.Set;
 
 /**
  * Binary serialization for nested field type info, translated into this compact format for storage in segments.
  * The index of the type info here is the same as the field index in {@link CompressedNestedDataComplexColumn#fields}
  */
-public class NestedLiteralTypeInfo
+public class NestedFieldTypeInfo
 {
   private static final byte STRING_MASK = 1;
   private static final byte LONG_MASK = 1 << 2;
   private static final byte DOUBLE_MASK = 1 << 3;
 
-  public static NestedLiteralTypeInfo read(ByteBuffer buffer, int length)
+  private static final byte STRING_ARRAY_MASK = 1 << 4;
+
+  private static final byte LONG_ARRAY_MASK = 1 << 5;
+
+  private static final byte DOUBLE_ARRAY_MASK = 1 << 6;

Review Comment:
   what about composite arrays?  Like if the array has a String and a double?  Or a String and an Object?
   
   I had intepretted these as "this type exists in the column" not as "every row is this type".  In which case, if there is an array, then it is true that the array type exists in the column, no?  And then if the only scalar type is a `long`, then you would know that the arrays are all longs, right?



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148805135


##########
processing/src/main/java/org/apache/druid/segment/nested/NestedFieldTypeInfo.java:
##########
@@ -31,29 +32,36 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.WritableByteChannel;
+import java.util.Objects;
 import java.util.Set;
 
 /**
  * Binary serialization for nested field type info, translated into this compact format for storage in segments.
  * The index of the type info here is the same as the field index in {@link CompressedNestedDataComplexColumn#fields}
  */
-public class NestedLiteralTypeInfo
+public class NestedFieldTypeInfo
 {
   private static final byte STRING_MASK = 1;
   private static final byte LONG_MASK = 1 << 2;
   private static final byte DOUBLE_MASK = 1 << 3;
 
-  public static NestedLiteralTypeInfo read(ByteBuffer buffer, int length)
+  private static final byte STRING_ARRAY_MASK = 1 << 4;
+
+  private static final byte LONG_ARRAY_MASK = 1 << 5;
+
+  private static final byte DOUBLE_ARRAY_MASK = 1 << 6;

Review Comment:
   that would work if we only set 1 type, or if any row being an array meant all rows are arrays, but it doesn't currently, so we sort of need to know what type of array, at least the way things currently work



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148806031


##########
sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java:
##########
@@ -190,6 +190,7 @@ public static void setupNullValues()
 
   public static final Map<String, Object> QUERY_CONTEXT_NO_STRINGIFY_ARRAY =
       DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false)
+                                   .put(PlannerContext.CTX_ENABLE_UNNEST, true)

Review Comment:
   `QUERY_CONTEXT_NO_STRINGIFY_ARRAY` is afaik only used on `CalciteArrayQueryTest` and `CalciteNestedDataQueryTest` so i think this is fine



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1148815506


##########
processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java:
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * {@link Indexed} specialized for storing int arrays, which must be sorted and unique, using 'front coding'.
+ *
+ * Front coding is a type of delta encoding, where sorted values are grouped into buckets. The first value of the bucket
+ * is written entirely, and remaining values are stored as a pair of an integer which indicates how much of the first
+ * int array of the bucket to use as a prefix, followed by the remaining ints after the prefix to complete the value.
+ *
+ * front coded indexed layout:
+ * | version | bucket size | has null? | number of values | size of "offsets" + "buckets" | "offsets" | "buckets" |
+ * | ------- | ----------- | --------- | ---------------- | ----------------------------- | --------- | --------- |
+ * |    byte |        byte |      byte |        vbyte int |                     vbyte int |     int[] |  bucket[] |
+ *
+ * "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access.
+ *
+ * bucket layout:
+ * | first value | prefix length | fragment | ... | prefix length | fragment |
+ * | ----------- | ------------- | -------- | --- | ------------- | -------- |
+ * |       int[] |     vbyte int |    int[] | ... |     vbyte int |    int[] |
+ *
+ * int array layout:
+ * | length      |  ints |
+ * | ----------- | ----- |
+ * |   vbyte int | int[] |
+ *
+ *
+ * Getting a value first picks the appropriate bucket, finds its offset in the underlying buffer, then scans the bucket
+ * values to seek to the correct position of the value within the bucket in order to reconstruct it using the prefix
+ * length.
+ *
+ * Finding the index of a value involves binary searching the first values of each bucket to find the correct bucket,
+ * then a linear scan within the bucket to find the matching value (or negative insertion point -1 for values that
+ * are not present).
+ *
+ * The value iterator reads an entire bucket at a time, reconstructing the values into an array to iterate within the
+ * bucket before moving onto the next bucket as the iterator is consumed.
+ *
+ * This class is not thread-safe since during operation modifies positions of a shared buffer.
+ */
+public final class FrontCodedIntArrayIndexed implements Indexed<int[]>
+{
+  public static Supplier<FrontCodedIntArrayIndexed> read(ByteBuffer buffer, ByteOrder ordering)
+  {
+    final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering);
+    final byte version = orderedBuffer.get();
+    Preconditions.checkArgument(version == 0, "only V0 exists, encountered " + version);
+    final int bucketSize = Byte.toUnsignedInt(orderedBuffer.get());
+    final boolean hasNull = NullHandling.IS_NULL_BYTE == orderedBuffer.get();
+    final int numValues = VByte.readInt(orderedBuffer);
+    // size of offsets + values
+    final int size = VByte.readInt(orderedBuffer);
+    final int offsetsPosition = orderedBuffer.position();
+    // move position to end of buffer
+    buffer.position(offsetsPosition + size);
+
+    return () -> new FrontCodedIntArrayIndexed(
+        buffer,
+        ordering,
+        bucketSize,
+        numValues,
+        hasNull,
+        offsetsPosition
+    );
+  }
+
+  private final ByteBuffer buffer;
+  private final int adjustedNumValues;
+  private final int adjustIndex;
+  private final int bucketSize;
+  private final int numBuckets;
+  private final int div;
+  private final int rem;
+  private final int offsetsPosition;
+  private final int bucketsPosition;
+  private final boolean hasNull;
+  private final int lastBucketNumValues;
+
+  private FrontCodedIntArrayIndexed(
+      ByteBuffer buffer,
+      ByteOrder order,
+      int bucketSize,
+      int numValues,
+      boolean hasNull,
+      int offsetsPosition
+  )
+  {
+    if (Integer.bitCount(bucketSize) != 1) {
+      throw new ISE("bucketSize must be a power of two but was[%,d]", bucketSize);
+    }
+    this.buffer = buffer.asReadOnlyBuffer().order(order);
+    this.bucketSize = bucketSize;
+    this.hasNull = hasNull;
+
+    this.numBuckets = (int) Math.ceil((double) numValues / (double) bucketSize);
+    this.adjustIndex = hasNull ? 1 : 0;
+    this.adjustedNumValues = numValues + adjustIndex;
+    this.div = Integer.numberOfTrailingZeros(bucketSize);
+    this.rem = bucketSize - 1;
+    this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem;
+    this.offsetsPosition = offsetsPosition;
+    this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES);
+  }
+
+  @Override
+  public int size()
+  {
+    return adjustedNumValues;
+  }
+
+  @Nullable
+  @Override
+  public int[] get(int index)
+  {
+    if (hasNull && index == 0) {
+      return null;
+    }
+    Indexed.checkIndex(index, adjustedNumValues);
+
+    // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust
+    // the index

Review Comment:
   this is like the same code/comments as in `FrontCodedIndexed`, just adapted for `int[]` instead of `byte[]`, what is the empty array here is the empty string there, so it is the same problem. I guess i can update the comments/javadocs on both of them to make it more apparent of why we special handle null values and how it relates to `VByte` encoding



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141624189


##########
processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexed.java:
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.segment.data;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * {@link Indexed} specialized for storing int arrays, which must be sorted and unique, using 'front coding'.
+ *
+ * Front coding is a type of delta encoding, where sorted values are grouped into buckets. The first value of the bucket
+ * is written entirely, and remaining values are stored as a pair of an integer which indicates how much of the first
+ * int array of the bucket to use as a prefix, followed by the remaining ints after the prefix to complete the value.
+ *
+ * front coded indexed layout:
+ * | version | bucket size | has null? | number of values | size of "offsets" + "buckets" | "offsets" | "buckets" |
+ * | ------- | ----------- | --------- | ---------------- | ----------------------------- | --------- | --------- |
+ * |    byte |        byte |      byte |        vbyte int |                     vbyte int |     int[] |  bucket[] |
+ *
+ * "offsets" are the ending offsets of each bucket stored in order, stored as plain integers for easy random access.
+ *
+ * bucket layout:
+ * | first value | prefix length | fragment | ... | prefix length | fragment |
+ * | ----------- | ------------- | -------- | --- | ------------- | -------- |
+ * |       int[] |     vbyte int |    int[] | ... |     vbyte int |    int[] |
+ *
+ * int array layout:
+ * | length      |  ints |
+ * | ----------- | ----- |
+ * |   vbyte int | int[] |
+ *
+ *
+ * Getting a value first picks the appropriate bucket, finds its offset in the underlying buffer, then scans the bucket
+ * values to seek to the correct position of the value within the bucket in order to reconstruct it using the prefix
+ * length.
+ *
+ * Finding the index of a value involves binary searching the first values of each bucket to find the correct bucket,
+ * then a linear scan within the bucket to find the matching value (or negative insertion point -1 for values that
+ * are not present).
+ *
+ * The value iterator reads an entire bucket at a time, reconstructing the values into an array to iterate within the
+ * bucket before moving onto the next bucket as the iterator is consumed.
+ *
+ * This class is not thread-safe since during operation modifies positions of a shared buffer.
+ */
+public final class FrontCodedIntArrayIndexed implements Indexed<int[]>
+{
+  public static Supplier<FrontCodedIntArrayIndexed> read(ByteBuffer buffer, ByteOrder ordering)
+  {
+    final ByteBuffer orderedBuffer = buffer.asReadOnlyBuffer().order(ordering);
+    final byte version = orderedBuffer.get();
+    Preconditions.checkArgument(version == 0, "only V0 exists, encountered " + version);
+    final int bucketSize = Byte.toUnsignedInt(orderedBuffer.get());
+    final boolean hasNull = NullHandling.IS_NULL_BYTE == orderedBuffer.get();
+    final int numValues = VByte.readInt(orderedBuffer);
+    // size of offsets + values
+    final int size = VByte.readInt(orderedBuffer);
+    final int offsetsPosition = orderedBuffer.position();
+    // move position to end of buffer
+    buffer.position(offsetsPosition + size);
+
+    return () -> new FrontCodedIntArrayIndexed(
+        buffer,
+        ordering,
+        bucketSize,
+        numValues,
+        hasNull,
+        offsetsPosition
+    );
+  }
+
+  private final ByteBuffer buffer;
+  private final int adjustedNumValues;
+  private final int adjustIndex;
+  private final int bucketSize;
+  private final int numBuckets;
+  private final int div;
+  private final int rem;
+  private final int offsetsPosition;
+  private final int bucketsPosition;
+  private final boolean hasNull;
+  private final int lastBucketNumValues;
+
+  private FrontCodedIntArrayIndexed(
+      ByteBuffer buffer,
+      ByteOrder order,
+      int bucketSize,
+      int numValues,
+      boolean hasNull,
+      int offsetsPosition
+  )
+  {
+    if (Integer.bitCount(bucketSize) != 1) {
+      throw new ISE("bucketSize must be a power of two but was[%,d]", bucketSize);
+    }
+    this.buffer = buffer.asReadOnlyBuffer().order(order);
+    this.bucketSize = bucketSize;
+    this.hasNull = hasNull;
+
+    this.numBuckets = (int) Math.ceil((double) numValues / (double) bucketSize);
+    this.adjustIndex = hasNull ? 1 : 0;
+    this.adjustedNumValues = numValues + adjustIndex;
+    this.div = Integer.numberOfTrailingZeros(bucketSize);
+    this.rem = bucketSize - 1;
+    this.lastBucketNumValues = (numValues & rem) == 0 ? bucketSize : numValues & rem;
+    this.offsetsPosition = offsetsPosition;
+    this.bucketsPosition = offsetsPosition + ((numBuckets - 1) * Integer.BYTES);
+  }
+
+  @Override
+  public int size()
+  {
+    return adjustedNumValues;
+  }
+
+  @Nullable
+  @Override
+  public int[] get(int index)
+  {
+    if (hasNull && index == 0) {
+      return null;
+    }
+    Indexed.checkIndex(index, adjustedNumValues);
+
+    // due to vbyte encoding, the null value is not actually stored in the bucket (no negative values), so we adjust
+    // the index

Review Comment:
   null is stored as a flag outside of the bucket because there is no way to store inside (we would typically use a length of -1 to indicate a null value, but since we cannot represent negative numbers it uses an external flag and index shifting. 0 length value is an empty array here and so separate from null.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141627649


##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -327,17 +390,58 @@ public void close()
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
     if (fields.indexOf(field) >= 0) {
       DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {
+        final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex();
+        if (elementNumber < 0) {
+          throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path);
+        }
+        DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(arrayField).getColumn();
+        ColumnValueSelector<?> arraySelector = col.makeColumnValueSelector(readableOffset);
+        return new BaseSingleValueDimensionSelector()
+        {
+          @Nullable
+          @Override
+          protected String getValue()
+          {
+            Object o = arraySelector.getObject();
+            if (o instanceof Object[]) {
+              Object[] array = (Object[]) o;
+              if (elementNumber < array.length) {
+                Object element = array[elementNumber];
+                if (element == null) {
+                  return null;
+                }
+                return String.valueOf(element);
+              }
+            }
+            return null;
+          }
+
+          @Override
+          public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+          {
+            arraySelector.inspectRuntimeShape(inspector);
+          }
+        };

Review Comment:
   the dictionary ids here are complete arrays, not the elements of the array, dimension selector doesn't really work for this because lookupName returns a string for a dictionary id instead of ARRAY<STRING>/ARRAY<LONG>/ARRAY<DOUBLE> array columns we have
   



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141632675


##########
processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java:
##########
@@ -509,50 +507,6 @@ public void createIndex(
     }
   }
 
-  public void createIndex(
-      InputStream inputDataStream,
-      String parserJson,
-      String transformSpecJson,
-      String aggregators,
-      File outDir,
-      long minTimestamp,
-      Granularity gran,
-      int maxRowCount,
-      boolean rollup
-  ) throws Exception
-  {
-    try {
-      StringInputRowParser parser = mapper.readValue(parserJson, StringInputRowParser.class);
-      TransformSpec transformSpec;
-      if (transformSpecJson != null) {
-        transformSpec = mapper.readValue(transformSpecJson, TransformSpec.class);
-        parser = new TransformingStringInputRowParser(parser.getParseSpec(), parser.getEncoding(), transformSpec);
-      }
-
-      LineIterator iter = IOUtils.lineIterator(inputDataStream, "UTF-8");
-      List<AggregatorFactory> aggregatorSpecs = mapper.readValue(
-          aggregators,
-          new TypeReference<List<AggregatorFactory>>()
-          {
-          }
-      );
-
-      createIndex(
-          iter,
-          parser,
-          aggregatorSpecs.toArray(new AggregatorFactory[0]),
-          outDir,
-          minTimestamp,
-          gran,
-          true,
-          maxRowCount,
-          rollup
-      );
-    }
-    finally {
-      Closeables.close(inputDataStream, true);
-    }
-  }

Review Comment:
   Hmm, I added this one in the PR that introduced nested columns. The same functionality that many of the methods that create segments in `AggregationTestHelper ` have is done better in `IndexBuilder`, which is why I focused on improving it and switching nested column tests to using it.
   
   There were no users of this method after i refactored my tests so I removed it. I can always add it back if you're worried about it, but imo everyone should use `IndexBuilder` and we should deprecate most of `AggregationTestHelper`, at least the parts around making segments. I didn't not migrate the other users away from it in this PR though.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146908747


##########
sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java:
##########
@@ -705,6 +705,23 @@ public void testQuery(
         .run();
   }
 
+  public void testQuery(
+      final String sql,
+      final Map<String, Object> queryContext,
+      final List<Query<?>> expectedQueries,
+      final List<Object[]> expectedResults,
+      final RowSignature expectedResultSignature
+  )
+  {
+    testBuilder()
+        .sql(sql)
+        .queryContext(queryContext)
+        .expectedQueries(expectedQueries)
+        .expectedResults(expectedResults)
+        .expectedSignature(expectedResultSignature)
+        .run();
+  }

Review Comment:
   removed in favor of just calling `testBuilder`, this should probably just be done everywhere, but its kind of tedious to switch



-- 
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


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146831847


##########
processing/src/main/java/org/apache/druid/segment/UnnestColumnValueSelectorCursor.java:
##########
@@ -307,14 +320,8 @@
    */
   private void advanceAndUpdate()
   {
-    if (unnestListForCurrentRow.isEmpty() || index >= unnestListForCurrentRow.size() - 1) {
-      index = 0;
-      baseCursor.advance();
-      if (!baseCursor.isDone()) {
-        getNextRow();
-      }
-    } else {
-      index++;
+    if (++index >= unnestListForCurrentRow.size()) {

Review Comment:
   ## User-controlled data in arithmetic expression
   
   This arithmetic expression depends on a [user-provided value](1), potentially causing an overflow.
   This arithmetic expression depends on a [user-provided value](2), potentially causing an overflow.
   This arithmetic expression depends on a [user-provided value](3), potentially causing an overflow.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4484)



##########
processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java:
##########
@@ -385,4 +389,28 @@
       );
     }
   }
+
+  private ColumnAnalysis analyzeArrayColumn(
+      @Nullable final ColumnCapabilities capabilities,
+      final int numCells,
+      @Nullable final ColumnHolder columnHolder

Review Comment:
   ## Useless parameter
   
   The parameter 'columnHolder' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4483)



##########
processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java:
##########
@@ -385,4 +389,28 @@
       );
     }
   }
+
+  private ColumnAnalysis analyzeArrayColumn(
+      @Nullable final ColumnCapabilities capabilities,
+      final int numCells,

Review Comment:
   ## Useless parameter
   
   The parameter 'numCells' is never used.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4482)



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146897221


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        // why not
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+        LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new LiteralFieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(eval);
       }
-      return fieldIndexer.processValue(fieldValue);
+      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedLiteral<?> processArrayOfLiteralsField(
+        ArrayList<NestedPathPart> fieldPath,
+        Object maybeArrayOfLiterals
+    )
+    {
+      final ExprEval<?> maybeLiteralArray = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+      if (maybeLiteralArray.type().isArray() && maybeLiteralArray.type().getElementType().isPrimitive()) {
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new LiteralFieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(maybeLiteralArray);
+      }
+      return null;

Review Comment:
   i modified this stuff a bit and updated javadocs so it is hopefully clearer, the new abstract method names are `processField` and `processArrayField` and the latter indicates that returning a non-null value halts further processing of arrays, otherwise the processor will continue for each element of the array



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -145,6 +175,10 @@ public DimensionSelector makeDimensionSelector(
     final int dimIndex = desc.getIndex();
     final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
     if (rootLiteralSelector != null) {
+      final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+      if (root.getTypes().getSingleType().isArray()) {
+        throw new UnsupportedOperationException("Not supported");
+      }

Review Comment:
   clarified exception



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146901232


##########
processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java:
##########
@@ -480,11 +480,25 @@ public Expr apply(List<Expr> args)
       final StructuredDataProcessor processor = new StructuredDataProcessor()
       {
         @Override
-        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
         {
           // do nothing, we only want the list of fields returned by this processor
           return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
         }
+
+        @Nullable
+        @Override
+        public ProcessedLiteral<?> processArrayOfLiteralsField(
+            ArrayList<NestedPathPart> fieldPath,
+            @Nullable Object maybeArrayOfLiterals
+        )
+        {
+          ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+          if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+            return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          }

Review Comment:
   updated javadocs and comments on stuff so is hopefully clearer



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146897593


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        // why not

Review Comment:
   removed stale comment



-- 
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


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146950014


##########
processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java:
##########
@@ -134,6 +134,10 @@
             analysis = analyzeStringColumn(capabilities, storageAdapter, columnName);
           }
           break;
+        case ARRAY:
+          final ColumnHolder arrayHolder = index != null ? index.getColumnHolder(columnName) : null;

Review Comment:
   ## Unread local variable
   
   Variable 'ColumnHolder arrayHolder' is never read.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4485)



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1135011007


##########
processing/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java:
##########
@@ -41,6 +42,18 @@
 public class DelimitedInputFormat extends FlatTextInputFormat
 {
   public static final String TYPE_KEY = "tsv";
+
+  public static DelimitedInputFormat ofColumns(String... columns)
+  {
+    return new DelimitedInputFormat(
+        Arrays.asList(columns),
+        null,
+        null,
+        false,
+        false,
+        0
+    );
+  }

Review Comment:
   Location nit: does this need to be in `main` rather than `test`?



##########
processing/src/main/java/org/apache/druid/data/input/impl/MapInputRowParser.java:
##########
@@ -77,22 +76,36 @@ public static InputRow parse(InputRowSchema inputRowSchema, Map<String, Object>
    * 3) If isIncludeAllDimensions is not set and {@link DimensionsSpec#getDimensionNames()} is empty,
    *    the dimensions in the given map is returned.
    *
-   * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}.
+   * In any case, the returned list does not include any dimensions in {@link DimensionsSpec#getDimensionExclusions()}
+   * or {@link TimestampSpec#getTimestampColumn()}.
    */
   private static List<String> findDimensions(
+      TimestampSpec timestampSpec,
       DimensionsSpec dimensionsSpec,
       Map<String, Object> rawInputRow
   )
   {
     if (dimensionsSpec.isIncludeAllDimensions()) {
       LinkedHashSet<String> dimensions = new LinkedHashSet<>(dimensionsSpec.getDimensionNames());
-      dimensions.addAll(Sets.difference(rawInputRow.keySet(), dimensionsSpec.getDimensionExclusions()));
+      for (String field : rawInputRow.keySet()) {
+        if (timestampSpec.getTimestampColumn().equals(field) || dimensionsSpec.getDimensionExclusions().contains(field)) {

Review Comment:
   can we just get these two things once?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -437,41 +471,51 @@ public Class<?> classOfObject()
       this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedLiteral<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedLiteral<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedLiteral<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // skip empty arrays for now, they will always be called 'string' arrays, which isn't very helpful here since
+          // it will pollute the type set
+          Preconditions.checkNotNull(columnType.getElementType(), "Array element type must not be null");

Review Comment:
   If this were to ever happen, I think I'd want to know which field was the bad one.



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -411,7 +445,7 @@ public Object getObject()
         if (0 <= dimIndex && dimIndex < dims.length) {
           final StructuredData data = (StructuredData) dims[dimIndex];
           if (data != null) {
-            return data.getValue();
+            return ExprEval.bestEffortOf(data.getValue()).value();

Review Comment:
   Is this counting on coercion or something?



##########
processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java:
##########
@@ -41,6 +41,14 @@ public class JsonInputFormat extends NestedInputFormat
 {
   public static final String TYPE_KEY = "json";
 
+  public static final JsonInputFormat DEFAULT = new JsonInputFormat(
+      JSONPathSpec.DEFAULT,
+      null,
+      null,
+      null,
+      null
+  );
+

Review Comment:
   Location nit: does this need to be in `main` rather than `test`?



##########
processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java:
##########
@@ -134,6 +134,11 @@ public Map<String, ColumnAnalysis> analyze(Segment segment)
             analysis = analyzeStringColumn(capabilities, storageAdapter, columnName);
           }
           break;
+        case ARRAY:
+          // todo (clint): this is wack, but works for now because arrays are always nested complex columns...

Review Comment:
   Instead of a todo like this.  How about an if check that validates that it's a nested column and, if it's not a nested column, throws an Exception that is clearly saying that the code is currently written assuming that the arrays are delivered only by nested columns?



##########
processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java:
##########
@@ -480,11 +480,25 @@ public Expr apply(List<Expr> args)
       final StructuredDataProcessor processor = new StructuredDataProcessor()
       {
         @Override
-        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
         {
           // do nothing, we only want the list of fields returned by this processor
           return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
         }
+
+        @Nullable
+        @Override
+        public ProcessedLiteral<?> processArrayOfLiteralsField(
+            ArrayList<NestedPathPart> fieldPath,
+            @Nullable Object maybeArrayOfLiterals
+        )
+        {
+          ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+          if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+            return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          }

Review Comment:
   The if condition makes me believe that we found an array and should be joyously celebrating, but the return value is a `NULL_LITERAL`, why is it correct to return a `NULL_LITERAL` and how is that different from just returning `null`?



##########
extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/NestedColumnParquetReaderTest.java:
##########
@@ -181,7 +181,7 @@ public void testNestedColumnSchemalessNestedTestFileNoNested() throws IOExceptio
     );
 
     List<InputRow> rows = readAllRows(reader);
-    Assert.assertEquals(ImmutableList.of("dim1", "metric1", "timestamp"), rows.get(0).getDimensions());
+    Assert.assertEquals(ImmutableList.of("dim1", "metric1"), rows.get(0).getDimensions());

Review Comment:
   Why the change in expectation?



##########
processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java:
##########
@@ -480,11 +480,25 @@ public Expr apply(List<Expr> args)
       final StructuredDataProcessor processor = new StructuredDataProcessor()
       {
         @Override
-        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
         {
           // do nothing, we only want the list of fields returned by this processor
           return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
         }
+
+        @Nullable
+        @Override
+        public ProcessedLiteral<?> processArrayOfLiteralsField(
+            ArrayList<NestedPathPart> fieldPath,
+            @Nullable Object maybeArrayOfLiterals
+        )
+        {
+          ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);

Review Comment:
   Would it make sense to break `ExprEval.bestEffortOf` into a bunch of checks for different groups of expected types (i.e. `ExprEval.maybeLiteral()` and `ExprEval.maybeArray`, etc.).  Calls to `bestEffortOf` can cascade through, but places like this that already know some of what they expect can call the one that more aligns with expectations?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        // why not
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        ExprEval<?> eval = ExprEval.bestEffortOf(fieldValue);
+        LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new LiteralFieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(eval);
       }
-      return fieldIndexer.processValue(fieldValue);
+      return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+    }
+
+    @Nullable
+    @Override
+    public ProcessedLiteral<?> processArrayOfLiteralsField(
+        ArrayList<NestedPathPart> fieldPath,
+        Object maybeArrayOfLiterals
+    )
+    {
+      final ExprEval<?> maybeLiteralArray = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+      if (maybeLiteralArray.type().isArray() && maybeLiteralArray.type().getElementType().isPrimitive()) {
+        final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
+        LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
+        if (fieldIndexer == null) {
+          estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
+          fieldIndexer = new LiteralFieldIndexer(globalDictionary);
+          fieldIndexers.put(fieldName, fieldIndexer);
+        }
+        return fieldIndexer.processValue(maybeLiteralArray);
+      }
+      return null;

Review Comment:
   This looks a lot like code in NestedDataExpressions, except this doesn't return the `NULL_LITERAL`.  I find myself wondering if the NestedDataExpressions code shouldn't look like this?



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -59,16 +62,43 @@ public class NestedDataColumnIndexer implements DimensionIndexer<StructuredData,
   protected final StructuredDataProcessor indexerProcessor = new StructuredDataProcessor()
   {
     @Override
-    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+    public ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
     {
-      final String fieldName = NestedPathFinder.toNormalizedJsonPath(fieldPath);
-      LiteralFieldIndexer fieldIndexer = fieldIndexers.get(fieldName);
-      if (fieldIndexer == null) {
-        estimatedFieldKeySize += StructuredDataProcessor.estimateStringSize(fieldName);
-        fieldIndexer = new LiteralFieldIndexer(globalDictionary);
-        fieldIndexers.put(fieldName, fieldIndexer);
+      // null value is always added to the global dictionary as id 0, so we can ignore them here
+      if (fieldValue != null) {
+        // why not

Review Comment:
   Because I don't know "why?"  (if this comment is attempting to add information, more words please).



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -145,6 +175,10 @@ public DimensionSelector makeDimensionSelector(
     final int dimIndex = desc.getIndex();
     final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
     if (rootLiteralSelector != null) {
+      final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+      if (root.getTypes().getSingleType().isArray()) {
+        throw new UnsupportedOperationException("Not supported");
+      }

Review Comment:
   I'm reading this as "if all we have are root-level entries and they are always arrays, then throw a UOE exception".  I'm pretty sure I'm reading it wrong, but wishing the error message gave me more context without me feeling like I need to expand lines on the review to know what this is validating.



##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -437,41 +471,51 @@ public Class<?> classOfObject()
       this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedLiteral<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedLiteral<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedLiteral<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // skip empty arrays for now, they will always be called 'string' arrays, which isn't very helpful here since
+          // it will pollute the type set

Review Comment:
   How does it do the skipping of empties?



-- 
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


[GitHub] [druid] clintropolis commented on pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on PR #13803:
URL: https://github.com/apache/druid/pull/13803#issuecomment-1434306736

   I want to get #13809 finished first and pull those changes into this PR to make things consistent with null value coercion


-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1136483400


##########
processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java:
##########
@@ -480,11 +480,25 @@ public Expr apply(List<Expr> args)
       final StructuredDataProcessor processor = new StructuredDataProcessor()
       {
         @Override
-        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, Object fieldValue)
+        public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList<NestedPathPart> fieldPath, @Nullable Object fieldValue)
         {
           // do nothing, we only want the list of fields returned by this processor
           return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
         }
+
+        @Nullable
+        @Override
+        public ProcessedLiteral<?> processArrayOfLiteralsField(
+            ArrayList<NestedPathPart> fieldPath,
+            @Nullable Object maybeArrayOfLiterals
+        )
+        {
+          ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+          if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+            return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          }

Review Comment:
   see other comment about the contracts of the process literal methods. Here the json_paths expression doesn't care about the literal size estimations, so it just ignores them, since all it is really looking for is the `StructuredDataProcessor.ProcessResults` to get the set of paths. Returning non-null on the array method means that the processing on that path stops, so the path is to the array, not a separate path for each array element.
   
   I'll try to explain a bit better with comments



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141629623


##########
processing/src/main/java/org/apache/druid/segment/nested/GlobalDimensionDictionary.java:
##########
@@ -42,45 +47,107 @@
   private final ComparatorDimensionDictionary<String> stringDictionary;
   private final ComparatorDimensionDictionary<Long> longDictionary;
   private final ComparatorDimensionDictionary<Double> doubleDictionary;
+  private final Set<Object[]> stringArrays;
+  private final Set<Object[]> longArrays;
+  private final Set<Object[]> doubleArrays;

Review Comment:
   this is also used for merging segments, so the original ints they referred to might have been remapped when the string/long/double dictionaries are merged beneath them, so we have to round trip like this to lookup the new values



##########
processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java:
##########
@@ -104,6 +108,32 @@ public StructuredDataProcessor.ProcessedLiteral<?> processLiteralField(ArrayList
       }
       return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
     }
+
+    @Nullable
+    @Override
+    public ProcessedLiteral<?> processArrayOfLiteralsField(
+        ArrayList<NestedPathPart> fieldPath,
+        @Nullable Object maybeArrayOfLiterals
+    )
+    {
+      ExprEval<?> eval = ExprEval.bestEffortOf(maybeArrayOfLiterals);
+      if (eval.type().isArray() && eval.type().getElementType().isPrimitive()) {
+        final GlobalDictionaryEncodedFieldColumnWriter<?> writer = fieldWriters.get(
+            NestedPathFinder.toNormalizedJsonPath(fieldPath)
+        );
+        if (writer != null) {
+          try {
+            writer.addValue(rowCount, eval.value());
+            // serializer doesn't use size estimate
+            return StructuredDataProcessor.ProcessedLiteral.NULL_LITERAL;
+          }
+          catch (IOException e) {
+            throw new RuntimeException(":(");

Review Comment:
   oops yes, this was a placeholder



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141629174


##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -426,6 +765,12 @@ public ColumnIndexSupplier getColumnIndexSupplier(List<NestedPathPart> path)
   {
     final String field = getField(path);
     if (fields.indexOf(field) < 0) {
+      if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+        final String arrayField = getField(path.subList(0, path.size() - 1));
+        if (fields.indexOf(arrayField) >= 0) {
+          return NoIndexesColumnIndexSupplier.getInstance();
+        }

Review Comment:
   no, we are writing out some value indexes for both whole arrays and individual elements, I just haven't wired everything up to native filters yet to take advantage of them so i'm forcing it to use value matchers until then so they work correctly



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141626378


##########
processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java:
##########
@@ -101,7 +101,7 @@ public Sequence<Cursor> makeCursors(
           Cursor retVal = cursor;
           ColumnCapabilities capabilities = cursor.getColumnSelectorFactory().getColumnCapabilities(dimensionToUnnest);
           if (capabilities != null) {
-            if (capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {
+            if (!capabilities.isArray() && capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue()) {

Review Comment:
   so, this array column is dictionary encoded, but is not dictionary encoded in the correct way for the unnest dimension cursor, instead the dictionary ids stored in the column are the entire arrays id. Unnest will try to incorrectly use a dimension selector for these array columns.
   
   Are there things legitimately typed as ARRAY but should use a dimension selector?



-- 
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


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "imply-cheddar (via GitHub)" <gi...@apache.org>.
imply-cheddar commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1141567707


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -145,6 +175,10 @@ public DimensionSelector makeDimensionSelector(
     final int dimIndex = desc.getIndex();
     final ColumnValueSelector<?> rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex);
     if (rootLiteralSelector != null) {
+      final LiteralFieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT);
+      if (root.getTypes().getSingleType().isArray()) {
+        throw new UnsupportedOperationException("Not supported");
+      }

Review Comment:
   Or put more words into the exception message so it's clear what's not supported?



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146894440


##########
processing/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java:
##########
@@ -41,6 +41,14 @@ public class JsonInputFormat extends NestedInputFormat
 {
   public static final String TYPE_KEY = "json";
 
+  public static final JsonInputFormat DEFAULT = new JsonInputFormat(
+      JSONPathSpec.DEFAULT,
+      null,
+      null,
+      null,
+      null
+  );
+

Review Comment:
   removed



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146902269


##########
processing/src/main/java/org/apache/druid/segment/NestedDataColumnIndexer.java:
##########
@@ -437,41 +471,51 @@ public Class<?> classOfObject()
       this.typeSet = new NestedLiteralTypeInfo.MutableTypeSet();
     }
 
-    private StructuredDataProcessor.ProcessedLiteral<?> processValue(@Nullable Object value)
+    private StructuredDataProcessor.ProcessedLiteral<?> processValue(ExprEval<?> eval)
     {
-      // null value is always added to the global dictionary as id 0, so we can ignore them here
-      if (value != null) {
-        // why not
-        ExprEval<?> eval = ExprEval.bestEffortOf(value);
-        final ColumnType columnType = ExpressionType.toColumnType(eval.type());
-
-        switch (columnType.getType()) {
-          case LONG:
-            globalDimensionDictionary.addLongValue(eval.asLong());
-            typeSet.add(ColumnType.LONG);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asLong(),
-                StructuredDataProcessor.getLongObjectEstimateSize()
-            );
-          case DOUBLE:
-            globalDimensionDictionary.addDoubleValue(eval.asDouble());
-            typeSet.add(ColumnType.DOUBLE);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asDouble(),
-                StructuredDataProcessor.getDoubleObjectEstimateSize()
-            );
-          case STRING:
-          default:
-            final String asString = eval.asString();
-            globalDimensionDictionary.addStringValue(asString);
-            typeSet.add(ColumnType.STRING);
-            return new StructuredDataProcessor.ProcessedLiteral<>(
-                eval.asString(),
-                StructuredDataProcessor.estimateStringSize(asString)
-            );
-        }
+      final ColumnType columnType = ExpressionType.toColumnType(eval.type());
+      int sizeEstimate;
+      switch (columnType.getType()) {
+        case LONG:
+          typeSet.add(ColumnType.LONG);
+          sizeEstimate = globalDimensionDictionary.addLongValue(eval.asLong());
+          return new StructuredDataProcessor.ProcessedLiteral<>(eval.asLong(), sizeEstimate);
+        case DOUBLE:
+          typeSet.add(ColumnType.DOUBLE);
+          sizeEstimate = globalDimensionDictionary.addDoubleValue(eval.asDouble());
+          return new StructuredDataProcessor.ProcessedLiteral<>(eval.asDouble(), sizeEstimate);
+        case ARRAY:
+          // skip empty arrays for now, they will always be called 'string' arrays, which isn't very helpful here since
+          // it will pollute the type set

Review Comment:
   it was missing some code to do that, updated and added tests for nulls, empties, and arrays of nulls



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146910014


##########
processing/src/test/java/org/apache/druid/segment/IndexBuilder.java:
##########
@@ -267,4 +395,44 @@ private static IncrementalIndex buildIncrementalIndexWithRows(
     }
     return incrementalIndex;
   }
+
+  private static IncrementalIndex buildIncrementalIndexWithInputSource(
+      IncrementalIndexSchema schema,
+      InputSource inputSource,
+      InputFormat inputFormat,
+      @Nullable TransformSpec transformSpec,
+      File inputSourceTmpDir,
+      int maxRows
+  )
+  {
+    Preconditions.checkNotNull(schema, "schema");
+    Preconditions.checkNotNull(inputSource, "inputSource");
+    Preconditions.checkNotNull(inputFormat, "inputFormat");
+    Preconditions.checkNotNull(inputSourceTmpDir, "inputSourceTmpDir");
+
+    final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder()
+        .setIndexSchema(schema)
+        .setMaxRowCount(maxRows)
+        .build();
+    TransformSpec tranformer = transformSpec != null ? transformSpec : TransformSpec.NONE;
+    InputRowSchema rowSchema = new InputRowSchema(schema.getTimestampSpec(), schema.getDimensionsSpec(), null);
+    InputSourceReader reader = inputSource.reader(rowSchema, inputFormat, inputSourceTmpDir);
+    InputSourceReader transformingReader = tranformer.decorate(reader);
+    try (CloseableIterator<InputRow> rowIterator = transformingReader.read()) {
+      while (rowIterator.hasNext()) {
+        incrementalIndex.add(rowIterator.next());

Review Comment:
   Hmm, good question, this looks wired up to the `maxRowCount` but callers are not checking `canAppendRow` or anything like that and `buildIncrementalIndex` can only return a single `IncrementalIndex` so I think all it can do is explode. There is also `intermediatePersistSize` which can be used to force a bunch of incremental indexes to be written when `mergeIndexes` is called to make sure that segment merging happens.
   
   Nothing much seems to be explicitly setting either of these things, and maybe could be removed or reworked in a follow-up.



-- 
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


[GitHub] [druid] clintropolis commented on a diff in pull request #13803: nested columns + arrays = array columns!

Posted by "clintropolis (via GitHub)" <gi...@apache.org>.
clintropolis commented on code in PR #13803:
URL: https://github.com/apache/druid/pull/13803#discussion_r1146906622


##########
processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java:
##########
@@ -327,17 +390,58 @@ public void close()
   }
 
   @Override
-  public DimensionSelector makeDimensionSelector(List<NestedPathPart> path, ReadableOffset readableOffset, ExtractionFn fn)
+  public DimensionSelector makeDimensionSelector(
+      List<NestedPathPart> path,
+      ReadableOffset readableOffset,
+      ExtractionFn fn
+  )
   {
     final String field = getField(path);
     Preconditions.checkNotNull(field, "Null field");
 
     if (fields.indexOf(field) >= 0) {
       DictionaryEncodedColumn<?> col = (DictionaryEncodedColumn<?>) getColumnHolder(field).getColumn();
       return col.makeDimensionSelector(readableOffset, fn);
-    } else {
-      return DimensionSelector.constant(null);
     }
+    if (!path.isEmpty() && path.get(path.size() - 1) instanceof NestedPathArrayElement) {
+      final NestedPathPart lastPath = path.get(path.size() - 1);
+      final String arrayField = getField(path.subList(0, path.size() - 1));
+      if (fields.indexOf(arrayField) >= 0) {

Review Comment:
   refactored to use `fieldIndex` after looking up once



-- 
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