clintropolis commented on code in PR #16068:
URL: https://github.com/apache/druid/pull/16068#discussion_r1526860696
##########
processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java:
##########
@@ -413,9 +414,12 @@ public static Object convertObjectToType(
case DOUBLE:
return coerceToObjectArrayWithElementCoercionFunction(obj,
DimensionHandlerUtils::convertObjectToDouble);
Review Comment:
we probably want to add a case here for other array types to just
recursively call this method on the element type for nested and non-primitive
arrays
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByColumnSelectorStrategyFactory.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
+import
org.apache.druid.query.groupby.epinephelinae.column.DictionaryBuildingGroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.FixedWidthGroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.PrebuiltDictionaryStringGroupByColumnSelectorStrategy;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnType;
+
+/**
+ * Creates {@link org.apache.druid.query.dimension.ColumnSelectorStrategy}s
for grouping dimensions
+ */
+public class GroupByColumnSelectorStrategyFactory implements
ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
+{
+ @Override
+ public GroupByColumnSelectorStrategy makeColumnSelectorStrategy(
+ ColumnCapabilities capabilities,
+ ColumnValueSelector selector
+ )
+ {
+ if (capabilities == null || capabilities.getType() == null) {
+ throw DruidException.defensive("Unable to deduce type for the grouping
dimension");
+ }
+ switch (capabilities.getType()) {
+ case STRING:
+ DimensionSelector dimSelector = (DimensionSelector) selector;
+ if (dimSelector.getValueCardinality() >= 0 &&
dimSelector.nameLookupPossibleInAdvance()) {
+ return PrebuiltDictionaryStringGroupByColumnSelectorStrategy.forType(
+ ColumnType.STRING,
+ selector,
+ capabilities
+ );
+ } else {
+ return
DictionaryBuildingGroupByColumnSelectorStrategy.forType(ColumnType.STRING);
+ }
+ case LONG:
+ return new FixedWidthGroupByColumnSelectorStrategy<Long>(
+ Byte.BYTES + Long.BYTES,
Review Comment:
i wonder if we should check for default value mode? we can leave out the
extra byte per value in that case and just use the regular type strategy for
numbers instead of nullable strategy. This would i guess require splitting
fixed width into nullable fixed width and not null fixed width though, which i
guess wouldn't be the worst thing. alternatively maybe `TypeStrategy` and
`NullableTypeStrategy` could have a common interface since they are pretty
close otherwise
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByColumnSelectorStrategyFactory.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
+import
org.apache.druid.query.groupby.epinephelinae.column.DictionaryBuildingGroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.FixedWidthGroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.PrebuiltDictionaryStringGroupByColumnSelectorStrategy;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnType;
+
+/**
+ * Creates {@link org.apache.druid.query.dimension.ColumnSelectorStrategy}s
for grouping dimensions
+ */
+public class GroupByColumnSelectorStrategyFactory implements
ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
Review Comment:
i know this isn't new, and definitely doesn't need done in this PR, but i
wonder if worth considering switching to using `ColumnProcessors` instead of
`ColumnSelectorStrategyFactory`
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByColumnSelectorStrategyFactory.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
+import
org.apache.druid.query.groupby.epinephelinae.column.DictionaryBuildingGroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.FixedWidthGroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy;
+import
org.apache.druid.query.groupby.epinephelinae.column.PrebuiltDictionaryStringGroupByColumnSelectorStrategy;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnType;
+
+/**
+ * Creates {@link org.apache.druid.query.dimension.ColumnSelectorStrategy}s
for grouping dimensions
+ */
+public class GroupByColumnSelectorStrategyFactory implements
ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
+{
+ @Override
+ public GroupByColumnSelectorStrategy makeColumnSelectorStrategy(
+ ColumnCapabilities capabilities,
+ ColumnValueSelector selector
+ )
+ {
+ if (capabilities == null || capabilities.getType() == null) {
+ throw DruidException.defensive("Unable to deduce type for the grouping
dimension");
+ }
+ switch (capabilities.getType()) {
+ case STRING:
+ DimensionSelector dimSelector = (DimensionSelector) selector;
+ if (dimSelector.getValueCardinality() >= 0 &&
dimSelector.nameLookupPossibleInAdvance()) {
+ return PrebuiltDictionaryStringGroupByColumnSelectorStrategy.forType(
+ ColumnType.STRING,
+ selector,
+ capabilities
+ );
+ } else {
+ return
DictionaryBuildingGroupByColumnSelectorStrategy.forType(ColumnType.STRING);
+ }
+ case LONG:
+ return new FixedWidthGroupByColumnSelectorStrategy<Long>(
+ Byte.BYTES + Long.BYTES,
+ true,
+ ColumnType.LONG
+ );
+ case FLOAT:
+ return new FixedWidthGroupByColumnSelectorStrategy<Float>(
+ Byte.BYTES + Float.BYTES,
+ true,
+ ColumnType.FLOAT
+ );
+ case DOUBLE:
+ return new FixedWidthGroupByColumnSelectorStrategy<Double>(
+ Byte.BYTES + Double.BYTES,
+ true,
+ ColumnType.DOUBLE
+ );
+ case ARRAY:
+ switch (capabilities.getElementType().getType()) {
+ case LONG:
+ case STRING:
+ case DOUBLE:
+ return
DictionaryBuildingGroupByColumnSelectorStrategy.forType(capabilities.toColumnType());
+ case FLOAT:
+ // Array<Float> not supported in expressions, ingestion
+ default:
+ throw new IAE("Cannot create query type helper from invalid type
[%s]", capabilities.asTypeString());
+
+ }
+ case COMPLEX:
+ return
DictionaryBuildingGroupByColumnSelectorStrategy.forType(capabilities.toColumnType());
+ default:
+ throw new IAE("Cannot create query type helper from invalid type
[%s]", capabilities.asTypeString());
Review Comment:
there is no reason other array types cannot group in the same way, so I
think we can just do this
```suggestion
case ARRAY:
case COMPLEX:
default:
return
DictionaryBuildingGroupByColumnSelectorStrategy.forType(capabilities.toColumnType());
```
##########
sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java:
##########
@@ -380,7 +380,7 @@ public VerifyNativeQueries(BaseExecuteQuery execStep)
public void verify()
{
for (QueryResults queryResults : execStep.results()) {
- verifyQuery(queryResults);
+// verifyQuery(queryResults);
Review Comment:
this i assume was for testing some stuff and needs to go away
##########
processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java:
##########
@@ -47,6 +49,9 @@ public class TypeStrategies
public static final StringTypeStrategy STRING = new StringTypeStrategy();
public static final ConcurrentHashMap<String, TypeStrategy<?>>
COMPLEX_STRATEGIES = new ConcurrentHashMap<>();
+ static {
+ ComplexMetrics.registerSerde(ColumnType.NESTED_DATA.getComplexTypeName(),
new NestedDataComplexTypeSerde());
+ }
Review Comment:
this should not be here, it happens in [the
module](https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/guice/NestedDataModule.java),
if a test is needing this then just register manually for the test
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/FixedWidthGroupByColumnSelectorStrategy.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae.column;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.query.DimensionComparisonUtils;
+import org.apache.druid.query.groupby.ResultRow;
+import org.apache.druid.query.groupby.epinephelinae.Grouper;
+import org.apache.druid.query.ordering.StringComparator;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.NullableTypeStrategy;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+import java.nio.ByteBuffer;
+
+/**
+ * Strategy for grouping dimensions which have fixed-width objects. It is only
used for numeric primitive types,
+ * however complex types can reuse this strategy if they can hint the engine
that they are always fixed width
+ * (for e.g. IP types). Such types donot need to be backed by a dictionary,
and hence are faster to group by.
+ *
+ * @param <T> Class of the dimension
+ */
+@NotThreadSafe
+public class FixedWidthGroupByColumnSelectorStrategy<T> implements
GroupByColumnSelectorStrategy
+{
+ /**
+ * Size of the key when materialized as bytes
+ */
+ final int keySizeBytes;
+
+ /**
+ * Indicates whether the type is primitive or not
+ */
+ final boolean isPrimitive;
+
+ /**
+ * Type of the dimension on which the grouping strategy is being used
+ */
+ final ColumnType columnType;
+
+ /**
+ * Nullable type strategy of the dimension
+ */
+ final NullableTypeStrategy<T> nullableTypeStrategy;
+
+ public FixedWidthGroupByColumnSelectorStrategy(
+ int keySizeBytes,
+ boolean isPrimitive,
+ ColumnType columnType
+ )
+ {
+ this.keySizeBytes = keySizeBytes;
+ this.isPrimitive = isPrimitive;
+ this.columnType = columnType;
+ this.nullableTypeStrategy = columnType.getNullableStrategy();
+ }
+
+ @Override
+ public int getGroupingKeySizeBytes()
+ {
+ return keySizeBytes;
+ }
+
+ @Override
+ public void processValueFromGroupingKey(
+ GroupByColumnSelectorPlus selectorPlus,
+ ByteBuffer key,
+ ResultRow resultRow,
+ int keyBufferPosition
+ )
+ {
+ resultRow.set(
+ selectorPlus.getResultRowPosition(),
+ nullableTypeStrategy.read(key, keyBufferPosition)
+ );
+ }
+
+ @Override
+ public int initColumnValues(ColumnValueSelector selector, int columnIndex,
Object[] valuess)
+ {
+ valuess[columnIndex] = getValue(selector);
+ return 0;
+ }
+
+ @Override
+ public void initGroupingKeyColumnValue(
+ int keyBufferPosition,
+ int dimensionIndex,
+ Object rowObj,
+ ByteBuffer keyBuffer,
+ int[] stack
+ )
+ {
+ int written;
+ if (rowObj == null) {
+ written = nullableTypeStrategy.write(keyBuffer, keyBufferPosition, null,
keySizeBytes);
+ stack[dimensionIndex] = 0;
+ } else {
+ written = nullableTypeStrategy.write(keyBuffer, keyBufferPosition, (T)
rowObj, keySizeBytes);
+ stack[dimensionIndex] = 1;
+ }
+ // Since this is a fixed width strategy, the caller should already have
allocated enough space to materialize the
+ // key object, and the type strategy should always be able to write to the
buffer
+ if (written < 0) {
+ throw DruidException.defensive("Unable to serialize the value [%s] to
buffer", rowObj);
+ }
+ }
+
+ /**
+ * This is used for multi-valued dimensions, for values after the first one.
None of the current types supported by
+ * this strategy handle multi-valued dimensions, therefore this short
circuits and returns false
+ */
+ @Override
+ public boolean checkRowIndexAndAddValueToGroupingKey(
+ int keyBufferPosition,
+ Object rowObj,
+ int rowValIdx,
+ ByteBuffer keyBuffer
+ )
+ {
+ return false;
+ }
+
+ @Override
+ public int writeToKeyBuffer(
+ int keyBufferPosition,
+ ColumnValueSelector selector,
+ ByteBuffer keyBuffer
+ )
+ {
+ T value = getValue(selector);
+ int written = nullableTypeStrategy.write(keyBuffer, keyBufferPosition,
value, keySizeBytes);
+ if (written < 0) {
+ throw DruidException.defensive("Unable to serialize the value [%s] to
buffer", value);
+ }
+ // This strategy doesn't use dictionary building and doesn't hold any
internal state, therefore size increase is nil.
+ return 0;
+ }
+
+ @Override
+ public Grouper.BufferComparator bufferComparator(
+ int keyBufferPosition,
+ @Nullable StringComparator stringComparator
+ )
+ {
+ return (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> {
+ T lhs = nullableTypeStrategy.read(lhsBuffer, lhsPosition +
keyBufferPosition);
+ T rhs = nullableTypeStrategy.read(rhsBuffer, rhsPosition +
keyBufferPosition);
+ if (stringComparator != null
+ &&
!DimensionComparisonUtils.isNaturalComparator(columnType.getType(),
stringComparator)) {
+ return stringComparator.compare(String.valueOf(lhs),
String.valueOf(rhs));
+ }
+ // Nulls are allowed while comparing
+ //noinspection ConstantConditions
+ return nullableTypeStrategy.compare(lhs, rhs);
+ };
+ }
+
+
+ @Override
+ public void reset()
+ {
+ // Nothing to reset
+ }
+
+ /**
+ * Returns true if the value at the selector is null. It unifies the null
handling of primitive numeric types and the
+ * other types
+ */
+ private boolean selectorIsNull(ColumnValueSelector columnValueSelector)
+ {
+ if (isPrimitive && columnValueSelector.isNull()) {
+ return true;
+ }
+ return !isPrimitive && (columnValueSelector.getObject() == null);
Review Comment:
nit:
```suggestion
if (isPrimitive) {
return columnValueSelector.isNull();
}
return columnValueSelector.getObject() == null;
```
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/DimensionToIdConverter.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae.column;
+
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nullable;
+
+/**
+ * Interface for converters of dimension to dictionary id.
+ *
+ * This is a slightly convoluted interface because it also encapsulates the
additional logic for handling multi-value
+ * dimensions. It has an additional step that converts the given dimensions to
"dimension holders", which represent the
+ * multi-value holders for a given dimension.
+ * Therefore, the conversion goes from ColumnValueSelector -> DimensionHolder
-> DictionaryID (for each dimension in the holder)
+ *
+ * The dimension holder is only applicable for multi-value strings.
+ * For other dimensions that cannot have multi-values the dimension holder is
identical to the dimension. They can be
+ * defensively cast or homogenised, for example doubles to floats for float
selectors or Long[] to Object[] for array
+ * selectors, so that the upstream callers can assume the class of the
dimensions. The size of these dimensions is always 1,
+ * and only contain a value at index 0.
+ *
+ * Converting a value to its dictionary id might require building dictionaries
on the fly while computing the id. The
+ * return type of the methods, except {@link #multiValueSize}, takes that into
account.
+ *
+ * The implementations can pre-convert the value to the dictionaryId while
extracting the dimensionHolder. Extracting
+ * dictionary id for a specific value from the (potentially multi-value
dimension holder) can be done by calling
+ * {@link #getIndividualValueDictId} and passing the index to the multi-value.
+ *
+ * @see IdToDimensionConverter for converting the dictionary values back to
dimensions
+ *
+ * @param <DimensionHolderType> Type of the dimension holder
+ */
+public interface DimensionToIdConverter<DimensionHolderType>
Review Comment:
i feel like we should special handle multi-value dimensions instead of
making new interfaces to accommodate them, though I'm still considering exactly
how we should do this... maybe just making a dedicate multi-value string
grouping strategy that doesn't fit any of the common patterns would be best.
Everything else should be grouping a single value and that is what we should
always do going forwards. Things with multiple values like arrays should
instead explicitly use UNNEST to aggregate individual elements of multiple
values.
In fact, I almost wonder if we should dump all of the multi-value grouping
stuff and just rewrite in `GroupingEngine.process` to wrap the storage adapter
in an unnest if we detect any multi-value dimensions...
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java:
##########
@@ -1531,32 +1548,136 @@ private RowBasedKeySerdeHelper makeNumericSerdeHelper(
}
}
- private class ArrayNumericRowBasedKeySerdeHelper implements
RowBasedKeySerdeHelper
+ private abstract class
DictionaryBuildingSingleValuedRowBasedKeySerdeHelper implements
RowBasedKeySerdeHelper
+ {
+ private final int keyBufferPosition;
+
+ public DictionaryBuildingSingleValuedRowBasedKeySerdeHelper(final int
keyBufferPosition)
+ {
+ this.keyBufferPosition = keyBufferPosition;
+ }
+
+ @Override
+ public int getKeyBufferValueSize()
+ {
+ return Integer.BYTES;
+ }
+
+ @Override
+ public boolean putToKeyBuffer(RowBasedKey key, int idx)
+ {
+ final Object obj = key.getKey()[idx];
+ int id = getReverseDictionary().getInt(obj);
+ if (id == DimensionDictionary.ABSENT_VALUE_ID) {
+ id = getDictionary().size();
+ getReverseDictionary().put(obj, id);
+ getDictionary().add(obj);
+ }
+ keyBuffer.putInt(id);
+ return true;
+ }
+
+ @Override
+ public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int
dimValIdx, Object[] dimValues)
+ {
+ dimValues[dimValIdx] = getDictionary().get(buffer.getInt(initialOffset
+ keyBufferPosition));
+ }
+
+ /**
+ * Raw type used because arrays and object dictionaries differ
+ */
+ @SuppressWarnings("rawtypes")
+ public abstract List getDictionary();
+
+ /**
+ * Raw types used because arrays and object dictionaries differ
+ */
+ @SuppressWarnings("rawtypes")
+ public abstract Object2IntMap getReverseDictionary();
+ }
+
+ private class ComplexRowBasedKeySerdeHelper extends
DictionaryBuildingSingleValuedRowBasedKeySerdeHelper
{
final int keyBufferPosition;
final BufferComparator bufferComparator;
- final TypeSignature<ValueType> elementType;
+ final ColumnType complexType;
+ final String complexTypeName;
+
+ final List<Object> complexTypeDictionary;
+ final Object2IntMap<Object> complexTypeReverseDictionary;
+
+ public ComplexRowBasedKeySerdeHelper(
+ int keyBufferPosition,
+ ColumnType complexType
+ )
+ {
+ super(keyBufferPosition);
+ this.keyBufferPosition = keyBufferPosition;
+ this.complexType = complexType;
+ this.complexTypeName =
Preconditions.checkNotNull(complexType.getComplexTypeName(), "complex type name
expected");
Review Comment:
if instead of `complexType.getComplexTypeName()` we just use
`complexType.asTypeString()` we can re-use this strategy generically for things
like nested arrays or arrays of complex types too. Suggest renaming to
`GenericDictionaryBuildingRowBasedKeySerdeHelper` or something.
I suppose we could pack dictionaries for all types into this since they are
all basically the same, to have less stuff to track... but it also seems ok to
leave the strings and primitive arrays handled separately in case we want to
further optimize stuff
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/IdToDimensionConverter.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae.column;
+
+/**
+ * Converts back the dictionaryId to the dimension value. The implementations
might or might not handle
+ * {@link GroupByColumnSelectorStrategy#GROUP_BY_MISSING_VALUE}. The callers
should handle those values appropriately on
+ * their own, and filter those out before trying to convert the dictionary id
back to value.
+ *
+ * The encoding - decoding workflow looks like:
+ *
+ * Encoding
+ * 1. {@link DimensionToIdConverter} extracts the multi-value holder for the
given row, which get's stored somewhere
+ * 2. For each entry in the multi-value object, the value gets encoded into a
dictionaryId, using {@link DimensionToIdConverter#getIndividualValueDictId}
+ * 3. The callers can use this integer dictionaryID to materialize the results
somewhere
+ *
+ * Decoding
+ * 1. The materialized dictionary id is deserialized back to an int, and then
decoded into value using {@link #idToKey}
+ *
+ * @see DimensionToIdConverter for converting the dimensions to dictionary ids
+ *
+ * @param <DimensionType> Type of the dimension's values
+ */
+public interface IdToDimensionConverter<DimensionType>
Review Comment:
hmm, this seems basically like `IdLookup` except generic, and the latter
method is only used internally to the class where this thing is also used.
I wonder if we can just drop this interface, make `IdLookup` generic, and
just define `canCompareIds` directly on
`KeyMappingGroupByColumnSelectorStrategy` and let stuff override it
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/KeyMappingGroupByColumnSelectorStrategy.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae.column;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.query.DimensionComparisonUtils;
+import org.apache.druid.query.groupby.ResultRow;
+import org.apache.druid.query.groupby.epinephelinae.Grouper;
+import org.apache.druid.query.ordering.StringComparator;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.NullableTypeStrategy;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+import java.nio.ByteBuffer;
+
+/**
+ * Strategy for grouping dimensions which can have variable-width objects.
Materializing such objects on the buffer
+ * require an additional step of mapping them to an integer index. The integer
index can be materialized on the buffer within
+ * a fixed width, and is often backed by a dictionary representing the actual
dimension object. It is used for arrays,
+ * strings, and complex types.
+ * <p>
+ * The visibility of the class is limited, and the callers must use one of the
two variants of the mapping strategy:
+ * 1. {@link PrebuiltDictionaryStringGroupByColumnSelectorStrategy}
+ * 2. {@link DictionaryBuildingGroupByColumnSelectorStrategy}
+ * <p>
+ * TODO(laksh): Vet this change
+ * {@code null} can be represented by either -1 or the position of null in the
dictionary it was stored when it was
+ * encountered. This is fine, because most of the time, the dictionary id has
no value of its own, and is converted back to
+ * the value it represents, before doing further operations. The only place
where it would matter would be when
+ * {@link IdToDimensionConverter#canCompareIds()} is true, and we compare
directly on the dictionary ids for prebuilt
+ * dictionaries (we can't compare ids for the dictionaries built on the fly in
the grouping strategy). However, in that case,
+ * it is guaranteed that the dictionaryId of null represented by the pre-built
dictionary would be the lowest (most likely 0)
+ * and therefore nulls (-1) would be adjacent to nulls (represented by the
lowest non-negative dictionary id), and would get
+ * grouped in the later merge stages.
+ *
+ * @param <DimensionType>> Class of the dimension
+ * @param <DimensionHolderType> Class of the "dimension holder". For
single-value dimensions, the holder's type and the
+ * holder's object are equivalent to the
dimension. For multi-value dimensions (only strings),
+ * the holder's type and the object are
different, where the type would be {@link
org.apache.druid.segment.data.IndexedInts}
+ * representing all the values in the
multi-valued string, while the dimension type would be
+ * String
+ * @see DimensionToIdConverter encoding logic for converting value to
dictionary
+ * @see IdToDimensionConverter decoding logic for converting back dictionary
to value
+ */
+@NotThreadSafe
+class KeyMappingGroupByColumnSelectorStrategy<DimensionType,
DimensionHolderType>
Review Comment:
yea, after looking more at this i really starting to feel like we should
either split out multi-value string grouping into its own strategy (or do the
other thing and dump it completely in favor of unnest adapter).
Everything except for mvds will spend time doing pointless stuff when there
is ever only 1 dictionary id per row, so it doesn't feel worth the
complexity/cost to have this odd strategy unified with the rest of the sane
ones.
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/FixedWidthGroupByColumnSelectorStrategy.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae.column;
+
+import org.apache.druid.error.DruidException;
+import org.apache.druid.query.DimensionComparisonUtils;
+import org.apache.druid.query.groupby.ResultRow;
+import org.apache.druid.query.groupby.epinephelinae.Grouper;
+import org.apache.druid.query.ordering.StringComparator;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.NullableTypeStrategy;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.NotThreadSafe;
+import java.nio.ByteBuffer;
+
+/**
+ * Strategy for grouping dimensions which have fixed-width objects. It is only
used for numeric primitive types,
+ * however complex types can reuse this strategy if they can hint the engine
that they are always fixed width
+ * (for e.g. IP types). Such types donot need to be backed by a dictionary,
and hence are faster to group by.
+ *
+ * @param <T> Class of the dimension
+ */
+@NotThreadSafe
+public class FixedWidthGroupByColumnSelectorStrategy<T> implements
GroupByColumnSelectorStrategy
+{
+ /**
+ * Size of the key when materialized as bytes
+ */
+ final int keySizeBytes;
+
+ /**
+ * Indicates whether the type is primitive or not
+ */
+ final boolean isPrimitive;
+
+ /**
+ * Type of the dimension on which the grouping strategy is being used
+ */
+ final ColumnType columnType;
+
+ /**
+ * Nullable type strategy of the dimension
+ */
+ final NullableTypeStrategy<T> nullableTypeStrategy;
+
+ public FixedWidthGroupByColumnSelectorStrategy(
+ int keySizeBytes,
+ boolean isPrimitive,
+ ColumnType columnType
+ )
+ {
+ this.keySizeBytes = keySizeBytes;
+ this.isPrimitive = isPrimitive;
+ this.columnType = columnType;
+ this.nullableTypeStrategy = columnType.getNullableStrategy();
+ }
+
+ @Override
+ public int getGroupingKeySizeBytes()
+ {
+ return keySizeBytes;
+ }
+
+ @Override
+ public void processValueFromGroupingKey(
+ GroupByColumnSelectorPlus selectorPlus,
+ ByteBuffer key,
+ ResultRow resultRow,
+ int keyBufferPosition
+ )
+ {
+ resultRow.set(
+ selectorPlus.getResultRowPosition(),
+ nullableTypeStrategy.read(key, keyBufferPosition)
+ );
+ }
+
+ @Override
+ public int initColumnValues(ColumnValueSelector selector, int columnIndex,
Object[] valuess)
+ {
+ valuess[columnIndex] = getValue(selector);
+ return 0;
+ }
+
+ @Override
+ public void initGroupingKeyColumnValue(
+ int keyBufferPosition,
+ int dimensionIndex,
+ Object rowObj,
+ ByteBuffer keyBuffer,
+ int[] stack
+ )
+ {
+ int written;
+ if (rowObj == null) {
+ written = nullableTypeStrategy.write(keyBuffer, keyBufferPosition, null,
keySizeBytes);
+ stack[dimensionIndex] = 0;
+ } else {
+ written = nullableTypeStrategy.write(keyBuffer, keyBufferPosition, (T)
rowObj, keySizeBytes);
+ stack[dimensionIndex] = 1;
+ }
+ // Since this is a fixed width strategy, the caller should already have
allocated enough space to materialize the
+ // key object, and the type strategy should always be able to write to the
buffer
+ if (written < 0) {
+ throw DruidException.defensive("Unable to serialize the value [%s] to
buffer", rowObj);
+ }
+ }
+
+ /**
+ * This is used for multi-valued dimensions, for values after the first one.
None of the current types supported by
+ * this strategy handle multi-valued dimensions, therefore this short
circuits and returns false
+ */
+ @Override
+ public boolean checkRowIndexAndAddValueToGroupingKey(
+ int keyBufferPosition,
+ Object rowObj,
+ int rowValIdx,
+ ByteBuffer keyBuffer
+ )
+ {
+ return false;
+ }
+
+ @Override
+ public int writeToKeyBuffer(
+ int keyBufferPosition,
+ ColumnValueSelector selector,
+ ByteBuffer keyBuffer
+ )
+ {
+ T value = getValue(selector);
+ int written = nullableTypeStrategy.write(keyBuffer, keyBufferPosition,
value, keySizeBytes);
+ if (written < 0) {
+ throw DruidException.defensive("Unable to serialize the value [%s] to
buffer", value);
+ }
+ // This strategy doesn't use dictionary building and doesn't hold any
internal state, therefore size increase is nil.
+ return 0;
+ }
+
+ @Override
+ public Grouper.BufferComparator bufferComparator(
+ int keyBufferPosition,
+ @Nullable StringComparator stringComparator
+ )
+ {
+ return (lhsBuffer, rhsBuffer, lhsPosition, rhsPosition) -> {
+ T lhs = nullableTypeStrategy.read(lhsBuffer, lhsPosition +
keyBufferPosition);
+ T rhs = nullableTypeStrategy.read(rhsBuffer, rhsPosition +
keyBufferPosition);
+ if (stringComparator != null
+ &&
!DimensionComparisonUtils.isNaturalComparator(columnType.getType(),
stringComparator)) {
+ return stringComparator.compare(String.valueOf(lhs),
String.valueOf(rhs));
+ }
+ // Nulls are allowed while comparing
+ //noinspection ConstantConditions
+ return nullableTypeStrategy.compare(lhs, rhs);
+ };
+ }
+
+
+ @Override
+ public void reset()
+ {
+ // Nothing to reset
+ }
+
+ /**
+ * Returns true if the value at the selector is null. It unifies the null
handling of primitive numeric types and the
+ * other types
+ */
+ private boolean selectorIsNull(ColumnValueSelector columnValueSelector)
+ {
+ if (isPrimitive && columnValueSelector.isNull()) {
+ return true;
+ }
+ return !isPrimitive && (columnValueSelector.getObject() == null);
+ }
+
+ /**
+ * Returns the value of the selector. It handles nullity of the value and
casts it to the proper type so that the
+ * upstream callers donot need to worry about handling incorrect types (for
example, if a double column value selector
+ * returns a long)
+ */
+ @Nullable
+ private T getValue(ColumnValueSelector columnValueSelector)
+ {
+ if (selectorIsNull(columnValueSelector)) {
+ return null;
+ }
+ // TODO(laksh): Check if calling .getObject() on primitive selectors be
problematic??
+ // Convert the object to the desired type
+ //noinspection unchecked
+ return (T)
DimensionHandlerUtils.convertObjectToType(columnValueSelector.getObject(),
columnType);
Review Comment:
i wonder if there is a nicer way we could do this by letting the thing that
creates this supply a value supplier method that could be backed by whatever
makes sense for the selector type, e.g. something like
```
public FixedWidthGroupByColumnSelectorStrategy(
int keySizeBytes,
boolean isPrimitive,
ColumnType columnType,
Function<ColumnValueSelector<?>, T> getterFn
)
```
and then
```
case LONG:
return new FixedWidthGroupByColumnSelectorStrategy<>(
Byte.BYTES + Long.BYTES,
true,
ColumnType.LONG,
ColumnValueSelector::getLong
);
```
and so on. It seems to work afaict and doesn't seem like it should be any
worse than going through getObject and dimension handler utils conversion
methods
##########
processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/column/PrebuiltDictionaryStringGroupByColumnSelectorStrategy.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.groupby.epinephelinae.column;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.data.IndexedInts;
+
+import javax.annotation.Nullable;
+
+/**
+ * Implementation of {@link KeyMappingGroupByColumnSelectorStrategy} that
relies on a prebuilt dictionary to map the
+ * dimension to the dictionaryId. It is more like a helper class, that handles
the different ways that dictionaries can be
+ * provided for different types. Currently, it only handles String dimensions.
Array dimensions are also backed by dictionaries,
+ * but not exposed via the ColumnValueSelector interface, hence this strategy
cannot handle array dimensions.
+ */
+public class PrebuiltDictionaryStringGroupByColumnSelectorStrategy
+{
+
+ /**
+ * Create the strategy for the provided column type
+ */
+ public static GroupByColumnSelectorStrategy forType(
+ final ColumnType columnType,
+ final ColumnValueSelector columnValueSelector,
+ final ColumnCapabilities columnCapabilities
+ )
+ {
+ if (columnType.equals(ColumnType.STRING)) {
+ return forString(columnValueSelector, columnCapabilities);
+ } else {
+ // This will change with array columns
+ throw DruidException.defensive("Only string columns expose prebuilt
dictionaries");
+ }
+ }
+
+ private static GroupByColumnSelectorStrategy forString(
+ final ColumnValueSelector columnValueSelector,
+ final ColumnCapabilities columnCapabilities
+ )
+ {
+ return new KeyMappingGroupByColumnSelectorStrategy<>(
+ new StringDimensionToIdConverter(),
+ ColumnType.STRING,
+ ColumnType.STRING.getNullableStrategy(),
+ NullHandling.defaultStringValue(),
+ new StringIdToDimensionConverter((DimensionSelector)
columnValueSelector, columnCapabilities)
+ );
+ }
+
+ /**
+ * Dimension to id converter for string dimensions and {@link
DimensionSelector}, where the dictionaries are prebuilt.
+ * The callers must ensure that's the case by checking that {@link
DimensionSelector#getValueCardinality()} is known
+ * and {@link DimensionSelector#nameLookupPossibleInAdvance()} is true.
+ */
+ private static class StringDimensionToIdConverter implements
DimensionToIdConverter<IndexedInts>
+ {
+ @Override
+ public MemoryEstimate<IndexedInts> getMultiValueHolder(
+ final ColumnValueSelector selector,
+ final IndexedInts reusableValue
+ )
+ {
+ return new MemoryEstimate<>(((DimensionSelector) selector).getRow(), 0);
+ }
+
+ @Override
+ public int multiValueSize(IndexedInts multiValueHolder)
+ {
+ return multiValueHolder.size();
+ }
+
+ @Override
+ public MemoryEstimate<Integer> getIndividualValueDictId(IndexedInts
multiValueHolder, int index)
+ {
+ // dictId is already encoded in the indexedInt supplied by the column
value selector
+ return new MemoryEstimate<>(multiValueHolder.get(index), 0);
+ }
+ }
+
+ /**
+ * ID to dimension converter for {@link DimensionSelector} with prebuilt
dictionary
+ */
+ private static class StringIdToDimensionConverter implements
IdToDimensionConverter<String>
+ {
+
+ final DimensionSelector dimensionSelector;
+
+ @Nullable
+ final ColumnCapabilities columnCapabilities;
+
+ public StringIdToDimensionConverter(
+ final DimensionSelector dimensionSelector,
+ @Nullable final ColumnCapabilities columnCapabilities
+ )
+ {
+ this.dimensionSelector = dimensionSelector;
+ this.columnCapabilities = columnCapabilities;
+ }
+
+ @Override
+ public String idToKey(int id)
+ {
+ // Converting back to the value is as simple as looking up the value in
the prebuilt dictionary
+ return dimensionSelector.lookupName(id);
+ }
+
+ @Override
+ public boolean canCompareIds()
+ {
+ return columnCapabilities != null
+ && columnCapabilities.hasBitmapIndexes()
+ && (columnCapabilities.areDictionaryValuesSorted()
+
.and(columnCapabilities.areDictionaryValuesUnique())).isTrue();
Review Comment:
this isn't correct, we should be checking `isDictionaryEncoded` not
`hasBitmapIndexes`
```suggestion
return columnCapabilities != null
&& columnCapabilities.isDictionaryEncoded()
.and(columnCapabilities.areDictionaryValuesSorted())
.and(columnCapabilities.areDictionaryValuesUnique())
.isTrue();
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]