diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java index eb874dbea27..fe60d3eda44 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java @@ -278,6 +278,26 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory && stringEncoding == that.stringEncoding; } + @Nullable + @Override + public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated) + { + if (this == preAggregated) { + return getCombiningFactory(); + } + if (getClass() != preAggregated.getClass()) { + return null; + } + HllSketchAggregatorFactory that = (HllSketchAggregatorFactory) preAggregated; + if (lgK == that.lgK && tgtHllType == that.tgtHllType && stringEncoding == that.stringEncoding && Objects.equals( + fieldName, + that.fieldName + )) { + return getCombiningFactory(); + } + return null; + } + @Override public int hashCode() { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java index dcc12f5c384..12bbccce9d0 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java @@ -231,6 +231,42 @@ public class HllSketchAggregatorFactoryTest Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey()); } + @Test + public void testCanSubstitute() + { + HllSketchBuildAggregatorFactory factory = new HllSketchBuildAggregatorFactory( + NAME, + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + true, + true + ); + HllSketchBuildAggregatorFactory other = new HllSketchBuildAggregatorFactory( + "other name", + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + false, + false + ); + + HllSketchBuildAggregatorFactory incompatible = new HllSketchBuildAggregatorFactory( + NAME, + "different field", + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + false, + false + ); + Assert.assertNotNull(other.substituteCombiningFactory(factory)); + Assert.assertNotNull(factory.substituteCombiningFactory(other)); + Assert.assertNull(factory.substituteCombiningFactory(incompatible)); + } + @Test public void testToString() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java index d0410897a07..26a360e716d 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java @@ -296,7 +296,7 @@ public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest frameReader = FrameReader.create(adapter.getRowSignature()); frameList = FrameSequenceBuilder.fromCursorFactory(adapter) .frameType(FrameType.ROW_BASED) - .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) + .maxRowsPerFrame(IntMath.divide(index.numRows(), MAX_FRAMES, RoundingMode.CEILING)) .frames() .toList(); } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java index e7f1402606f..452a658e9d4 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java @@ -165,7 +165,7 @@ public class DatasourceRecordReaderSegmentReaderTest Assert.assertEquals(18, count); // Check the index - Assert.assertEquals(9, index.size()); + Assert.assertEquals(9, index.numRows()); final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test")); final List dimensions = index.getDimensionNames(false); Assert.assertEquals(2, dimensions.size()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 90383b2b8e3..8908d0d50c7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1913,6 +1913,7 @@ public class CompactionTaskTest null, null, null, + null, null ); } diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/AggregateProjectionSpec.java b/processing/src/main/java/org/apache/druid/data/input/impl/AggregateProjectionSpec.java new file mode 100644 index 00000000000..5d96ba497fa --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/AggregateProjectionSpec.java @@ -0,0 +1,212 @@ +/* + * 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.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Lists; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * API type to specify an aggregating projection on {@link org.apache.druid.segment.incremental.IncrementalIndexSchema} + * + * Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of + * projections and need to extract out a base interface from this class. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonTypeName(AggregateProjectionSpec.TYPE_NAME) +public class AggregateProjectionSpec +{ + public static final String TYPE_NAME = "aggregate"; + + private final String name; + private final List groupingColumns; + private final VirtualColumns virtualColumns; + private final AggregatorFactory[] aggregators; + private final List ordering; + @Nullable + private final String timeColumnName; + + @JsonCreator + public AggregateProjectionSpec( + @JsonProperty("name") String name, + @JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns, + @JsonProperty("groupingColumns") @Nullable List groupingColumns, + @JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators + ) + { + this.name = name; + if (CollectionUtils.isNullOrEmpty(groupingColumns)) { + throw InvalidInput.exception("groupingColumns must not be null or empty"); + } + this.groupingColumns = groupingColumns; + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; + // in the future this should be expanded to support user specified ordering, but for now we compute it based on + // the grouping columns, which is consistent with how rollup ordering works for incremental index base table + final ProjectionOrdering ordering = computeOrdering(this.virtualColumns, this.groupingColumns); + this.ordering = ordering.ordering; + this.timeColumnName = ordering.timeColumnName; + this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public List getGroupingColumns() + { + return groupingColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public AggregatorFactory[] getAggregators() + { + return aggregators; + } + + @JsonProperty + public List getOrdering() + { + return ordering; + } + + @JsonIgnore + public AggregateProjectionMetadata.Schema toMetadataSchema() + { + return new AggregateProjectionMetadata.Schema( + name, + timeColumnName, + virtualColumns, + groupingColumns.stream().map(DimensionSchema::getName).collect(Collectors.toList()), + aggregators, + ordering + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AggregateProjectionSpec that = (AggregateProjectionSpec) o; + return Objects.equals(name, that.name) + && Objects.equals(groupingColumns, that.groupingColumns) + && Objects.equals(virtualColumns, that.virtualColumns) + && Objects.deepEquals(aggregators, that.aggregators) + && Objects.equals(ordering, that.ordering); + } + + @Override + public int hashCode() + { + return Objects.hash(name, groupingColumns, virtualColumns, Arrays.hashCode(aggregators), ordering); + } + + @Override + public String toString() + { + return "AggregateProjectionSpec{" + + "name='" + name + '\'' + + ", groupingColumns=" + groupingColumns + + ", virtualColumns=" + virtualColumns + + ", aggregators=" + Arrays.toString(aggregators) + + ", ordering=" + ordering + + '}'; + } + + + private static ProjectionOrdering computeOrdering(VirtualColumns virtualColumns, List groupingColumns) + { + final List ordering = Lists.newArrayListWithCapacity(groupingColumns.size()); + + String timeColumnName = null; + Granularity granularity = null; + // try to find the __time column equivalent, which might be a time_floor expression to model granularity + // bucketing. The time column is decided as the finest granularity on __time detected. If the projection does + // not have a time-like column, the granularity will be handled as ALL for the projection and all projection + // rows will use a synthetic timestamp of the minimum timestamp of the incremental index + for (final DimensionSchema dimension : groupingColumns) { + ordering.add(OrderBy.ascending(dimension.getName())); + if (ColumnHolder.TIME_COLUMN_NAME.equals(dimension.getName())) { + timeColumnName = dimension.getName(); + granularity = Granularities.NONE; + } else { + final VirtualColumn vc = virtualColumns.getVirtualColumn(dimension.getName()); + final Granularity maybeGranularity = Granularities.fromVirtualColumn(vc); + if (granularity == null && maybeGranularity != null) { + granularity = maybeGranularity; + timeColumnName = dimension.getName(); + } else if (granularity != null && maybeGranularity != null && maybeGranularity.isFinerThan(granularity)) { + granularity = maybeGranularity; + timeColumnName = dimension.getName(); + } + } + } + return new ProjectionOrdering(ordering, timeColumnName); + } + + private static final class ProjectionOrdering + { + private final List ordering; + @Nullable + private final String timeColumnName; + + private ProjectionOrdering(List ordering, @Nullable String timeColumnName) + { + this.ordering = ordering; + this.timeColumnName = timeColumnName; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java index 05150ea830f..08de0a364ec 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java @@ -23,9 +23,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Query; import org.apache.druid.query.expression.TimestampFloorExprMacro; +import org.apache.druid.segment.AggregateProjectionMetadata; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; @@ -160,4 +162,32 @@ public class Granularities ExprMacroTable.granularity() ); } + + /** + * Converts a virtual column with a single input time column into a {@link Granularity} if it is a + * {@link TimestampFloorExprMacro.TimestampFloorExpr}. + *

+ * IMPORTANT - this method DOES NOT VERIFY that the virtual column has a single input that is a time column + * ({@link ColumnHolder#TIME_COLUMN_NAME} or equivalent projection time column as defined by + * {@link AggregateProjectionMetadata.Schema#getTimeColumnName()}). Callers must verify this externally before + * calling this method by examining {@link VirtualColumn#requiredColumns()}. + *

+ * This method also does not handle other time expressions, or if the virtual column is just an identifier for a + * time column + */ + @Nullable + public static Granularity fromVirtualColumn(VirtualColumn virtualColumn) + { + if (virtualColumn instanceof ExpressionVirtualColumn) { + final ExpressionVirtualColumn expressionVirtualColumn = (ExpressionVirtualColumn) virtualColumn; + final Expr expr = expressionVirtualColumn.getParsedExpression().get(); + if (expr instanceof TimestampFloorExprMacro.TimestampFloorExpr) { + final TimestampFloorExprMacro.TimestampFloorExpr gran = (TimestampFloorExprMacro.TimestampFloorExpr) expr; + if (gran.getArg().getBindingIfIdentifier() != null) { + return gran.getGranularity(); + } + } + } + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java index db734651797..1273aebc019 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java @@ -190,6 +190,12 @@ public class DefaultQueryMetrics> implements QueryMet setDimension("segment", segmentIdentifier); } + @Override + public void projection(String projection) + { + setDimension("projection", projection); + } + @Override public void identity(String identity) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index ced9f0d4e2d..f3fc1a80f83 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -88,6 +88,12 @@ public class QueryContexts public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit"; public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold"; public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled"; + + // projection context keys + public static final String NO_PROJECTIONS = "noProjections"; + public static final String FORCE_PROJECTION = "forceProjections"; + public static final String USE_PROJECTION = "useProjection"; + // Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the // query's runtime public static final String QUERY_RESOURCE_ID = "queryResourceId"; diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java index 6dfad11fae4..812b1dd9134 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java @@ -243,6 +243,11 @@ public interface QueryMetrics> void segment(String segmentIdentifier); + /** + * If a projection was used during segment processing, set its name as the projection dimension + */ + void projection(String projection); + /** * @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were * used to construct {@link org.apache.druid.segment.BitmapOffset} or diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 8567c4b6d6c..3d34500b710 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ColumnTypeFactory; import org.apache.druid.segment.column.ValueType; @@ -365,6 +367,38 @@ public abstract class AggregatorFactory implements Cacheable throw new UOE("Cannot change output name for AggregatorFactory[%s].", this.getClass().getName()); } + /** + * Check to see if we can make a 'combining' factory of this aggregator that is suitable to process input from a + * selector of values produced by the other {@link AggregatorFactory} representing pre-aggregated data. Typically, + * this means that this and the other aggregator have the same inputs ({@link #requiredFields()}, and the same + * options for how the data was constructed into the intermediary type. If suitable, this method returns a + * 'combining' aggregator factory of this aggregator to use to process the pre-aggregated data which was produced by + * the other aggregator. + *

+ * This method is used indirectly in service of checking if a + * {@link org.apache.druid.segment.projections.QueryableProjection} can be used instead of the base table during + * {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)}, which checks if this + * aggregator can be substituted for its combining aggregator if and only if there exists a column that a cursor can + * read which was created by an aggregator that satisfies this method. In other words, this aggregator is the 'query' + * aggregator defined on the {@link CursorBuildSpec}, the argument to this method is the aggregator which created + * some column whose selectors are available to the cursor. If all aggregators on the {@link CursorBuildSpec} can be + * paired with aggregators from the underlying table in the cursor factory, then + * {@link CursorHolder#isPreAggregated()} will be set to true indicating that query engines should use this + * substituted aggregator instead of the original aggregators. + * + * @param preAggregated {@link AggregatorFactory} which produced the partially aggregated values which are + * available in a selector + * @return a "combining" {@link AggregatorFactory} to use with the pre-aggregated selector data + */ + @Nullable + public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated) + { + if (equals(preAggregated.withName(getName()))) { + return getCombiningFactory(); + } + return null; + } + /** * Merges the list of AggregatorFactory[] (presumable from metadata of some segments being merged) and * returns merged AggregatorFactory[] (for the metadata for merged segment). diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index 4d12327c896..c4c9a7875ef 100755 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -454,13 +454,4 @@ public class AggregatorUtil } return false; } - - public static List getCombiningAggregators(List aggs) - { - List combining = new ArrayList<>(aggs.size()); - for (AggregatorFactory agg : aggs) { - combining.add(agg.getCombiningFactory()); - } - return combining; - } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 67583fc1fc0..139bda6d50f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -20,6 +20,7 @@ package org.apache.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -60,7 +61,6 @@ import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.ResultMergeQueryRunner; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.dimension.DefaultDimensionSpec; @@ -509,7 +509,7 @@ public class GroupingEngine final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec)); if (cursorHolder.isPreAggregated()) { - query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated())); } final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory); diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java index ef484df5f18..518611e408d 100644 --- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java @@ -151,6 +151,12 @@ public class DefaultSearchQueryMetrics implements SearchQueryMetrics delegateQueryMetrics.segment(segmentIdentifier); } + @Override + public void projection(String projection) + { + delegateQueryMetrics.projection(projection); + } + @Override public void filterBundle(FilterBundle.BundleInfo bundleInfo) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index dbec221248e..4d4fe7f571a 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -20,6 +20,7 @@ package org.apache.druid.query.timeseries; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.inject.Inject; import org.apache.druid.collections.NonBlockingPool; @@ -38,7 +39,6 @@ import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; @@ -104,7 +104,7 @@ public class TimeseriesQueryEngine final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); if (cursorHolder.isPreAggregated()) { - query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated())); } try { final Sequence> result; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index d10d26242e3..202a852c2be 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn; +import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; @@ -30,7 +31,6 @@ import org.apache.druid.query.CursorGranularizer; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; @@ -89,7 +89,7 @@ public class TopNQueryEngine final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); if (cursorHolder.isPreAggregated()) { - query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated())); } final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { diff --git a/processing/src/main/java/org/apache/druid/segment/AggregateProjectionMetadata.java b/processing/src/main/java/org/apache/druid/segment/AggregateProjectionMetadata.java new file mode 100644 index 00000000000..7712cecb303 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/AggregateProjectionMetadata.java @@ -0,0 +1,463 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; +import com.google.common.collect.Lists; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.projections.Projections; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + * Aggregate projection schema and row count information to store in {@link Metadata} which itself is stored inside a + * segment, defining which projections exist for the segment. + *

+ * Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of + * projections and need to extract out a base interface from this class. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonTypeName(AggregateProjectionSpec.TYPE_NAME) +public class AggregateProjectionMetadata +{ + private static final Interner SCHEMA_INTERNER = Interners.newWeakInterner(); + + public static final Comparator COMPARATOR = (o1, o2) -> { + int rowCompare = Integer.compare(o1.numRows, o2.numRows); + if (rowCompare != 0) { + return rowCompare; + } + return Schema.COMPARATOR.compare(o1.getSchema(), o2.getSchema()); + }; + + private final Schema schema; + private final int numRows; + + @JsonCreator + public AggregateProjectionMetadata( + @JsonProperty("schema") Schema schema, + @JsonProperty("numRows") int numRows + ) + { + this.schema = SCHEMA_INTERNER.intern(schema); + this.numRows = numRows; + } + + @JsonProperty + public Schema getSchema() + { + return schema; + } + + @JsonProperty + public int getNumRows() + { + return numRows; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AggregateProjectionMetadata that = (AggregateProjectionMetadata) o; + return numRows == that.numRows && Objects.equals(schema, that.schema); + } + + @Override + public int hashCode() + { + return Objects.hash(schema, numRows); + } + + public static class Schema + { + /** + * It is not likely the best way to find the best matching projections, but it is the one we have for now. This + * comparator is used to sort all the projections in a segment "best" first, where best is defined as fewest grouping + * columns, most virtual columns and aggregators, as an approximation of likely to have the fewest number of rows to + * scan. + */ + public static final Comparator COMPARATOR = (o1, o2) -> { + // coarsest granularity first + if (o1.getGranularity().isFinerThan(o2.getGranularity())) { + return 1; + } + if (o2.getGranularity().isFinerThan(o1.getGranularity())) { + return -1; + } + // fewer dimensions first + final int dimsCompare = Integer.compare( + o1.groupingColumns.size(), + o2.groupingColumns.size() + ); + if (dimsCompare != 0) { + return dimsCompare; + } + // more metrics first + int metCompare = Integer.compare(o2.aggregators.length, o1.aggregators.length); + if (metCompare != 0) { + return metCompare; + } + // more virtual columns first + final int virtCompare = Integer.compare( + o2.virtualColumns.getVirtualColumns().length, + o1.virtualColumns.getVirtualColumns().length + ); + if (virtCompare != 0) { + return virtCompare; + } + return o1.name.compareTo(o2.name); + }; + + private final String name; + @Nullable + private final String timeColumnName; + private final VirtualColumns virtualColumns; + private final List groupingColumns; + private final AggregatorFactory[] aggregators; + private final List ordering; + private final List orderingWithTimeSubstitution; + + // computed fields + private final int timeColumnPosition; + private final Granularity granularity; + + @JsonCreator + public Schema( + @JsonProperty("name") String name, + @JsonProperty("timeColumnName") @Nullable String timeColumnName, + @JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns, + @JsonProperty("groupingColumns") List groupingColumns, + @JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators, + @JsonProperty("ordering") List ordering + ) + { + this.name = name; + if (CollectionUtils.isNullOrEmpty(groupingColumns)) { + throw InvalidInput.exception("groupingColumns must not be null or empty"); + } + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; + this.groupingColumns = groupingColumns; + this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators; + this.ordering = ordering; + + int foundTimePosition = -1; + this.orderingWithTimeSubstitution = Lists.newArrayListWithCapacity(ordering.size()); + Granularity granularity = null; + for (int i = 0; i < ordering.size(); i++) { + OrderBy orderBy = ordering.get(i); + if (orderBy.getColumnName().equals(timeColumnName)) { + orderingWithTimeSubstitution.add(new OrderBy(ColumnHolder.TIME_COLUMN_NAME, orderBy.getOrder())); + foundTimePosition = i; + timeColumnName = groupingColumns.get(foundTimePosition); + final VirtualColumn vc = this.virtualColumns.getVirtualColumn(groupingColumns.get(foundTimePosition)); + if (vc != null) { + granularity = Granularities.fromVirtualColumn(vc); + } else { + granularity = Granularities.NONE; + } + } else { + orderingWithTimeSubstitution.add(orderBy); + } + } + this.timeColumnName = timeColumnName; + this.timeColumnPosition = foundTimePosition; + this.granularity = granularity == null ? Granularities.ALL : granularity; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + @Nullable + public String getTimeColumnName() + { + return timeColumnName; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public List getGroupingColumns() + { + return groupingColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public AggregatorFactory[] getAggregators() + { + return aggregators; + } + + @JsonProperty + public List getOrdering() + { + return ordering; + } + + @JsonIgnore + public List getOrderingWithTimeColumnSubstitution() + { + return orderingWithTimeSubstitution; + } + + @JsonIgnore + public int getTimeColumnPosition() + { + return timeColumnPosition; + } + + @JsonIgnore + public Granularity getGranularity() + { + return granularity; + } + + /** + * Check if this projection "matches" a {@link CursorBuildSpec} for a query to see if we can use a projection + * instead. For a projection to match, all grouping columns of the build spec must match, virtual columns of the + * build spec must either be available as a physical column on the projection, or the inputs to the virtual column + * must be available on the projection, and all aggregators must be compatible with pre-aggregated columns of the + * projection per {@link AggregatorFactory#substituteCombiningFactory(AggregatorFactory)}. If the projection + * matches, this method returns a {@link Projections.ProjectionMatch} which contains an updated + * {@link CursorBuildSpec} which has the remaining virtual columns from the original build spec which must still be + * computed and the 'combining' aggregator factories to process the pre-aggregated data from the projection, as well + * as a mapping of query column names to projection column names. + * + * @param queryCursorBuildSpec the {@link CursorBuildSpec} that contains the required inputs to build a + * {@link CursorHolder} for a query + * @param physicalColumnChecker Helper utility which can determine if a physical column required by + * queryCursorBuildSpec is available on the projection OR does not exist on the base + * table either + * @return a {@link Projections.ProjectionMatch} if the {@link CursorBuildSpec} matches the projection, which + * contains information such as which + */ + @Nullable + public Projections.ProjectionMatch matches( + CursorBuildSpec queryCursorBuildSpec, + Projections.PhysicalColumnChecker physicalColumnChecker + ) + { + if (!queryCursorBuildSpec.isCompatibleOrdering(orderingWithTimeSubstitution)) { + return null; + } + final List queryGrouping = queryCursorBuildSpec.getGroupingColumns(); + Projections.ProjectionMatchBuilder matchBuilder = new Projections.ProjectionMatchBuilder(); + + if (timeColumnName != null) { + matchBuilder.remapColumn(timeColumnName, ColumnHolder.TIME_COLUMN_NAME); + } + if (queryGrouping != null) { + for (String queryColumn : queryGrouping) { + matchBuilder = matchRequiredColumn( + matchBuilder, + queryColumn, + queryCursorBuildSpec.getVirtualColumns(), + physicalColumnChecker + ); + if (matchBuilder == null) { + return null; + } + } + } + if (queryCursorBuildSpec.getFilter() != null) { + for (String queryColumn : queryCursorBuildSpec.getFilter().getRequiredColumns()) { + matchBuilder = matchRequiredColumn( + matchBuilder, + queryColumn, + queryCursorBuildSpec.getVirtualColumns(), + physicalColumnChecker + ); + if (matchBuilder == null) { + return null; + } + } + } + if (!CollectionUtils.isNullOrEmpty(queryCursorBuildSpec.getAggregators())) { + boolean allMatch = true; + for (AggregatorFactory queryAgg : queryCursorBuildSpec.getAggregators()) { + boolean foundMatch = false; + for (AggregatorFactory projectionAgg : aggregators) { + final AggregatorFactory combining = queryAgg.substituteCombiningFactory(projectionAgg); + if (combining != null) { + matchBuilder.remapColumn(queryAgg.getName(), projectionAgg.getName()).addPreAggregatedAggregator(combining); + foundMatch = true; + } + } + allMatch = allMatch && foundMatch; + } + if (!allMatch) { + return null; + } + } + return matchBuilder.build(queryCursorBuildSpec); + } + + /** + * Ensure that the projection has the specified column required by a {@link CursorBuildSpec} in one form or another. + * If the column is a {@link VirtualColumn} on the build spec, ensure that the projection has an equivalent virtual + * column, or has the required inputs to compute the virtual column. If an equivalent virtual column exists, its + * name will be added to {@link Projections.ProjectionMatchBuilder#remapColumn(String, String)} so the query + * virtual column name can be mapped to the projection physical column name. If no equivalent virtual column exists, + * but the inputs are available on the projection to compute it, it will be added to + * {@link Projections.ProjectionMatchBuilder#addReferenceedVirtualColumn(VirtualColumn)}. + *

+ * Finally, if the column is not a virtual column in the query, it is checked with + * {@link Projections.PhysicalColumnChecker} which true if the column is present on the projection OR if the column + * is NOT present on the base table (meaning missing columns that do not exist anywhere do not disqualify a + * projection from being used). + * + * @param matchBuilder match state to add mappings of query virtual columns to projection physical columns + * and query virtual columns which still must be computed from projection physical + * columns + * @param column Column name to check + * @param queryVirtualColumns {@link VirtualColumns} from the {@link CursorBuildSpec} required by the query + * @param physicalColumnChecker Helper to check if the physical column exists on a projection, or does not exist on + * the base table + * @return {@link Projections.ProjectionMatchBuilder} with updated state per the rules described above, or null + * if the column cannot be matched + */ + @Nullable + private Projections.ProjectionMatchBuilder matchRequiredColumn( + Projections.ProjectionMatchBuilder matchBuilder, + String column, + VirtualColumns queryVirtualColumns, + Projections.PhysicalColumnChecker physicalColumnChecker + ) + { + final VirtualColumn buildSpecVirtualColumn = queryVirtualColumns.getVirtualColumn(column); + if (buildSpecVirtualColumn != null) { + // check to see if we have an equivalent virtual column defined in the projection, if so we can + final VirtualColumn projectionEquivalent = virtualColumns.findEquivalent(buildSpecVirtualColumn); + if (projectionEquivalent != null) { + if (!buildSpecVirtualColumn.getOutputName().equals(projectionEquivalent.getOutputName())) { + matchBuilder.remapColumn( + buildSpecVirtualColumn.getOutputName(), + projectionEquivalent.getOutputName() + ); + } + return matchBuilder; + } + + matchBuilder.addReferenceedVirtualColumn(buildSpecVirtualColumn); + final List requiredInputs = buildSpecVirtualColumn.requiredColumns(); + if (requiredInputs.size() == 1 && ColumnHolder.TIME_COLUMN_NAME.equals(requiredInputs.get(0))) { + // special handle time granularity. in the future this should be reworked to push this concept into the + // virtual column and underlying expression itself, but this will do for now + final Granularity virtualGranularity = Granularities.fromVirtualColumn(buildSpecVirtualColumn); + if (virtualGranularity != null) { + if (virtualGranularity.isFinerThan(granularity)) { + return null; + } + return matchBuilder.remapColumn(column, timeColumnName); + } else { + // anything else with __time requires none granularity + if (Granularities.NONE.equals(granularity)) { + return matchBuilder; + } + return null; + } + } else { + for (String required : requiredInputs) { + matchBuilder = matchRequiredColumn( + matchBuilder, + required, + queryVirtualColumns, + physicalColumnChecker + ); + if (matchBuilder == null) { + return null; + } + } + return matchBuilder; + } + } else { + if (physicalColumnChecker.check(name, column)) { + return matchBuilder; + } + return null; + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Schema schema = (Schema) o; + return Objects.equals(name, schema.name) + && Objects.equals(timeColumnName, schema.timeColumnName) + && Objects.equals(virtualColumns, schema.virtualColumns) + && Objects.equals(groupingColumns, schema.groupingColumns) + && Objects.deepEquals(aggregators, schema.aggregators) + && Objects.equals(ordering, schema.ordering); + } + + @Override + public int hashCode() + { + return Objects.hash( + name, + timeColumnName, + virtualColumns, + groupingColumns, + Arrays.hashCode(aggregators), + ordering + ); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index a2230cb2972..d0209b0bfea 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -27,6 +27,7 @@ import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.math.expr.Evals; @@ -360,8 +361,42 @@ public class AutoTypeColumnIndexer implements DimensionIndexer() + return new ColumnValueSelector() { + @Override + public double getDouble() + { + Object o = StructuredData.unwrap(getObject()); + return Numbers.tryParseDouble(o, 0.0); + } + + @Override + public float getFloat() + { + Object o = StructuredData.unwrap(getObject()); + return Numbers.tryParseFloat(o, 0.0f); + } + + @Override + public long getLong() + { + Object o = StructuredData.unwrap(getObject()); + return Numbers.tryParseLong(o, 0L); + } + + @Override + public boolean isNull() + { + final Object o = StructuredData.unwrap(getObject()); + if (o instanceof Number) { + return false; + } + if (o instanceof String) { + return GuavaUtils.tryParseLong((String) o) == null && Doubles.tryParse((String) o) == null; + } + return true; + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java index 7ce9709f30f..c2bba50b3d8 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java @@ -75,6 +75,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 SimpleDictionaryMergingIterator.makePeekingComparator(); private final String name; + private final String outputName; private final IndexSpec indexSpec; private final SegmentWriteOutMedium segmentWriteOutMedium; private final Closer closer; @@ -84,9 +85,23 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 @Nullable private final ColumnType castToType; private boolean isVariantType = false; + private byte variantTypeByte = 0x00; + /** + * @param name column name + * @param outputName output smoosh file name. if this is a base table column, it will be the equivalent to + * name, however if this merger is for a projection, this will be prefixed with the + * projection name so that multiple projections can store the same column name at + * different smoosh file "paths" + * @param castToType optional mechanism to enforce that all values are a specific type + * @param indexSpec segment level storage options such as compression format and bitmap type + * @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment + * @param closer resource closer if this merger needs to attach any closables that should be cleaned up + * when the segment is finished writing + */ public AutoTypeColumnMerger( String name, + String outputName, @Nullable ColumnType castToType, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, @@ -95,6 +110,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 { this.name = name; + this.outputName = outputName; this.castToType = castToType; this.indexSpec = indexSpec; this.segmentWriteOutMedium = segmentWriteOutMedium; @@ -165,7 +181,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) { logicalType = ColumnType.STRING; serializer = new ScalarStringColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -179,7 +195,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 switch (logicalType.getType()) { case LONG: serializer = new ScalarLongColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -187,7 +203,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 break; case DOUBLE: serializer = new ScalarDoubleColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -195,7 +211,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 break; case STRING: serializer = new ScalarStringColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -203,7 +219,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 break; case ARRAY: serializer = new VariantColumnSerializer( - name, + outputName, logicalType, null, indexSpec, @@ -222,6 +238,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 // mixed type column, but only root path, we can use VariantArrayColumnSerializer // pick the least restrictive type for the logical type isVariantType = true; + variantTypeByte = rootTypes.getByteValue(); for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) { logicalType = ColumnType.leastRestrictiveType(logicalType, type); } @@ -230,9 +247,9 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType); } serializer = new VariantColumnSerializer( - name, + outputName, null, - rootTypes.getByteValue(), + variantTypeByte, indexSpec, segmentWriteOutMedium, closer @@ -241,7 +258,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 // all the bells and whistles logicalType = ColumnType.NESTED_DATA; serializer = new NestedDataColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -262,7 +279,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 sortedLookup.getSortedDoubles(), () -> new ArrayDictionaryMergingIterator( sortedArrayLookups, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); stringCardinality = sortedLookup.getStringCardinality(); @@ -284,7 +301,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 ); final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator( sortedArrayLookups, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ); serializer.serializeDictionaries( () -> stringIterator, @@ -367,6 +384,76 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 return descriptorBuilder.build(); } + protected DictionaryIdLookup getIdLookup() + { + return serializer.getDictionaryIdLookup(); + } + + @Override + public void attachParent(DimensionMergerV9 parent, List projectionAdapters) throws IOException + { + DruidException.conditionalDefensive( + parent instanceof AutoTypeColumnMerger, + "Projection parent dimension must be same type, got [%s]", + parent.getClass() + ); + AutoTypeColumnMerger autoParent = (AutoTypeColumnMerger) parent; + logicalType = autoParent.logicalType; + isVariantType = autoParent.isVariantType; + if (autoParent.serializer instanceof ScalarStringColumnSerializer) { + serializer = new ScalarStringColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else if (autoParent.serializer instanceof ScalarLongColumnSerializer) { + serializer = new ScalarLongColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else if (autoParent.serializer instanceof ScalarDoubleColumnSerializer) { + serializer = new ScalarDoubleColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else if (autoParent.serializer instanceof VariantColumnSerializer) { + if (autoParent.isVariantType) { + serializer = new VariantColumnSerializer( + outputName, + null, + variantTypeByte, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else { + serializer = new VariantColumnSerializer( + outputName, + logicalType, + null, + indexSpec, + segmentWriteOutMedium, + closer + ); + } + } else { + serializer = new NestedDataColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } + + serializer.setDictionaryIdLookup(autoParent.getIdLookup()); + serializer.open(); + } + public static class ArrayDictionaryMergingIterator implements Iterator { private static final Comparator> PEEKING_ITERATOR_COMPARATOR = diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index a45c4b2d3e0..d4f6a47b05d 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -26,6 +26,7 @@ import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.filter.Filter; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -54,10 +55,12 @@ public class CursorBuildSpec private final VirtualColumns virtualColumns; @Nullable private final List aggregators; - private final List orderByColumns; + private final List preferredOrdering; private final QueryContext queryContext; + private final boolean isAggregate; + @Nullable private final QueryMetrics queryMetrics; @@ -77,9 +80,10 @@ public class CursorBuildSpec this.groupingColumns = groupingColumns; this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns"); this.aggregators = aggregators; - this.orderByColumns = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering"); + this.preferredOrdering = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering"); this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext"); this.queryMetrics = queryMetrics; + this.isAggregate = !CollectionUtils.isNullOrEmpty(groupingColumns) || !CollectionUtils.isNullOrEmpty(aggregators); } /** @@ -142,7 +146,7 @@ public class CursorBuildSpec */ public List getPreferredOrdering() { - return orderByColumns; + return preferredOrdering; } /** @@ -166,6 +170,39 @@ public class CursorBuildSpec return queryMetrics; } + /** + * Returns true if {@link #getGroupingColumns()} is not null or empty and/or {@link #getAggregators()} is not null or + * empty. This method is useful for quickly checking if it is worth considering if a {@link CursorFactory} should + * attempt to produce a {@link CursorHolder} that is {@link CursorHolder#isPreAggregated()} to satisfy the build spec. + */ + public boolean isAggregate() + { + return isAggregate; + } + + /** + * Returns true if the supplied ordering matches {@link #getPreferredOrdering()}, meaning that the supplied ordering + * has everything which is in the preferred ordering in the same direction and order. The supplied ordering may have + * additional columns beyond the preferred ordering and still satisify this method. + */ + public boolean isCompatibleOrdering(List ordering) + { + // if the build spec doesn't prefer an ordering, any order is ok + if (preferredOrdering.isEmpty()) { + return true; + } + // all columns must be present in ordering if the build spec specifies them + if (ordering.size() < preferredOrdering.size()) { + return false; + } + for (int i = 0; i < preferredOrdering.size(); i++) { + if (!ordering.get(i).equals(preferredOrdering.get(i))) { + return false; + } + } + return true; + } + public static class CursorBuildSpecBuilder { @Nullable @@ -195,7 +232,7 @@ public class CursorBuildSpec this.groupingColumns = buildSpec.groupingColumns; this.virtualColumns = buildSpec.virtualColumns; this.aggregators = buildSpec.aggregators; - this.preferredOrdering = buildSpec.orderByColumns; + this.preferredOrdering = buildSpec.preferredOrdering; this.queryContext = buildSpec.queryContext; this.queryMetrics = buildSpec.queryMetrics; } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java index 79bf2b4e557..808e1eb5384 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java @@ -75,6 +75,18 @@ public interface CursorHolder extends Closeable return false; } + /** + * Returns a set of replacement {@link AggregatorFactory} if and only if {@link #isPreAggregated()} is true. The + * query engine should replace the query aggregators with these aggregators, which are combining aggregators derived + * from the {@link CursorBuildSpec} passed into {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}. If + * {@link #isPreAggregated()} is not true, this method returns null + */ + @Nullable + default List getAggregatorsForPreAggregated() + { + return null; + } + /** * Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns * an empty list then the cursor has no defined ordering. diff --git a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java index b11498137fe..a498e081b2c 100644 --- a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java @@ -70,6 +70,7 @@ public abstract class DictionaryEncodedColumnMerger> imp private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class); protected final String dimensionName; + protected final String outputName; protected final ProgressIndicator progress; protected final Closer closer; protected final IndexSpec indexSpec; @@ -81,6 +82,7 @@ public abstract class DictionaryEncodedColumnMerger> imp protected int rowCount = 0; protected int cardinality = 0; protected boolean hasNull = false; + protected boolean writeDictionary = true; @Nullable protected GenericIndexedWriter bitmapWriter; @@ -102,6 +104,7 @@ public abstract class DictionaryEncodedColumnMerger> imp public DictionaryEncodedColumnMerger( String dimensionName, + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -110,6 +113,7 @@ public abstract class DictionaryEncodedColumnMerger> imp ) { this.dimensionName = dimensionName; + this.outputName = outputName; this.indexSpec = indexSpec; this.capabilities = capabilities; this.segmentWriteOutMedium = segmentWriteOutMedium; @@ -171,8 +175,9 @@ public abstract class DictionaryEncodedColumnMerger> imp numMergeIndex++; } - String dictFilename = StringUtils.format("%s.dim_values", dimensionName); + String dictFilename = StringUtils.format("%s.dim_values", outputName); dictionaryWriter = makeDictionaryWriter(dictFilename); + firstDictionaryValue = null; dictionarySize = 0; dictionaryWriter.open(); @@ -338,7 +343,7 @@ public abstract class DictionaryEncodedColumnMerger> imp long dimStartTime = System.currentTimeMillis(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - String bmpFilename = StringUtils.format("%s.inverted", dimensionName); + String bmpFilename = StringUtils.format("%s.inverted", outputName); bitmapWriter = new GenericIndexedWriter<>( segmentWriteOutMedium, bmpFilename, @@ -402,11 +407,11 @@ public abstract class DictionaryEncodedColumnMerger> imp { final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); - String filenameBase = StringUtils.format("%s.forward_dim", dimensionName); + String filenameBase = StringUtils.format("%s.forward_dim", outputName); if (capabilities.hasMultipleValues().isTrue()) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create( - dimensionName, + outputName, segmentWriteOutMedium, filenameBase, cardinality, @@ -414,12 +419,12 @@ public abstract class DictionaryEncodedColumnMerger> imp ); } else { encodedValueSerializer = - new VSizeColumnarMultiIntsSerializer(dimensionName, segmentWriteOutMedium, cardinality); + new VSizeColumnarMultiIntsSerializer(outputName, segmentWriteOutMedium, cardinality); } } else { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( - dimensionName, + outputName, segmentWriteOutMedium, filenameBase, cardinality, diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java index 853850625cb..4ffc7ee04c4 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java @@ -106,13 +106,17 @@ public interface DimensionHandler * * See {@link DimensionMergerV9} interface for more information. * - * @param indexSpec Specification object for the index merge - * @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if needed - * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler - * @param progress ProgressIndicator used by the merging process + * @param outputName Output "file" name for the column to use for serializers, to control where it is + * stored in the segments internal files + * @param indexSpec Specification object for the index merge + * @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if + * needed + * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler + * @param progress ProgressIndicator used by the merging process * @return A new DimensionMergerV9 object. */ DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java index b5049a89817..5e3f5cad886 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java @@ -21,6 +21,9 @@ package org.apache.druid.segment; import org.apache.druid.segment.column.ColumnDescriptor; +import java.io.IOException; +import java.util.List; + /** * Processing related interface * @@ -35,4 +38,13 @@ public interface DimensionMergerV9 extends DimensionMerger * @return ColumnDescriptor that IndexMergerV9 will use to build a column. */ ColumnDescriptor makeColumnDescriptor(); + + /** + * Attaches the {@link DimensionMergerV9} of a "projection" parent column so that stuff like value dictionaries can + * be shared between parent and child + */ + default void attachParent(DimensionMergerV9 parent, List projectionAdapters) throws IOException + { + // do nothing + } } diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java index e8166708922..035384e522b 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java @@ -77,6 +77,7 @@ public class DoubleDimensionHandler implements DimensionHandler>> projectionsColumns = new LinkedHashMap<>(); + final Metadata metadata = getMetdata(smooshedFiles, mapper, inDir); + if (metadata != null && metadata.getProjections() != null) { + for (AggregateProjectionMetadata projectionSpec : metadata.getProjections()) { + final Map> projectionColumns = readProjectionColumns( + mapper, + loadFailed, + projectionSpec, + smooshedFiles, + columns, + dataInterval + ); + + projectionsColumns.put(projectionSpec.getSchema().getName(), projectionColumns); + } + } final QueryableIndex index = new SimpleQueryableIndex( dataInterval, @@ -650,30 +669,15 @@ public class IndexIO segmentBitmapSerdeFactory.getBitmapFactory(), columns, smooshedFiles, - lazy + lazy, + metadata, + projectionsColumns ) { @Override public Metadata getMetadata() { - try { - ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); - if (metadataBB != null) { - return mapper.readValue( - SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), - Metadata.class - ); - } - } - catch (JsonParseException | JsonMappingException ex) { - // Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which - // is no longer supported then it is OK to not use the metadata instead of failing segment loading - log.warn(ex, "Failed to load metadata for segment [%s]", inDir); - } - catch (IOException ex) { - log.warn(ex, "Failed to read metadata for segment [%s]", inDir); - } - return null; + return getMetdata(smooshedFiles, mapper, inDir); } }; @@ -682,6 +686,87 @@ public class IndexIO return index; } + private Map> readProjectionColumns( + ObjectMapper mapper, + SegmentLazyLoadFailCallback loadFailed, + AggregateProjectionMetadata projectionSpec, + SmooshedFileMapper smooshedFiles, + Map> columns, + Interval dataInterval + ) throws IOException + { + final Map> projectionColumns = new LinkedHashMap<>(); + for (String groupingColumn : projectionSpec.getSchema().getGroupingColumns()) { + final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, groupingColumn); + final ByteBuffer colBuffer = smooshedFiles.mapFile(smooshName); + + final ColumnHolder parentColumn; + if (columns.containsKey(groupingColumn)) { + parentColumn = columns.get(groupingColumn).get(); + } else { + parentColumn = null; + } + registerColumnHolder( + true, + projectionColumns, + groupingColumn, + mapper, + colBuffer, + smooshedFiles, + parentColumn, + loadFailed + ); + + if (groupingColumn.equals(projectionSpec.getSchema().getTimeColumnName())) { + projectionColumns.put(ColumnHolder.TIME_COLUMN_NAME, projectionColumns.get(groupingColumn)); + } + } + for (AggregatorFactory aggregator : projectionSpec.getSchema().getAggregators()) { + final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, aggregator.getName()); + final ByteBuffer aggBuffer = smooshedFiles.mapFile(smooshName); + registerColumnHolder( + true, + projectionColumns, + aggregator.getName(), + mapper, + aggBuffer, + smooshedFiles, + null, + loadFailed + ); + } + if (projectionSpec.getSchema().getTimeColumnName() == null) { + projectionColumns.put( + ColumnHolder.TIME_COLUMN_NAME, + Projections.makeConstantTimeSupplier(projectionSpec.getNumRows(), dataInterval.getStartMillis()) + ); + } + return projectionColumns; + } + + @Nullable + private Metadata getMetdata(SmooshedFileMapper smooshedFiles, ObjectMapper mapper, File inDir) + { + try { + ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); + if (metadataBB != null) { + return mapper.readValue( + SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), + Metadata.class + ); + } + } + catch (JsonParseException | JsonMappingException ex) { + // Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which + // is no longer supported then it is OK to not use the metadata instead of failing segment loading + log.warn(ex, "Failed to load metadata for segment [%s]", inDir); + } + catch (IOException ex) { + log.warn(ex, "Failed to read metadata for segment [%s]", inDir); + } + return null; + } + /** * Return a list of columns that contains given inputs merged. The returned column names are in * the original order that is used when this segment is created. @@ -740,6 +825,7 @@ public class IndexIO mapper, colBuffer, smooshedFiles, + null, loadFailed ); } @@ -752,6 +838,7 @@ public class IndexIO ObjectMapper mapper, ByteBuffer colBuffer, SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parentColumn, SegmentLazyLoadFailCallback loadFailed ) throws IOException { @@ -767,7 +854,8 @@ public class IndexIO internedColumnName, mapper, colBuffer, - smooshedFiles + smooshedFiles, + parentColumn ); } catch (IOException | RuntimeException e) { @@ -782,7 +870,8 @@ public class IndexIO internedColumnName, mapper, colBuffer, - smooshedFiles + smooshedFiles, + parentColumn ); columns.put(internedColumnName, () -> columnHolder); } @@ -797,11 +886,12 @@ public class IndexIO String columnName, // columnName is not used in this method, but used in tests. ObjectMapper mapper, ByteBuffer byteBuffer, - SmooshedFileMapper smooshedFiles + SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parentColumn ) throws IOException { ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class); - return serde.read(byteBuffer, columnConfig, smooshedFiles); + return serde.read(byteBuffer, columnConfig, smooshedFiles, parentColumn); } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java index f397b02814c..1a7a587c8c1 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java @@ -354,7 +354,7 @@ public interface IndexMerger // type of column doesn't have any kind of special per-index encoding that needs to be converted to the "global" // encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9. //noinspection ObjectEquality - anySelectorChanged |= convertedDimensionSelector != sourceDimensionSelector; + anySelectorChanged = anySelectorChanged || convertedDimensionSelector != sourceDimensionSelector; convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues( indexNumber, diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index fd3400d6c1f..04cc579208d 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.inject.Inject; @@ -51,6 +52,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; import org.apache.druid.segment.loading.SegmentizerFactory; +import org.apache.druid.segment.projections.Projections; import org.apache.druid.segment.serde.ColumnPartSerde; import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.serde.ComplexMetricSerde; @@ -64,6 +66,7 @@ import org.apache.druid.segment.serde.LongNumericColumnPartSerdeV2; import org.apache.druid.segment.serde.NullColumnPartSerde; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -138,7 +141,6 @@ public class IndexMergerV9 implements IndexMerger final DimensionsSpecInspector dimensionsSpecInspector, final List mergedMetrics, final Function, TimeAndDimsIterator> rowMergerFn, - final boolean fillRowNumConversions, final IndexSpec indexSpec, final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException @@ -154,7 +156,7 @@ public class IndexMergerV9 implements IndexMerger .filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim)) .collect(Collectors.toList()); - final Metadata segmentMetadata; + Metadata segmentMetadata; if (metricAggs != null) { AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length]; for (int i = 0; i < metricAggs.length; i++) { @@ -215,18 +217,20 @@ public class IndexMergerV9 implements IndexMerger mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats); final Map handlers = makeDimensionHandlers(mergedDimensions, dimFormats); + final Map mergersMap = Maps.newHashMapWithExpectedSize(mergedDimensions.size()); final List mergers = new ArrayList<>(); for (int i = 0; i < mergedDimensions.size(); i++) { DimensionHandler handler = handlers.get(mergedDimensions.get(i)); - mergers.add( - handler.makeMerger( - indexSpec, - segmentWriteOutMedium, - dimFormats.get(i).toColumnCapabilities(), - progress, - closer - ) + DimensionMergerV9 merger = handler.makeMerger( + mergedDimensions.get(i), + indexSpec, + segmentWriteOutMedium, + dimFormats.get(i).toColumnCapabilities(), + progress, + closer ); + mergers.add(merger); + mergersMap.put(mergedDimensions.get(i), merger); } /************* Setup Dim Conversions **************/ @@ -255,8 +259,7 @@ public class IndexMergerV9 implements IndexMerger timeAndDimsIterator, timeWriter, metricWriters, - mergers, - fillRowNumConversions + mergers ); /************ Create Inverted Indexes and Finalize Build Columns *************/ @@ -291,6 +294,20 @@ public class IndexMergerV9 implements IndexMerger progress.stopSection(section); + if (segmentMetadata != null && !CollectionUtils.isNullOrEmpty(segmentMetadata.getProjections())) { + segmentMetadata = makeProjections( + v9Smoosher, + segmentMetadata.getProjections(), + adapters, + indexSpec, + segmentWriteOutMedium, + progress, + closer, + mergersMap, + segmentMetadata + ); + } + /************* Make index.drd & metadata.drd files **************/ progress.progress(); makeIndexBinary( @@ -332,6 +349,193 @@ public class IndexMergerV9 implements IndexMerger } } + private Metadata makeProjections( + final FileSmoosher smoosher, + final List projections, + final List adapters, + final IndexSpec indexSpec, + final SegmentWriteOutMedium segmentWriteOutMedium, + final ProgressIndicator progress, + final Closer closer, + final Map parentMergers, + final Metadata segmentMetadata + ) throws IOException + { + final List projectionMetadata = Lists.newArrayListWithCapacity(projections.size()); + for (AggregateProjectionMetadata spec : projections) { + final List projectionAdapters = Lists.newArrayListWithCapacity(adapters.size()); + final AggregateProjectionMetadata.Schema projectionSchema = spec.getSchema(); + for (IndexableAdapter adapter : adapters) { + projectionAdapters.add(adapter.getProjectionAdapter(projectionSchema.getName())); + } + // we can use the first adapter to get the dimensions and metrics because the projection schema should be + // identical across all segments. This is validated by segment metadata merging + final List dimensions = projectionAdapters.get(0).getDimensionNames(false); + final List metrics = Arrays.stream(projectionSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); + + + final List mergers = new ArrayList<>(); + final Map columnFormats = Maps.newLinkedHashMapWithExpectedSize(dimensions.size() + metrics.size()); + + for (String dimension : dimensions) { + final ColumnFormat dimensionFormat = projectionAdapters.get(0).getFormat(dimension); + columnFormats.put(dimension, dimensionFormat); + DimensionHandler handler = dimensionFormat.getColumnHandler(dimension); + DimensionMergerV9 merger = handler.makeMerger( + Projections.getProjectionSmooshV9FileName(spec, dimension), + indexSpec, + segmentWriteOutMedium, + dimensionFormat.toColumnCapabilities(), + progress, + closer + ); + if (parentMergers.containsKey(dimension)) { + merger.attachParent(parentMergers.get(dimension), projectionAdapters); + } else { + merger.writeMergedValueDictionary(projectionAdapters); + } + mergers.add(merger); + } + for (String metric : metrics) { + columnFormats.put(metric, projectionAdapters.get(0).getFormat(metric)); + } + + final GenericColumnSerializer timeWriter; + if (projectionSchema.getTimeColumnName() != null) { + timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); + } else { + timeWriter = null; + } + final ArrayList metricWriters = + setupMetricsWriters( + segmentWriteOutMedium, + metrics, + columnFormats, + indexSpec, + Projections.getProjectionSmooshV9Prefix(spec) + ); + + Function, TimeAndDimsIterator> rowMergerFn = + rowIterators -> new RowCombiningTimeAndDimsIterator(rowIterators, projectionSchema.getAggregators(), metrics); + + List perIndexRowIterators = Lists.newArrayListWithCapacity(projectionAdapters.size()); + for (int i = 0; i < projectionAdapters.size(); ++i) { + final IndexableAdapter adapter = projectionAdapters.get(i); + TransformableRowIterator target = adapter.getRows(); + perIndexRowIterators.add(IndexMerger.toMergedIndexRowIterator(target, i, mergers)); + } + final TimeAndDimsIterator timeAndDimsIterator = rowMergerFn.apply(perIndexRowIterators); + closer.register(timeAndDimsIterator); + + int rowCount = 0; + List rowNumConversions = new ArrayList<>(projectionAdapters.size()); + for (IndexableAdapter adapter : projectionAdapters) { + int[] arr = new int[adapter.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); + } + + final String section = "walk through and merge projection[" + projectionSchema.getName() + "] rows"; + progress.startSection(section); + long startTime = System.currentTimeMillis(); + long time = startTime; + while (timeAndDimsIterator.moveToNext()) { + progress.progress(); + TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer(); + if (timeWriter != null) { + timeWriter.serialize(timeAndDims.timestampSelector); + } + + for (int metricIndex = 0; metricIndex < timeAndDims.getNumMetrics(); metricIndex++) { + metricWriters.get(metricIndex).serialize(timeAndDims.getMetricSelector(metricIndex)); + } + + for (int dimIndex = 0; dimIndex < timeAndDims.getNumDimensions(); dimIndex++) { + DimensionMergerV9 merger = mergers.get(dimIndex); + if (merger.hasOnlyNulls()) { + continue; + } + merger.processMergedRow(timeAndDims.getDimensionSelector(dimIndex)); + } + + RowCombiningTimeAndDimsIterator comprisedRows = (RowCombiningTimeAndDimsIterator) timeAndDimsIterator; + + for (int originalIteratorIndex = comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(0); + originalIteratorIndex >= 0; + originalIteratorIndex = + comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(originalIteratorIndex + 1)) { + + IntBuffer conversionBuffer = rowNumConversions.get(originalIteratorIndex); + int minRowNum = comprisedRows.getMinCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex); + int maxRowNum = comprisedRows.getMaxCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex); + + for (int rowNum = minRowNum; rowNum <= maxRowNum; rowNum++) { + while (conversionBuffer.position() < rowNum) { + conversionBuffer.put(INVALID_ROW); + } + conversionBuffer.put(rowCount); + } + } + if ((++rowCount % 500000) == 0) { + log.debug( + "walked 500,000/%d rows of projection[%s] in %,d millis.", + rowCount, + projectionSchema.getName(), + System.currentTimeMillis() - time + ); + time = System.currentTimeMillis(); + } + } + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); + } + log.debug( + "completed walk through of %,d rows of projection[%s] in %,d millis.", + rowCount, + projectionSchema.getName(), + System.currentTimeMillis() - startTime + ); + progress.stopSection(section); + + final String section2 = "build projection[" + projectionSchema.getName() + "] inverted index and columns"; + progress.startSection(section2); + if (projectionSchema.getTimeColumnName() != null) { + makeTimeColumn( + smoosher, + progress, + timeWriter, + indexSpec, + Projections.getProjectionSmooshV9FileName(spec, projectionSchema.getTimeColumnName()) + ); + } + makeMetricsColumns( + smoosher, + progress, + metrics, + columnFormats, + metricWriters, + indexSpec, + Projections.getProjectionSmooshV9Prefix(spec) + ); + + for (int i = 0; i < dimensions.size(); i++) { + final String dimension = dimensions.get(i); + DimensionMergerV9 merger = mergers.get(i); + merger.writeIndexes(rowNumConversions); + if (!merger.hasOnlyNulls()) { + ColumnDescriptor columnDesc = merger.makeColumnDescriptor(); + makeColumn(smoosher, Projections.getProjectionSmooshV9FileName(spec, dimension), columnDesc); + } + } + + progress.stopSection(section2); + projectionMetadata.add(new AggregateProjectionMetadata(projectionSchema, rowCount)); + } + return segmentMetadata.withProjections(projectionMetadata); + } + private void makeIndexBinary( final FileSmoosher v9Smoosher, final List adapters, @@ -348,7 +552,7 @@ public class IndexMergerV9 implements IndexMerger columnSet.addAll(mergedMetrics); Preconditions.checkState( columnSet.size() == mergedDimensions.size() + mergedMetrics.size(), - "column names are not unique in dims%s and mets%s", + "column names are not unique in dims[%s] and mets[%s]", mergedDimensions, mergedMetrics ); @@ -444,6 +648,18 @@ public class IndexMergerV9 implements IndexMerger final List metWriters, final IndexSpec indexSpec ) throws IOException + { + makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsTypes, metWriters, indexSpec, ""); + } + private void makeMetricsColumns( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final List mergedMetrics, + final Map metricsTypes, + final List metWriters, + final IndexSpec indexSpec, + final String namePrefix + ) throws IOException { final String section = "make metric columns"; progress.startSection(section); @@ -483,8 +699,9 @@ public class IndexMergerV9 implements IndexMerger default: throw new ISE("Unknown type[%s]", type); } - makeColumn(v9Smoosher, metric, builder.build()); - log.debug("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime); + final String columnName = namePrefix + metric; + makeColumn(v9Smoosher, columnName, builder.build()); + log.debug("Completed metric column[%s] in %,d millis.", columnName, System.currentTimeMillis() - metricStartTime); } log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); @@ -547,6 +764,17 @@ public class IndexMergerV9 implements IndexMerger final GenericColumnSerializer timeWriter, final IndexSpec indexSpec ) throws IOException + { + makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec, ColumnHolder.TIME_COLUMN_NAME); + } + + private void makeTimeColumn( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final GenericColumnSerializer timeWriter, + final IndexSpec indexSpec, + final String name + ) throws IOException { final String section = "make time column"; progress.startSection(section); @@ -557,7 +785,7 @@ public class IndexMergerV9 implements IndexMerger .setValueType(ValueType.LONG) .addSerde(createLongColumnPartSerde(timeWriter, indexSpec)) .build(); - makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator); + makeColumn(v9Smoosher, name, serdeficator); log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); } @@ -601,23 +829,19 @@ public class IndexMergerV9 implements IndexMerger final TimeAndDimsIterator timeAndDimsIterator, final GenericColumnSerializer timeWriter, final ArrayList metricWriters, - final List mergers, - final boolean fillRowNumConversions + final List mergers ) throws IOException { final String section = "walk through and merge rows"; progress.startSection(section); long startTime = System.currentTimeMillis(); - List rowNumConversions = null; int rowCount = 0; - if (fillRowNumConversions) { - rowNumConversions = new ArrayList<>(adapters.size()); - for (IndexableAdapter adapter : adapters) { - int[] arr = new int[adapter.getNumRows()]; - Arrays.fill(arr, INVALID_ROW); - rowNumConversions.add(IntBuffer.wrap(arr)); - } + List rowNumConversions = new ArrayList<>(adapters.size()); + for (IndexableAdapter adapter : adapters) { + int[] arr = new int[adapter.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); } long time = System.currentTimeMillis(); @@ -656,9 +880,7 @@ public class IndexMergerV9 implements IndexMerger } conversionBuffer.put(rowCount); } - } - } else if (timeAndDimsIterator instanceof MergingRowIterator) { RowPointer rowPointer = (RowPointer) timeAndDims; IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum()); @@ -668,11 +890,9 @@ public class IndexMergerV9 implements IndexMerger } conversionBuffer.put(rowCount); } else { - if (fillRowNumConversions) { - throw new IllegalStateException( - "Filling row num conversions is supported only with RowCombining and Merging iterators" - ); - } + throw new IllegalStateException( + "Filling row num conversions is supported only with RowCombining and Merging iterators" + ); } if ((++rowCount % 500000) == 0) { @@ -680,10 +900,8 @@ public class IndexMergerV9 implements IndexMerger time = System.currentTimeMillis(); } } - if (rowNumConversions != null) { - for (IntBuffer rowNumConversion : rowNumConversions) { - rowNumConversion.rewind(); - } + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); } log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); progress.stopSection(section); @@ -711,28 +929,40 @@ public class IndexMergerV9 implements IndexMerger final Map metricsTypes, final IndexSpec indexSpec ) throws IOException + { + return setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsTypes, indexSpec, ""); + } + + private ArrayList setupMetricsWriters( + final SegmentWriteOutMedium segmentWriteOutMedium, + final List mergedMetrics, + final Map metricsTypes, + final IndexSpec indexSpec, + final String prefix + ) throws IOException { ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); for (String metric : mergedMetrics) { TypeSignature type = metricsTypes.get(metric).getLogicalType(); + final String outputName = prefix + metric; GenericColumnSerializer writer; switch (type.getType()) { case LONG: - writer = createLongColumnSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); break; case FLOAT: - writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); break; case DOUBLE: - writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); break; case COMPLEX: ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName()); if (serde == null) { throw new ISE("Unknown type[%s]", type.getComplexTypeName()); } - writer = serde.getSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = serde.getSerializer(segmentWriteOutMedium, outputName, indexSpec); break; default: throw new ISE("Unknown type[%s]", type); @@ -891,7 +1121,7 @@ public class IndexMergerV9 implements IndexMerger FileUtils.mkdirp(outDir); - log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); + log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.numRows()); return multiphaseMerge( Collections.singletonList( new IncrementalIndexAdapter( @@ -1179,7 +1409,6 @@ public class IndexMergerV9 implements IndexMerger new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec), mergedMetrics, rowMergerFn, - true, indexSpec, segmentWriteOutMediumFactory ); diff --git a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java index 023ca700192..1d708e20844 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java @@ -64,6 +64,8 @@ public interface IndexableAdapter TransformableRowIterator getRows(); + IndexableAdapter getProjectionAdapter(String projection); + BitmapValues getBitmapValues(String dimension, int dictId); ColumnCapabilities getCapabilities(String column); diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java index 64a9f98cd44..e93a71be883 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java @@ -77,6 +77,7 @@ public class LongDimensionHandler implements DimensionHandler @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -85,7 +86,7 @@ public class LongDimensionHandler implements DimensionHandler ) { return new LongDimensionMergerV9( - dimensionName, + outputName, indexSpec, segmentWriteOutMedium ); diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java index e4ab6ba3849..b6a96e4f83c 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java @@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium; public class LongDimensionMergerV9 extends NumericDimensionMergerV9 { - LongDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) + LongDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - super(dimensionName, indexSpec, segmentWriteOutMedium); + super(outputName, indexSpec, segmentWriteOutMedium); } @Override GenericColumnSerializer setupEncodedValueWriter() { - return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); + return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/Metadata.java b/processing/src/main/java/org/apache/druid/segment/Metadata.java index e21c673e279..1da2ecce93c 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -22,6 +22,7 @@ package org.apache.druid.segment; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularity; @@ -56,6 +57,8 @@ public class Metadata private final Boolean rollup; @Nullable private final List ordering; + @Nullable + private final List projections; public Metadata( @JsonProperty("container") @Nullable Map container, @@ -63,7 +66,8 @@ public class Metadata @JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec, @JsonProperty("queryGranularity") @Nullable Granularity queryGranularity, @JsonProperty("rollup") @Nullable Boolean rollup, - @JsonProperty("ordering") @Nullable List ordering + @JsonProperty("ordering") @Nullable List ordering, + @JsonProperty("projections") @Nullable List projections ) { this.container = container == null ? new ConcurrentHashMap<>() : container; @@ -72,6 +76,7 @@ public class Metadata this.queryGranularity = queryGranularity; this.rollup = rollup; this.ordering = ordering; + this.projections = projections; } @JsonProperty @@ -124,6 +129,27 @@ public class Metadata return ordering; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public List getProjections() + { + return projections; + } + + public Metadata withProjections(List projections) + { + return new Metadata( + container, + aggregators, + timestampSpec, + queryGranularity, + rollup, + ordering, + projections + ); + } + public Metadata putAll(@Nullable Map other) { if (other != null) { @@ -155,6 +181,7 @@ public class Metadata List gransToMerge = new ArrayList<>(); List rollupToMerge = new ArrayList<>(); List> orderingsToMerge = new ArrayList<>(); + List> projectionsToMerge = new ArrayList<>(); for (Metadata metadata : toBeMerged) { if (metadata != null) { @@ -176,6 +203,7 @@ public class Metadata } orderingsToMerge.add(metadata.getOrdering()); + projectionsToMerge.add(metadata.getProjections()); mergedContainer.putAll(metadata.container); } else { //if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then @@ -204,6 +232,7 @@ public class Metadata Granularity.mergeGranularities(gransToMerge); final List mergedOrdering = mergeOrderings(orderingsToMerge); + validateProjections(projectionsToMerge); Boolean rollup = null; if (rollupToMerge != null && !rollupToMerge.isEmpty()) { @@ -227,7 +256,8 @@ public class Metadata mergedTimestampSpec, mergedGranularity, rollup, - mergedOrdering + mergedOrdering, + projectionsToMerge.get(0) // we're going to replace this later with updated rowcount ); } @@ -246,13 +276,14 @@ public class Metadata Objects.equals(timestampSpec, metadata.timestampSpec) && Objects.equals(queryGranularity, metadata.queryGranularity) && Objects.equals(rollup, metadata.rollup) && - Objects.equals(ordering, metadata.ordering); + Objects.equals(ordering, metadata.ordering) && + Objects.equals(projections, metadata.projections); } @Override public int hashCode() { - return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup); + return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup, ordering, projections); } @Override @@ -265,6 +296,7 @@ public class Metadata ", queryGranularity=" + queryGranularity + ", rollup=" + rollup + ", ordering=" + ordering + + ", projections=" + projections + '}'; } @@ -308,4 +340,34 @@ public class Metadata mergedOrdering.add(orderBy); } } + + public static void validateProjections(List> projectionsToMerge) + { + final Map projectionsMap = new HashMap<>(); + // dedupe by name, fail if somehow incompatible projections are defined + int nullCount = 0; + int expectedSize = -1; + for (List projections : projectionsToMerge) { + if (projections == null) { + nullCount++; + continue; + } + if (expectedSize < 0) { + expectedSize = projections.size(); + } else if (projections.size() != expectedSize) { + throw DruidException.defensive("Unable to merge projections: mismatched projections count"); + } + for (AggregateProjectionMetadata projection : projections) { + AggregateProjectionMetadata prev = projectionsMap.putIfAbsent(projection.getSchema().getName(), projection); + if (prev != null && !prev.getSchema().equals(projection.getSchema())) { + throw DruidException.defensive("Unable to merge projections: mismatched projections [%s] and [%s]", prev, projection); + } + } + } + if (nullCount > 0) { + if (nullCount != projectionsToMerge.size()) { + throw DruidException.defensive("Unable to merge projections: some projections were null"); + } + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java index e679b2d2d9c..348d40e4f2d 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java @@ -77,6 +77,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler getProjection(CursorBuildSpec cursorBuildSpec) + { + return null; + } + + @Nullable + default QueryableIndex getProjectionQueryableIndex(String name) + { + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java index 45892326711..8745761c183 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java @@ -20,13 +20,19 @@ package org.apache.druid.segment; import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.data.Offset; +import org.apache.druid.segment.projections.QueryableProjection; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorOffset; import javax.annotation.Nullable; import java.util.LinkedHashSet; +import java.util.List; public class QueryableIndexCursorFactory implements CursorFactory { @@ -40,6 +46,46 @@ public class QueryableIndexCursorFactory implements CursorFactory @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { + QueryableProjection projection = index.getProjection(spec); + if (projection != null) { + return new QueryableIndexCursorHolder(projection.getRowSelector(), projection.getCursorBuildSpec()) + { + @Override + protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset( + ColumnCache columnCache, + Offset baseOffset + ) + { + return projection.wrapColumnSelectorFactory( + super.makeColumnSelectorFactoryForOffset(columnCache, baseOffset) + ); + } + + @Override + protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( + ColumnCache columnCache, + VectorOffset baseOffset + ) + { + return projection.wrapVectorColumnSelectorFactory( + super.makeVectorColumnSelectorFactoryForOffset(columnCache, baseOffset) + ); + } + + @Override + public boolean isPreAggregated() + { + return true; + } + + @Nullable + @Override + public List getAggregatorsForPreAggregated() + { + return projection.getCursorBuildSpec().getAggregators(); + } + }; + } return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build()); } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 4b295931f0a..54e482051ae 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -214,11 +214,9 @@ public class QueryableIndexCursorHolder implements CursorHolder } final Offset baseCursorOffset = offset.clone(); - final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( - virtualColumns, - Cursors.getTimeOrdering(ordering), - baseCursorOffset.getBaseReadableOffset(), - columnCache + final ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactoryForOffset( + columnCache, + baseCursorOffset ); // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter // needs to use a value matcher @@ -327,7 +325,20 @@ public class QueryableIndexCursorHolder implements CursorHolder } - private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( + protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset( + ColumnCache columnCache, + Offset baseOffset + ) + { + return new QueryableIndexColumnSelectorFactory( + virtualColumns, + Cursors.getTimeOrdering(ordering), + baseOffset.getBaseReadableOffset(), + columnCache + ); + } + + protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( ColumnCache columnCache, VectorOffset baseOffset ) diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index d9913036b5a..fdec3477cd8 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -262,6 +262,14 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter return new RowIteratorImpl(); } + @Override + public IndexableAdapter getProjectionAdapter(String projection) + { + QueryableIndex projectionIndex = input.getProjectionQueryableIndex(projection); + DruidException.conditionalDefensive(projectionIndex != null, "Projection[%s] was not found", projection); + return new QueryableIndexIndexableAdapter(projectionIndex); + } + /** * On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link * SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same diff --git a/processing/src/main/java/org/apache/druid/segment/RemapColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RemapColumnSelectorFactory.java new file mode 100644 index 00000000000..4d4858a2aea --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/RemapColumnSelectorFactory.java @@ -0,0 +1,73 @@ +/* + * 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; + +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.ColumnCapabilities; + +import javax.annotation.Nullable; +import java.util.Map; + +public class RemapColumnSelectorFactory implements ColumnSelectorFactory +{ + private final ColumnSelectorFactory delegate; + private final Map remap; + + public RemapColumnSelectorFactory(ColumnSelectorFactory delegate, Map remap) + { + this.delegate = delegate; + this.remap = remap; + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension())); + return delegate.makeDimensionSelector(remapDimensionSpec); + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String columnName) + { + return delegate.makeColumnValueSelector(remap.getOrDefault(columnName, columnName)); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(remap.getOrDefault(column, column)); + } + + @Nullable + @Override + public RowIdSupplier getRowIdSupplier() + { + return delegate.getRowIdSupplier(); + } + + @Nullable + @Override + public ExpressionType getType(String name) + { + return delegate.getType(remap.getOrDefault(name, name)); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 9518876cb75..6b48d304b80 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -24,32 +24,45 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; import com.google.common.collect.Maps; +import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.segment.projections.Projections; +import org.apache.druid.segment.projections.QueryableProjection; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.SortedSet; /** * */ public abstract class SimpleQueryableIndex implements QueryableIndex { + public static final Interner> ORDERING_INTERNER = Interners.newWeakInterner(); + private final Interval dataInterval; private final List columnNames; private final Indexed availableDimensions; private final BitmapFactory bitmapFactory; private final Map> columns; + private final List ordering; + private final Map projectionsMap; + private final SortedSet projections; + private final Map>> projectionColumns; private final SmooshedFileMapper fileMapper; private final Supplier> dimensionHandlers; - private final List ordering; public SimpleQueryableIndex( Interval dataInterval, @@ -59,6 +72,20 @@ public abstract class SimpleQueryableIndex implements QueryableIndex SmooshedFileMapper fileMapper, boolean lazy ) + { + this(dataInterval, dimNames, bitmapFactory, columns, fileMapper, lazy, null, null); + } + + public SimpleQueryableIndex( + Interval dataInterval, + Indexed dimNames, + BitmapFactory bitmapFactory, + Map> columns, + SmooshedFileMapper fileMapper, + boolean lazy, + @Nullable Metadata metadata, + @Nullable Map>> projectionColumns + ) { Preconditions.checkNotNull(columns.get(ColumnHolder.TIME_COLUMN_NAME)); this.dataInterval = Preconditions.checkNotNull(dataInterval, "dataInterval"); @@ -79,18 +106,35 @@ public abstract class SimpleQueryableIndex implements QueryableIndex this.columns = columns; this.fileMapper = fileMapper; + this.projectionColumns = projectionColumns == null ? Collections.emptyMap() : projectionColumns; + if (lazy) { this.dimensionHandlers = Suppliers.memoize(() -> initDimensionHandlers(availableDimensions)); } else { this.dimensionHandlers = () -> initDimensionHandlers(availableDimensions); } - - final Metadata metadata = getMetadata(); - if (metadata != null && metadata.getOrdering() != null) { - this.ordering = metadata.getOrdering(); + if (metadata != null) { + if (metadata.getOrdering() != null) { + this.ordering = ORDERING_INTERNER.intern(metadata.getOrdering()); + } else { + this.ordering = Cursors.ascendingTimeOrder(); + } + if (metadata.getProjections() != null) { + this.projectionsMap = Maps.newHashMapWithExpectedSize(metadata.getProjections().size()); + this.projections = new ObjectAVLTreeSet<>(AggregateProjectionMetadata.COMPARATOR); + for (AggregateProjectionMetadata projection : metadata.getProjections()) { + projections.add(projection); + projectionsMap.put(projection.getSchema().getName(), projection); + } + } else { + this.projectionsMap = Collections.emptyMap(); + this.projections = Collections.emptySortedSet(); + } } else { // When sort order isn't set in metadata.drd, assume the segment is sorted by __time. this.ordering = Cursors.ascendingTimeOrder(); + this.projections = Collections.emptySortedSet(); + this.projectionsMap = Collections.emptyMap(); } } @@ -177,4 +221,54 @@ public abstract class SimpleQueryableIndex implements QueryableIndex } return dimensionHandlerMap; } + + @Nullable + @Override + public QueryableProjection getProjection(CursorBuildSpec cursorBuildSpec) + { + return Projections.findMatchingProjection( + cursorBuildSpec, + projections, + (projectionName, columnName) -> + projectionColumns.get(projectionName).containsKey(columnName) || getColumnCapabilities(columnName) == null, + this::getProjectionQueryableIndex + ); + } + + @Override + public QueryableIndex getProjectionQueryableIndex(String name) + { + final AggregateProjectionMetadata projectionSpec = projectionsMap.get(name); + return new SimpleQueryableIndex( + dataInterval, + new ListIndexed<>(projectionSpec.getSchema().getGroupingColumns()), + bitmapFactory, + projectionColumns.get(name), + fileMapper, + true, + null, + null + ) + { + @Override + public Metadata getMetadata() + { + return null; + } + + @Override + public int getNumRows() + { + return projectionSpec.getNumRows(); + } + + @Override + public List getOrdering() + { + // return ordering with projection time column substituted with __time so query engines can treat it equivalently + return projectionSpec.getSchema().getOrderingWithTimeColumnSubstitution(); + } + }; + } } + diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java index 49366023231..3bb1e721727 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java @@ -164,6 +164,7 @@ public class StringDimensionHandler implements DimensionHandler spatialWriter; + /** + * @param dimensionName column name + * @param outputName output smoosh file name. if this is a base table column, it will be the equivalent to + * name, however if this merger is for a projection, this will be prefixed with the + * projection name so that multiple projections can store the same column name at + * different smoosh file "paths" + * @param indexSpec segment level storage options such as compression format and bitmap type + * @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment + * @param capabilities options for writing the column such as if we should write bitmap or spatial indexes + * @param progress hook to update status of what this merger is doing during segment persist and merging + * @param closer resource closer if this merger needs to attach any closables that should be cleaned up + * when the segment is finished writing + */ public StringDimensionMergerV9( String dimensionName, + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -72,7 +87,7 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger projectionAdapters) throws IOException + { + DruidException.conditionalDefensive( + parent instanceof StringDimensionMergerV9, + "Projection parent column must be same type, got [%s]", + parent.getClass() + ); + StringDimensionMergerV9 stringParent = (StringDimensionMergerV9) parent; + dictionarySize = stringParent.dictionarySize; + dimConversions = stringParent.dimConversions; + dictionaryWriter = stringParent.dictionaryWriter; + cardinality = dictionaryWriter.getCardinality(); + adapters = projectionAdapters; + setupEncodedValueWriter(); + writeDictionary = false; + } + /** * Write spatial indexes for string columns that have them */ diff --git a/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java b/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java index 4472ef099ae..86fce26bb2a 100644 --- a/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java +++ b/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java @@ -94,8 +94,8 @@ public class TimeAndDimsPointer implements Comparable this.timestampSelector = timestampSelector; this.timePosition = timePosition; Preconditions.checkArgument( - timePosition >= 0 && timePosition <= dimensionSelectors.length, - "timePosition[%s] is out of range 0 .. [%s]", + timePosition <= dimensionSelectors.length, + "timePosition[%s] must be less than [%s]", timePosition, dimensionSelectors.length ); diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java index 276cbb03f1a..08b268241bd 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java @@ -99,7 +99,8 @@ public class ColumnDescriptor implements Serializer public ColumnHolder read( ByteBuffer buffer, ColumnConfig columnConfig, - SmooshedFileMapper smooshedFiles + SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parent ) { final ColumnBuilder builder = new ColumnBuilder() @@ -108,9 +109,8 @@ public class ColumnDescriptor implements Serializer .setFileMapper(smooshedFiles); for (ColumnPartSerde part : parts) { - part.getDeserializer().read(buffer, builder, columnConfig); + part.getDeserializer().read(buffer, builder, columnConfig, parent); } - return builder.build(); } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java index 11b472c26d3..94a41a9e862 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.column; +import com.google.common.base.Supplier; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; @@ -45,8 +46,15 @@ public interface ColumnHolder } int getLength(); + BaseColumn getColumn(); + @Nullable + default Supplier getColumnSupplier() + { + return this::getColumn; + } + @Nullable ColumnIndexSupplier getIndexSupplier(); diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java index 30dc4a464ca..ca38c34ed68 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java @@ -99,6 +99,12 @@ class SimpleColumnHolder implements ColumnHolder return columnSupplier == null ? UnknownTypeComplexColumn.instance() : columnSupplier.get(); } + @Override + public Supplier getColumnSupplier() + { + return columnSupplier; + } + @Nullable @Override public ColumnIndexSupplier getIndexSupplier() diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index 8adc47f6533..9f8efaf73bc 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -52,6 +52,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionIndexer; @@ -75,6 +76,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.projections.QueryableProjection; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -85,6 +87,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.Closeable; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; @@ -363,10 +366,16 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C incrementalIndexSchema.getTimestampSpec(), this.queryGranularity, this.rollup, - getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()) + getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()), + Collections.emptyList() ); } + @Nullable + public abstract QueryableProjection getProjection(CursorBuildSpec buildSpec); + + public abstract IncrementalIndexRowSelector getProjection(String name); + public abstract boolean canAppendRow(); public abstract String getOutOfRowsReason(); @@ -436,6 +445,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C } @Nullable + @Override public ColumnFormat getColumnFormat(String columnName) { if (timeAndMetricsColumnFormats.containsKey(columnName)) { @@ -670,7 +680,8 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C return numEntries.get() == 0; } - public int size() + @Override + public int numRows() { return numEntries.get(); } @@ -718,6 +729,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C * * @param includeTime whether to include {@link ColumnHolder#TIME_COLUMN_NAME}. */ + @Override public List getDimensionNames(final boolean includeTime) { synchronized (dimensionDescs) { @@ -745,6 +757,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C /** * Returns a descriptor for each dimension. Does not inclue {@link ColumnHolder#TIME_COLUMN_NAME}. */ + @Override public List getDimensions() { synchronized (dimensionDescs) { @@ -777,6 +790,12 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C return metadata.getOrdering(); } + @Override + public int getTimePosition() + { + return timePosition; + } + public static ColumnValueSelector makeMetricColumnValueSelector( IncrementalIndexRowSelector rowSelector, IncrementalIndexRowHolder currEntry, @@ -861,7 +880,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C ) { synchronized (dimensionDescs) { - if (size() != 0) { + if (numRows() != 0) { throw new ISE("Cannot load dimension order[%s] when existing index is not empty.", dimensionDescs.keySet()); } for (String dim : oldDimensionOrder) { @@ -888,6 +907,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates); } + @Override public List getMetricNames() { return ImmutableList.copyOf(metricDescs.keySet()); @@ -932,7 +952,6 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); } - private static String getSimplifiedEventStringFromRow(InputRow inputRow) { if (inputRow instanceof MapBasedInputRow) { @@ -985,6 +1004,14 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C this.indexer = handler.makeIndexer(useMaxMemoryEstimates); } + public DimensionDesc(int index, String name, DimensionHandler handler, DimensionIndexer indexer) + { + this.index = index; + this.name = name; + this.handler = handler; + this.indexer = indexer; + } + public int getIndex() { return index; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java index aae1c133575..35736cc4975 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.incremental; import it.unimi.dsi.fastutil.ints.IntIterator; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.error.DruidException; import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.IndexableAdapter; @@ -46,13 +47,15 @@ import java.util.stream.Collectors; public class IncrementalIndexAdapter implements IndexableAdapter { private final Interval dataInterval; - private final IncrementalIndex index; + private final IncrementalIndexRowSelector index; private final Map accessors; + private final BitmapFactory bitmapFactory; - public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory) + public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndexRowSelector index, BitmapFactory bitmapFactory) { this.dataInterval = dataInterval; this.index = index; + this.bitmapFactory = bitmapFactory; final List dimensions = index.getDimensions(); accessors = dimensions @@ -73,7 +76,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter * a null value. */ private void processRows( - IncrementalIndex index, + IncrementalIndexRowSelector index, BitmapFactory bitmapFactory, List dimensions ) @@ -112,7 +115,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter @Override public int getNumRows() { - return index.size(); + return index.numRows(); } @Override @@ -182,6 +185,16 @@ public class IncrementalIndexAdapter implements IndexableAdapter return new IncrementalIndexRowIterator(index); } + @Override + public IndexableAdapter getProjectionAdapter(String projection) + { + if (index instanceof IncrementalIndex) { + IncrementalIndexRowSelector projectionSelector = ((IncrementalIndex) index).getProjection(projection); + return new IncrementalIndexAdapter(dataInterval, projectionSelector, bitmapFactory); + } + throw DruidException.defensive("projection inception"); + } + @Override public BitmapValues getBitmapValues(String dimension, int index) { @@ -225,7 +238,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter @Override public Metadata getMetadata() { - return index.getMetadata(); + if (index instanceof IncrementalIndex) { + return ((IncrementalIndex) index).getMetadata(); + } + throw DruidException.defensive("cannot get metadata of projection"); } static class MutableBitmapValues implements BitmapValues diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index 9d60edef044..68cc263950d 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -50,15 +50,15 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro IncrementalIndexColumnSelectorFactory( IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder rowHolder, VirtualColumns virtualColumns, - Order timeOrder, - IncrementalIndexRowHolder rowHolder + Order timeOrder ) { + this.rowSelector = rowSelector; this.virtualColumns = virtualColumns; this.timeOrder = timeOrder; this.rowHolder = rowHolder; - this.rowSelector = rowSelector; this.snapshotColumnInspector = new ColumnInspector() { @Nullable @@ -85,12 +85,8 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro final String dimension = dimensionSpec.getDimension(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrder != Order.NONE) { - return new SingleScanTimeDimensionSelector( - makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), - extractionFn, - timeOrder - ); + if (isTimeColumn(dimension) && timeOrder != Order.NONE) { + return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, timeOrder); } final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(dimensionSpec.getDimension()); @@ -122,7 +118,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro if (virtualColumns.exists(columnName)) { return virtualColumns.makeColumnValueSelector(columnName, this); } - if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { + if (isTimeColumn(columnName)) { return rowHolder; } @@ -140,6 +136,9 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro public ColumnCapabilities getColumnCapabilities(String columnName) { // Use snapshotColumnInspector instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater) + if (isTimeColumn(columnName)) { + return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, ColumnHolder.TIME_COLUMN_NAME); + } return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, columnName); } @@ -155,4 +154,9 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro { return rowHolder.get().getRowIndex(); } + + private boolean isTimeColumn(String columnName) + { + return ColumnHolder.TIME_COLUMN_NAME.equals(columnName); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java index b73a7b682a3..5f42fbaaf47 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java @@ -20,6 +20,8 @@ package org.apache.druid.segment.incremental; import com.google.common.collect.Iterables; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; @@ -28,8 +30,10 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.projections.QueryableProjection; import javax.annotation.Nullable; +import java.util.List; public class IncrementalIndexCursorFactory implements CursorFactory { @@ -77,7 +81,35 @@ public class IncrementalIndexCursorFactory implements CursorFactory @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return new IncrementalIndexCursorHolder(index, spec); + final QueryableProjection projection = index.getProjection(spec); + if (projection == null) { + return new IncrementalIndexCursorHolder(index, spec); + } else { + // currently we only have aggregated projections, so isPreAggregated is always true + return new IncrementalIndexCursorHolder( + projection.getRowSelector(), + projection.getCursorBuildSpec() + ) + { + @Override + public ColumnSelectorFactory makeSelectorFactory(CursorBuildSpec buildSpec, IncrementalIndexRowHolder currEntry) + { + return projection.wrapColumnSelectorFactory(super.makeSelectorFactory(buildSpec, currEntry)); + } + + @Override + public boolean isPreAggregated() + { + return true; + } + + @Override + public List getAggregatorsForPreAggregated() + { + return projection.getCursorBuildSpec().getAggregators(); + } + }; + } } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 72ec9116d1f..43b887ef633 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -29,7 +29,9 @@ import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.ValueMatchers; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import java.util.Iterator; import java.util.List; @@ -70,10 +72,13 @@ public class IncrementalIndexCursorHolder implements CursorHolder spec.getQueryMetrics().vectorized(false); } + IncrementalIndexRowHolder currentRow = new IncrementalIndexRowHolder(); return new IncrementalIndexCursor( rowSelector, + currentRow, + makeSelectorFactory(spec, currentRow), spec, - Cursors.getTimeOrdering(ordering) + getTimeOrder(ordering) ); } @@ -83,47 +88,50 @@ public class IncrementalIndexCursorHolder implements CursorHolder return ordering; } + public ColumnSelectorFactory makeSelectorFactory(CursorBuildSpec buildSpec, IncrementalIndexRowHolder currEntry) + { + return new IncrementalIndexColumnSelectorFactory( + rowSelector, + currEntry, + buildSpec.getVirtualColumns(), + getTimeOrder() + ); + } + static class IncrementalIndexCursor implements Cursor { - private final IncrementalIndexRowSelector rowSelector; + private final Iterable cursorIterable; private final IncrementalIndexRowHolder currEntry; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher filterMatcher; private final int maxRowIndex; + @MonotonicNonNull private Iterator baseIter; - private Iterable cursorIterable; - private boolean emptyRange; private int numAdvanced; private boolean done; IncrementalIndexCursor( - IncrementalIndexRowSelector index, + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currentRow, + ColumnSelectorFactory selectorFactory, CursorBuildSpec buildSpec, Order timeOrder ) { - currEntry = new IncrementalIndexRowHolder(); + currEntry = currentRow; + columnSelectorFactory = selectorFactory; // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 - maxRowIndex = index.getLastRowIndex(); + maxRowIndex = rowSelector.getLastRowIndex(); numAdvanced = -1; - rowSelector = index; cursorIterable = rowSelector.getFacts().timeRangeIterable( timeOrder == Order.DESCENDING, buildSpec.getInterval().getStartMillis(), buildSpec.getInterval().getEndMillis() ); - columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( - rowSelector, - buildSpec.getVirtualColumns(), - timeOrder, - currEntry - ); filterMatcher = buildSpec.getFilter() == null ? ValueMatchers.allTrue() : buildSpec.getFilter().makeMatcher(columnSelectorFactory); - emptyRange = !cursorIterable.iterator().hasNext(); - reset(); } @@ -228,7 +236,7 @@ public class IncrementalIndexCursorHolder implements CursorHolder numAdvanced++; } - done = !foundMatched && (emptyRange || !baseIter.hasNext()); + done = !foundMatched; } private boolean beyondMaxRowIndex(int rowIndex) @@ -239,4 +247,13 @@ public class IncrementalIndexCursorHolder implements CursorHolder return rowIndex > maxRowIndex; } } + + private static Order getTimeOrder(List ordering) + { + if (!ordering.isEmpty() && ColumnHolder.TIME_COLUMN_NAME.equals(ordering.get(0).getColumnName())) { + return ordering.get(0).getOrder(); + } else { + return Order.NONE; + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java index c015206a9da..d8b6d46105a 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java @@ -134,6 +134,6 @@ public class IncrementalIndexPhysicalSegmentInspector implements PhysicalSegment @Override public int getNumRows() { - return index.size(); + return index.numRows(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java index 15909ac1b52..b0d5cbb45b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java @@ -49,7 +49,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator private final RowPointer currentRowPointer; private final TimeAndDimsPointer markedRowPointer; - IncrementalIndexRowIterator(IncrementalIndex incrementalIndex) + IncrementalIndexRowIterator(IncrementalIndexRowSelector incrementalIndex) { this.timeAndDimsIterator = incrementalIndex.getFacts().persistIterable().iterator(); this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter); @@ -59,7 +59,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator } private static RowPointer makeRowPointer( - IncrementalIndex incrementalIndex, + IncrementalIndexRowSelector incrementalIndex, IncrementalIndexRowHolder rowHolder, RowNumCounter rowNumCounter ) @@ -67,9 +67,9 @@ class IncrementalIndexRowIterator implements TransformableRowIterator ColumnSelectorFactory columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( incrementalIndex, + rowHolder, VirtualColumns.EMPTY, - incrementalIndex.timePosition == 0 ? Order.ASCENDING : Order.NONE, - rowHolder + incrementalIndex.getTimePosition() == 0 ? Order.ASCENDING : Order.NONE ); ColumnValueSelector[] dimensionSelectors = incrementalIndex .getDimensions() @@ -92,7 +92,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator return new RowPointer( rowHolder, - incrementalIndex.timePosition, + incrementalIndex.getTimePosition(), dimensionSelectors, dimensionHandlers, metricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java index bafa127e881..09e877190f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java @@ -21,6 +21,7 @@ package org.apache.druid.segment.incremental; import org.apache.druid.query.OrderBy; import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.column.ColumnFormat; import javax.annotation.Nullable; import java.util.List; @@ -30,6 +31,22 @@ import java.util.List; */ public interface IncrementalIndexRowSelector extends ColumnInspector { + /** + * Returns list of {@link IncrementalIndex.DimensionDesc} for the row selector + */ + List getDimensions(); + + /** + * Returns list of dimension names for the row selector, optionally including the time column. If time is included, + * the order columns appear in this list will match {@link #getOrdering()} + */ + List getDimensionNames(boolean includeTime); + + /** + * Returns list of all metric column names for the row selector + */ + List getMetricNames(); + /** * get {@link IncrementalIndex.DimensionDesc} for the specified column, if available, which provides access to things * like {@link org.apache.druid.segment.DimensionIndexer} and {@link org.apache.druid.segment.DimensionHandler} as @@ -45,16 +62,32 @@ public interface IncrementalIndexRowSelector extends ColumnInspector @Nullable IncrementalIndex.MetricDesc getMetric(String s); + /** + * Get {@link ColumnFormat} for a dimension, metrics, or time column, or null if the column does not exist + */ + @Nullable + ColumnFormat getColumnFormat(String columnName); + /** * Ordering for the data in the facts table */ List getOrdering(); + /** + * Position of the time column in {@link #getOrdering()} + */ + int getTimePosition(); + /** * Are there any {@link IncrementalIndexRow} stored in the {@link FactsHolder}? */ boolean isEmpty(); + /** + * Number of rows in {@link FactsHolder} + */ + int numRows(); + /** * Get the {@link FactsHolder} containing all of the {@link IncrementalIndexRow} backing this selector */ diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index 8866c413dc7..fdeda126a68 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; @@ -28,6 +29,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; /** */ @@ -46,6 +49,8 @@ public class IncrementalIndexSchema private final AggregatorFactory[] metrics; private final boolean rollup; + private final List projections; + public IncrementalIndexSchema( long minTimestamp, TimestampSpec timestampSpec, @@ -53,7 +58,8 @@ public class IncrementalIndexSchema VirtualColumns virtualColumns, DimensionsSpec dimensionsSpec, AggregatorFactory[] metrics, - boolean rollup + boolean rollup, + List projections ) { this.minTimestamp = minTimestamp; @@ -63,6 +69,7 @@ public class IncrementalIndexSchema this.dimensionsSpec = dimensionsSpec; this.metrics = metrics; this.rollup = rollup; + this.projections = projections; } public long getMinTimestamp() @@ -100,6 +107,11 @@ public class IncrementalIndexSchema return rollup; } + public List getProjections() + { + return projections; + } + public static class Builder { private long minTimestamp; @@ -109,6 +121,7 @@ public class IncrementalIndexSchema private DimensionsSpec dimensionsSpec; private AggregatorFactory[] metrics; private boolean rollup; + private List projections; public Builder() { @@ -118,6 +131,7 @@ public class IncrementalIndexSchema this.dimensionsSpec = DimensionsSpec.EMPTY; this.metrics = new AggregatorFactory[]{}; this.rollup = true; + this.projections = Collections.emptyList(); } public Builder withMinTimestamp(long minTimestamp) @@ -176,6 +190,12 @@ public class IncrementalIndexSchema return this; } + public Builder withProjections(@Nullable List projections) + { + this.projections = projections == null ? Collections.emptyList() : projections; + return this; + } + public IncrementalIndexSchema build() { return new IncrementalIndexSchema( @@ -185,7 +205,8 @@ public class IncrementalIndexSchema virtualColumns, dimensionsSpec, metrics, - rollup + rollup, + projections ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnHeapAggregateProjection.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnHeapAggregateProjection.java new file mode 100644 index 00000000000..75650a1c182 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnHeapAggregateProjection.java @@ -0,0 +1,376 @@ +/* + * 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.incremental; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.EncodedKeyComponent; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnFormat; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nullable; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Projection of {@link OnheapIncrementalIndex} for {@link org.apache.druid.data.input.impl.AggregateProjectionSpec} + */ +public class OnHeapAggregateProjection implements IncrementalIndexRowSelector +{ + private final AggregateProjectionMetadata.Schema projectionSchema; + private final List dimensions; + private final int[] parentDimensionIndex; + private final AggregatorFactory[] aggregatorFactories; + private final Map dimensionsMap; + private final Map aggregatorsMap; + private final Map columnFormats; + private final FactsHolder factsHolder; + private final IncrementalIndex.InputRowHolder inputRowHolder = new IncrementalIndex.InputRowHolder(); + private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); + private final ColumnSelectorFactory virtualSelectorFactory; + private final Map aggSelectors; + private final boolean useMaxMemoryEstimates; + private final long maxBytesPerRowForAggregators; + private final long minTimestamp; + private final AtomicInteger rowCounter = new AtomicInteger(0); + private final AtomicInteger numEntries = new AtomicInteger(0); + + public OnHeapAggregateProjection( + AggregateProjectionMetadata.Schema schema, + List dimensions, + Map dimensionsMap, + int[] parentDimensionIndex, + long minTimestamp, + boolean useMaxMemoryEstimates, + long maxBytesPerRowForAggregators + ) + { + this.projectionSchema = schema; + this.dimensions = dimensions; + this.parentDimensionIndex = parentDimensionIndex; + this.dimensionsMap = dimensionsMap; + this.minTimestamp = minTimestamp; + final IncrementalIndex.IncrementalIndexRowComparator rowComparator = new IncrementalIndex.IncrementalIndexRowComparator( + projectionSchema.getTimeColumnPosition() < 0 ? dimensions.size() : projectionSchema.getTimeColumnPosition(), + dimensions + ); + this.factsHolder = new OnheapIncrementalIndex.RollupFactsHolder( + rowComparator, + dimensions, + projectionSchema.getTimeColumnPosition() == 0 + ); + this.useMaxMemoryEstimates = useMaxMemoryEstimates; + this.maxBytesPerRowForAggregators = maxBytesPerRowForAggregators; + + this.virtualSelectorFactory = new OnheapIncrementalIndex.CachingColumnSelectorFactory( + IncrementalIndex.makeColumnSelectorFactory(schema.getVirtualColumns(), inputRowHolder, null) + ); + this.aggSelectors = new LinkedHashMap<>(); + this.aggregatorsMap = new LinkedHashMap<>(); + this.aggregatorFactories = new AggregatorFactory[schema.getAggregators().length]; + this.columnFormats = new LinkedHashMap<>(); + for (IncrementalIndex.DimensionDesc dimension : dimensions) { + if (dimension.getName().equals(projectionSchema.getTimeColumnName())) { + columnFormats.put( + dimension.getName(), + new CapabilitiesBasedFormat(ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG)) + ); + } else { + columnFormats.put(dimension.getName(), dimension.getIndexer().getFormat()); + } + } + int i = 0; + for (AggregatorFactory agg : schema.getAggregators()) { + IncrementalIndex.MetricDesc metricDesc = new IncrementalIndex.MetricDesc(aggregatorsMap.size(), agg); + aggregatorsMap.put(metricDesc.getName(), metricDesc); + columnFormats.put(metricDesc.getName(), new CapabilitiesBasedFormat(metricDesc.getCapabilities())); + final ColumnSelectorFactory factory; + if (agg.getIntermediateType().is(ValueType.COMPLEX)) { + factory = new OnheapIncrementalIndex.CachingColumnSelectorFactory( + IncrementalIndex.makeColumnSelectorFactory(VirtualColumns.EMPTY, inputRowHolder, agg) + ); + } else { + factory = virtualSelectorFactory; + } + aggSelectors.put(agg.getName(), factory); + aggregatorFactories[i++] = agg; + } + } + + /** + * Add row to projection {@link #factsHolder}, updating totalSizeInBytes estimate + */ + public void addToFacts( + IncrementalIndexRow key, + InputRow inputRow, + List parseExceptionMessages, + AtomicLong totalSizeInBytes + ) + { + inputRowHolder.set(inputRow); + final Object[] projectionDims = new Object[dimensions.size()]; + for (int i = 0; i < projectionDims.length; i++) { + int parentDimIndex = parentDimensionIndex[i]; + if (parentDimIndex < 0) { + IncrementalIndex.DimensionDesc desc = dimensions.get(i); + final ColumnValueSelector virtualSelector = virtualSelectorFactory.makeColumnValueSelector(desc.getName()); + EncodedKeyComponent k = desc.getIndexer().processRowValsToUnsortedEncodedKeyComponent( + virtualSelector.getObject(), + false + ); + projectionDims[i] = k.getComponent(); + totalSizeInBytes.addAndGet(k.getEffectiveSizeBytes()); + } else { + projectionDims[i] = key.dims[parentDimensionIndex[i]]; + } + } + final IncrementalIndexRow subKey = new IncrementalIndexRow( + projectionSchema.getTimeColumnName() != null + ? projectionSchema.getGranularity().bucketStart(DateTimes.utc(key.getTimestamp())).getMillis() + : minTimestamp, + projectionDims, + dimensions + ); + + final int priorIndex = factsHolder.getPriorIndex(subKey); + + final Aggregator[] aggs; + if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { + aggs = aggregators.get(priorIndex); + long aggForProjectionSizeDelta = OnheapIncrementalIndex.doAggregate( + aggregatorFactories, + aggs, + inputRowHolder, + parseExceptionMessages, + useMaxMemoryEstimates, + false + ); + totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : aggForProjectionSizeDelta); + } else { + aggs = new Aggregator[aggregatorFactories.length]; + long aggSizeForProjectionRow = factorizeAggs(aggregatorFactories, aggs); + aggSizeForProjectionRow += OnheapIncrementalIndex.doAggregate( + aggregatorFactories, + aggs, + inputRowHolder, + parseExceptionMessages, + useMaxMemoryEstimates, + false + ); + final long estimatedSizeOfAggregators = + useMaxMemoryEstimates ? maxBytesPerRowForAggregators : aggSizeForProjectionRow; + final long projectionRowSize = key.estimateBytesInMemory() + + estimatedSizeOfAggregators + + OnheapIncrementalIndex.ROUGH_OVERHEAD_PER_MAP_ENTRY; + totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : projectionRowSize); + numEntries.incrementAndGet(); + } + final int rowIndex = rowCounter.getAndIncrement(); + aggregators.put(rowIndex, aggs); + factsHolder.putIfAbsent(subKey, rowIndex); + } + + @Override + public FactsHolder getFacts() + { + return factsHolder; + } + + @Override + public List getDimensions() + { + return dimensions; + } + + @Override + public List getMetricNames() + { + return ImmutableList.copyOf(aggregatorsMap.keySet()); + } + + @Override + public IncrementalIndex.DimensionDesc getDimension(String columnName) + { + return dimensionsMap.get(columnName); + } + + @Override + public IncrementalIndex.MetricDesc getMetric(String columnName) + { + return aggregatorsMap.get(columnName); + } + + @Override + public List getOrdering() + { + // return ordering with projection time column substituted with __time so query engines can treat it equivalently + return projectionSchema.getOrderingWithTimeColumnSubstitution(); + } + + @Override + public int getTimePosition() + { + return projectionSchema.getTimeColumnPosition(); + } + + @Override + public boolean isEmpty() + { + return rowCounter.get() == 0; + } + + @Override + public int getLastRowIndex() + { + return rowCounter.get(); + } + + @Override + public float getMetricFloatValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].getFloat(); + } + + @Override + public long getMetricLongValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].getLong(); + } + + @Override + public double getMetricDoubleValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].getDouble(); + } + + @Nullable + @Override + public Object getMetricObjectValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].get(); + } + + @Override + public boolean isNull(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].isNull(); + } + + @Override + public ColumnFormat getColumnFormat(String columnName) + { + return columnFormats.get(columnName); + } + + @Override + public int numRows() + { + return numEntries.get(); + } + + @Override + public List getDimensionNames(boolean includeTime) + { + synchronized (dimensionsMap) { + if (includeTime && projectionSchema.getTimeColumnName() != null) { + final ImmutableList.Builder listBuilder = + ImmutableList.builderWithExpectedSize(dimensionsMap.size() + 1); + int i = 0; + if (i == projectionSchema.getTimeColumnPosition()) { + listBuilder.add(projectionSchema.getTimeColumnName()); + } + for (String dimName : dimensionsMap.keySet()) { + listBuilder.add(dimName); + i++; + if (i == projectionSchema.getTimeColumnPosition()) { + listBuilder.add(projectionSchema.getTimeColumnName()); + } + } + return listBuilder.build(); + } else { + return ImmutableList.copyOf(dimensionsMap.keySet()); + } + } + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + if (ColumnHolder.TIME_COLUMN_NAME.equals(column) || Objects.equals(column, projectionSchema.getTimeColumnName())) { + return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG).setHasNulls(false); + } + if (dimensionsMap.containsKey(column)) { + return dimensionsMap.get(column).getCapabilities(); + } + if (aggregatorsMap.containsKey(column)) { + return aggregatorsMap.get(column).getCapabilities(); + } + return null; + } + + public Map getDimensionsMap() + { + return dimensionsMap; + } + + public AggregateProjectionMetadata toMetadata() + { + return new AggregateProjectionMetadata(projectionSchema, numEntries.get()); + } + + private long factorizeAggs(AggregatorFactory[] aggregatorFactories, Aggregator[] aggs) + { + long totalInitialSizeBytes = 0L; + final long aggReferenceSize = Long.BYTES; + for (int i = 0; i < aggregatorFactories.length; i++) { + final AggregatorFactory agg = aggregatorFactories[i]; + // Creates aggregators to aggregate from input into output fields + if (useMaxMemoryEstimates) { + aggs[i] = agg.factorize(aggSelectors.get(agg.getName())); + } else { + AggregatorAndSize aggregatorAndSize = agg.factorizeWithSize(aggSelectors.get(agg.getName())); + aggs[i] = aggregatorAndSize.getAggregator(); + totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); + totalInitialSizeBytes += aggReferenceSize; + } + } + return totalInitialSizeBytes; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java index 8c554e016fc..d6c67a2fcaa 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java @@ -24,9 +24,13 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; +import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; @@ -37,19 +41,27 @@ import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.Metadata; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.projections.Projections; +import org.apache.druid.segment.projections.QueryableProjection; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.Deque; @@ -58,6 +70,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.SortedSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; @@ -66,6 +79,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -86,7 +100,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex /** * overhead per {@link ConcurrentSkipListMap.Node} object in facts table */ - private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; + static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; + private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); @@ -135,6 +150,9 @@ public class OnheapIncrementalIndex extends IncrementalIndex @Nullable private String outOfRowsReason = null; + private final SortedSet aggregateProjections; + private final HashMap projections; + OnheapIncrementalIndex( IncrementalIndexSchema incrementalIndexSchema, int maxRowCount, @@ -158,6 +176,80 @@ public class OnheapIncrementalIndex extends IncrementalIndex this.maxBytesPerRowForAggregators = useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0; this.useMaxMemoryEstimates = useMaxMemoryEstimates; + + this.aggregateProjections = new ObjectAVLTreeSet<>(AggregateProjectionMetadata.COMPARATOR); + this.projections = new HashMap<>(); + initializeProjections(incrementalIndexSchema, useMaxMemoryEstimates); + } + + private void initializeProjections(IncrementalIndexSchema incrementalIndexSchema, boolean useMaxMemoryEstimates) + { + for (AggregateProjectionSpec projectionSpec : incrementalIndexSchema.getProjections()) { + // initialize them all with 0 rows + AggregateProjectionMetadata.Schema schema = projectionSpec.toMetadataSchema(); + aggregateProjections.add(new AggregateProjectionMetadata(schema, 0)); + final List descs = new ArrayList<>(); + // mapping of position in descs on the projection to position in the parent incremental index. Like the parent + // incremental index, the time (or time-like) column does not have a dimension descriptor and is specially + // handled as the timestamp of the row. Unlike the parent incremental index, an aggregating projection will + // always have its time-like column in the grouping columns list, so its position in this array specifies -1 + final int[] parentDimIndex = new int[projectionSpec.getGroupingColumns().size()]; + Arrays.fill(parentDimIndex, -1); + int i = 0; + final Map dimensionsMap = new HashMap<>(); + for (DimensionSchema dimension : projectionSpec.getGroupingColumns()) { + if (dimension.getName().equals(schema.getTimeColumnName())) { + continue; + } + final DimensionDesc parent = getDimension(dimension.getName()); + if (parent == null) { + // this dimension only exists in the child, it needs its own handler + final DimensionDesc childOnly = new DimensionDesc( + i++, + dimension.getName(), + dimension.getDimensionHandler(), + useMaxMemoryEstimates + ); + descs.add(childOnly); + dimensionsMap.put(dimension.getName(), childOnly); + } else { + if (!dimension.getColumnType().equals(parent.getCapabilities().toColumnType())) { + // special handle auto column schema, who reports type as json in schema, but indexer reports whatever + // type it has seen, which is string at this stage + boolean allowAuto = ColumnType.NESTED_DATA.equals(dimension.getColumnType()) && + parent.getIndexer() instanceof AutoTypeColumnIndexer; + InvalidInput.conditionalException( + allowAuto, + "projection[%s] contains dimension[%s] with different type[%s] than type[%s] in base table", + projectionSpec.getName(), + dimension.getName(), + dimension.getColumnType(), + parent.getCapabilities().toColumnType() + ); + } + // make a new DimensionDesc from the child, containing all of the parents stuff but with the childs position + final DimensionDesc child = new DimensionDesc( + i++, + parent.getName(), + parent.getHandler(), + parent.getIndexer() + ); + descs.add(child); + dimensionsMap.put(dimension.getName(), child); + parentDimIndex[child.getIndex()] = parent.getIndex(); + } + } + final OnHeapAggregateProjection projection = new OnHeapAggregateProjection( + projectionSpec.toMetadataSchema(), + descs, + dimensionsMap, + parentDimIndex, + incrementalIndexSchema.getMinTimestamp(), + this.useMaxMemoryEstimates, + this.maxBytesPerRowForAggregators + ); + projections.put(projectionSpec.getName(), projection); + } } /** @@ -202,6 +294,19 @@ public class OnheapIncrementalIndex extends IncrementalIndex return facts; } + @Override + public Metadata getMetadata() + { + if (aggregateProjections.isEmpty()) { + return super.getMetadata(); + } + final List projectionMetadata = projections.values() + .stream() + .map(OnHeapAggregateProjection::toMetadata) + .collect(Collectors.toList()); + return super.getMetadata().withProjections(projectionMetadata); + } + @Override protected void initAggs( final AggregatorFactory[] metrics, @@ -254,6 +359,12 @@ public class OnheapIncrementalIndex extends IncrementalIndex final List parseExceptionMessages = new ArrayList<>(); final AtomicLong totalSizeInBytes = getBytesInMemory(); + // add to projections first so if one is chosen by queries the data will always be ahead of the base table since + // rows are not added atomically to all facts holders at once + for (OnHeapAggregateProjection projection : projections.values()) { + projection.addToFacts(key, inputRowHolder.getRow(), parseExceptionMessages, totalSizeInBytes); + } + final int priorIndex = facts.getPriorIndex(key); Aggregator[] aggs; @@ -336,11 +447,10 @@ public class OnheapIncrementalIndex extends IncrementalIndex } // Creates aggregators to combine already aggregated field if (preserveExistingMetrics) { + AggregatorFactory combiningAgg = agg.getCombiningFactory(); if (useMaxMemoryEstimates) { - AggregatorFactory combiningAgg = agg.getCombiningFactory(); aggs[i + metrics.length] = combiningAgg.factorize(combiningAggSelectors.get(combiningAgg.getName())); } else { - AggregatorFactory combiningAgg = agg.getCombiningFactory(); AggregatorAndSize aggregatorAndSize = combiningAgg.factorizeWithSize(combiningAggSelectors.get(combiningAgg.getName())); aggs[i + metrics.length] = aggregatorAndSize.getAggregator(); totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); @@ -368,7 +478,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex return doAggregate(metrics, aggs, inputRowHolder, parseExceptionsHolder, useMaxMemoryEstimates, preserveExistingMetrics); } - private static long doAggregate( + static long doAggregate( AggregatorFactory[] metrics, Aggregator[] aggs, InputRowHolder inputRowHolder, @@ -431,10 +541,29 @@ public class OnheapIncrementalIndex extends IncrementalIndex } } + @Nullable + @Override + public QueryableProjection getProjection(CursorBuildSpec buildSpec) + { + return Projections.findMatchingProjection( + buildSpec, + aggregateProjections, + (specName, columnName) -> projections.get(specName).getDimensionsMap().containsKey(columnName) + || getColumnCapabilities(columnName) == null, + projections::get + ); + } + + @Override + public IncrementalIndexRowSelector getProjection(String name) + { + return projections.get(name); + } + @Override public boolean canAppendRow() { - final boolean countCheck = size() < maxRowCount; + final boolean countCheck = numRows() < maxRowCount; // if maxBytesInMemory = -1, then ignore sizeCheck final boolean sizeCheck = maxBytesInMemory <= 0 || getBytesInMemory().get() < maxBytesInMemory; final boolean canAdd = countCheck && sizeCheck; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java index bfccf74c17b..4569df39ef6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java @@ -100,6 +100,12 @@ public final class DictionaryIdLookup implements Closeable this.arrayDictionaryWriter = arrayDictionaryWriter; } + public int[] getArrayValue(int id) + { + ensureArrayDictionaryLoaded(); + return arrayDictionary.get(id - arrayOffset()); + } + @Nullable public Object getDictionaryValue(int id) { @@ -401,4 +407,28 @@ public final class DictionaryIdLookup implements Closeable } }; } + + public int getStringCardinality() + { + ensureStringDictionaryLoaded(); + return stringDictionary == null ? 0 : stringDictionary.size(); + } + + public int getLongCardinality() + { + ensureLongDictionaryLoaded(); + return longDictionary == null ? 0 : longDictionary.size(); + } + + public int getDoubleCardinality() + { + ensureDoubleDictionaryLoaded(); + return doubleDictionary == null ? 0 : doubleDictionary.size(); + } + + public int getArrayCardinality() + { + ensureArrayDictionaryLoaded(); + return arrayDictionary == null ? 0 : arrayDictionary.size(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java index e630b64d7d6..50bca997735 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java @@ -69,7 +69,9 @@ public abstract class NestedCommonFormatColumnSerializer implements GenericColum public abstract String getColumnName(); - public abstract DictionaryIdLookup getGlobalLookup(); + public abstract DictionaryIdLookup getDictionaryIdLookup(); + + public abstract void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup); public abstract boolean hasNulls(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java index 7dea3b664fd..6a3405e58fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java @@ -156,6 +156,7 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ private Map> fieldWriters; private int rowCount = 0; private boolean closedForWrite = false; + private boolean writeDictionary = true; private boolean dictionarySerialized = false; private ByteBuffer columnNameBytes = null; @@ -180,11 +181,19 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ } @Override - public DictionaryIdLookup getGlobalLookup() + public DictionaryIdLookup getDictionaryIdLookup() { return globalDictionaryIdLookup; } + @Override + public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup) + { + this.globalDictionaryIdLookup = dictionaryIdLookup; + this.writeDictionary = false; + this.dictionarySerialized = true; + } + @Override public boolean hasNulls() { @@ -421,35 +430,50 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ ) throws IOException { Preconditions.checkState(closedForWrite, "Not closed yet!"); - Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + if (writeDictionary) { + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + } writeV0Header(channel, columnNameBytes); fieldsWriter.writeTo(channel, smoosher); fieldsInfoWriter.writeTo(channel, smoosher); - - if (globalDictionaryIdLookup.getStringBufferMapper() != null) { - SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper(); - for (String internalName : fileMapper.getInternalFilenames()) { - smoosher.add(internalName, fileMapper.mapFile(internalName)); + if (writeDictionary) { + if (globalDictionaryIdLookup.getStringBufferMapper() != null) { + SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper(); + for (String internalName : fileMapper.getInternalFilenames()) { + smoosher.add(internalName, fileMapper.mapFile(internalName)); + } + } else { + writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); + } + if (globalDictionaryIdLookup.getLongBuffer() != null) { + writeInternal( + smoosher, + globalDictionaryIdLookup.getLongBuffer(), + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); + } + if (globalDictionaryIdLookup.getDoubleBuffer() != null) { + writeInternal( + smoosher, + globalDictionaryIdLookup.getDoubleBuffer(), + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); + } + if (globalDictionaryIdLookup.getArrayBuffer() != null) { + writeInternal( + smoosher, + globalDictionaryIdLookup.getArrayBuffer(), + ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); } - } else { - writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); - } - if (globalDictionaryIdLookup.getLongBuffer() != null) { - writeInternal(smoosher, globalDictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } - if (globalDictionaryIdLookup.getDoubleBuffer() != null) { - writeInternal(smoosher, globalDictionaryIdLookup.getDoubleBuffer(), ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } - if (globalDictionaryIdLookup.getArrayBuffer() != null) { - writeInternal(smoosher, globalDictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); } writeInternal(smoosher, rawWriter, RAW_FILE_NAME); if (!nullRowsBitmap.isEmpty()) { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java index 7c5e722ca67..7b5678be074 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java @@ -55,7 +55,8 @@ public class NestedDataColumnSupplier implements Supplier> doubleDictionarySupplier = FixedIndexed.read( - doubleDictionaryBuffer, - ColumnType.DOUBLE.getStrategy(), - byteOrder, - Double.BYTES - ); + final Supplier> doubleDictionarySupplier; + if (parent != null) { + doubleDictionarySupplier = parent.doubleDictionarySupplier; + } else { + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + } final Supplier doubles = CompressedColumnarDoublesSuppliers.fromByteBuffer( doublesValueColumn, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java index 718981ae19a..050848f6503 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java @@ -142,4 +142,13 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); } } + + @Override + public int getCardinality() + { + if (writeDictionary) { + return dictionaryWriter.getCardinality(); + } + return dictionaryIdLookup.getDoubleCardinality(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java index 66527530336..a8d1fa057d8 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java @@ -90,7 +90,8 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier> longDictionarySupplier = FixedIndexed.read( - longDictionaryBuffer, - ColumnType.LONG.getStrategy(), - byteOrder, - Long.BYTES - ); + + final Supplier> longDictionarySupplier; + if (parent != null) { + longDictionarySupplier = parent.longDictionarySupplier; + } else { + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + } final Supplier longs = CompressedColumnarLongsSupplier.fromByteBuffer( longsValueColumn, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java index 00d7c16813d..89408b20388 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java @@ -142,4 +142,13 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); } } + + @Override + public int getCardinality() + { + if (writeDictionary) { + return dictionaryWriter.getCardinality(); + } + return dictionaryIdLookup.getLongCardinality(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java index 1010003a611..dc18970ad79 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java @@ -62,6 +62,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested protected ByteBuffer columnNameBytes = null; protected boolean hasNulls; + protected boolean writeDictionary = true; public ScalarNestedCommonFormatColumnSerializer( @@ -98,6 +99,8 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested protected abstract void writeDictionaryFile(FileSmoosher smoosher) throws IOException; + public abstract int getCardinality(); + @Override public String getColumnName() { @@ -105,11 +108,19 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested } @Override - public DictionaryIdLookup getGlobalLookup() + public DictionaryIdLookup getDictionaryIdLookup() { return dictionaryIdLookup; } + @Override + public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup) + { + this.dictionaryIdLookup = dictionaryIdLookup; + this.writeDictionary = false; + this.dictionarySerialized = true; + } + @Override public boolean hasNulls() { @@ -166,7 +177,9 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested ) throws IOException { Preconditions.checkState(closedForWrite, "Not closed yet!"); - Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + if (writeDictionary) { + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + } // write out compressed dictionaryId int column and bitmap indexes by iterating intermediate value column // the intermediate value column should be replaced someday by a cooler compressed int column writer that allows @@ -184,7 +197,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested name, segmentWriteOutMedium, filenameBase, - dictionaryWriter.getCardinality(), + getCardinality(), compressionToUse, segmentWriteOutMedium.getCloser() ); @@ -198,7 +211,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested bitmapIndexWriter.open(); bitmapIndexWriter.setObjectsNotSorted(); final MutableBitmap[] bitmaps; - bitmaps = new MutableBitmap[dictionaryWriter.getCardinality()]; + bitmaps = new MutableBitmap[getCardinality()]; for (int i = 0; i < bitmaps.length; i++) { bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } @@ -220,7 +233,9 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested } writeV0Header(channel, columnNameBytes); - writeDictionaryFile(smoosher); + if (writeDictionary) { + writeDictionaryFile(smoosher); + } writeInternal(smoosher, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME); writeValueColumn(smoosher); writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java index c4850905b7f..7f1111c2e8b 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java @@ -49,7 +49,8 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier> dictionarySupplier; - final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME - ); + if (parent != null) { + dictionarySupplier = parent.dictionarySupplier; + } else { + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME + ); + + dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + mapper, + stringDictionaryBuffer, + byteOrder + ); + } - dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( - mapper, - stringDictionaryBuffer, - byteOrder - ); final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, @@ -106,6 +112,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier> dictionarySupplier; private final Supplier encodedColumnSupplier; private final GenericIndexed valueIndexes; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java index 380da1ea982..230a9433cdc 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java @@ -131,4 +131,13 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); } } + + @Override + public int getCardinality() + { + if (writeDictionary) { + return dictionaryWriter.getCardinality(); + } + return dictionaryIdLookup.getStringCardinality(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java index 3cd9e4f7308..23555b2ea2d 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java @@ -68,7 +68,8 @@ public class VariantColumnAndIndexSupplier implements Supplier arrayDictionarySupplier; final Supplier> arrayElementDictionarySupplier; - final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME - ); + if (parent != null) { + stringDictionarySupplier = parent.stringDictionarySupplier; + longDictionarySupplier = parent.longDictionarySupplier; + doubleDictionarySupplier = parent.doubleDictionarySupplier; + arrayDictionarySupplier = parent.arrayDictionarySupplier; + arrayElementDictionarySupplier = parent.arrayElementDictionarySupplier; + } else { + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME + ); + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME + ); + final ByteBuffer arrayElementDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME + ); + + stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + mapper, + stringDictionaryBuffer, + byteOrder + ); + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + byteOrder + ); + arrayElementDictionarySupplier = FixedIndexed.read( + arrayElementDictionaryBuffer, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, + byteOrder, + Integer.BYTES + ); + } - stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( - mapper, - stringDictionaryBuffer, - byteOrder - ); final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, @@ -115,21 +167,6 @@ public class VariantColumnAndIndexSupplier implements Supplier= arrayBaseId) { - int[] array = arrayDictionaryWriter.get(dictId - arrayBaseId); + int[] array = dictionaryIdLookup.getArrayValue(dictId); for (int elementId : array) { arrayElements.computeIfAbsent( elementId, @@ -396,11 +407,13 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer ); bitmaps[i] = null; // Reclaim memory } - for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { - arrayElementDictionaryWriter.write(arrayElement.getIntKey()); - arrayElementIndexWriter.write( - indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) - ); + if (writeDictionary) { + for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { + arrayElementDictionaryWriter.write(arrayElement.getIntKey()); + arrayElementIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) + ); + } } writeV0Header(channel, columnNameBytes); @@ -408,31 +421,37 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte})); } - if (dictionaryIdLookup.getStringBufferMapper() != null) { - SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper(); - for (String internalName : fileMapper.getInternalFilenames()) { - smoosher.add(internalName, fileMapper.mapFile(internalName)); + if (writeDictionary) { + if (dictionaryIdLookup.getStringBufferMapper() != null) { + SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper(); + for (String internalName : fileMapper.getInternalFilenames()) { + smoosher.add(internalName, fileMapper.mapFile(internalName)); + } + } else { + writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); + } + if (dictionaryIdLookup.getLongBuffer() != null) { + writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); + } else { + writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); + } + if (dictionaryIdLookup.getDoubleBuffer() != null) { + writeInternal( + smoosher, + dictionaryIdLookup.getDoubleBuffer(), + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); + } + if (dictionaryIdLookup.getArrayBuffer() != null) { + writeInternal(smoosher, dictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); + } else { + writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); } - } else { - writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); - } - if (dictionaryIdLookup.getLongBuffer() != null) { - writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } - if (dictionaryIdLookup.getDoubleBuffer() != null) { - writeInternal(smoosher, dictionaryIdLookup.getDoubleBuffer(), ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } - if (dictionaryIdLookup.getArrayBuffer() != null) { - writeInternal(smoosher, dictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); - } - writeInternal(smoosher, arrayElementDictionaryWriter, ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME); + writeInternal(smoosher, arrayElementDictionaryWriter, ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME); + } writeInternal(smoosher, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME); writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME); writeInternal(smoosher, arrayElementIndexWriter, ColumnSerializerUtils.ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME); diff --git a/processing/src/main/java/org/apache/druid/segment/projections/Projections.java b/processing/src/main/java/org/apache/druid/segment/projections/Projections.java new file mode 100644 index 00000000000..c38a875226f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/projections/Projections.java @@ -0,0 +1,260 @@ +/* + * 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.projections; + +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.ConstantExprEvalSelector; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.NumericColumn; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.vector.ConstantVectorSelectors; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorValueSelector; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.function.Function; + +public class Projections +{ + public static Supplier makeConstantTimeSupplier(int numRows, long constant) + { + return Suppliers.memoize( + () -> new ColumnBuilder().setNumericColumnSupplier(() -> new ConstantTimeColumn(numRows, constant)) + .setColumnFormat( + new CapabilitiesBasedFormat( + ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG) + ) + ) + .setType(ColumnType.LONG) + .setHasNulls(false) + .build() + ); + } + + @Nullable + public static QueryableProjection findMatchingProjection( + CursorBuildSpec cursorBuildSpec, + SortedSet projections, + PhysicalColumnChecker physicalChecker, + Function getRowSelector + ) + { + if (cursorBuildSpec.getQueryContext().getBoolean(QueryContexts.NO_PROJECTIONS, false)) { + return null; + } + final String name = cursorBuildSpec.getQueryContext().getString(QueryContexts.USE_PROJECTION); + + if (cursorBuildSpec.isAggregate()) { + for (AggregateProjectionMetadata spec : projections) { + if (name != null && !name.equals(spec.getSchema().getName())) { + continue; + } + ProjectionMatch match = spec.getSchema().matches(cursorBuildSpec, physicalChecker); + if (match != null) { + if (cursorBuildSpec.getQueryMetrics() != null) { + cursorBuildSpec.getQueryMetrics().projection(spec.getSchema().getName()); + } + return new QueryableProjection<>( + match.getCursorBuildSpec(), + match.getRemapColumns(), + getRowSelector.apply(spec.getSchema().getName()) + ); + } + } + } + if (name != null) { + throw InvalidInput.exception("Projection[%s] specified, but does not satisfy query", name); + } + if (cursorBuildSpec.getQueryContext().getBoolean(QueryContexts.FORCE_PROJECTION, false)) { + throw InvalidInput.exception("Force projections specified, but none satisfy query"); + } + return null; + } + + public static String getProjectionSmooshV9FileName(AggregateProjectionMetadata projectionSpec, String columnName) + { + return getProjectionSmooshV9Prefix(projectionSpec) + columnName; + } + + public static String getProjectionSmooshV9Prefix(AggregateProjectionMetadata projectionSpec) + { + return projectionSpec.getSchema().getName() + "/"; + } + + /** + * Returns true if column is defined in {@link AggregateProjectionSpec#getGroupingColumns()} OR if the column does not + * exist in the base table. Part of determining if a projection can be used for a given {@link CursorBuildSpec}, + * + * @see AggregateProjectionMetadata.Schema#matches(CursorBuildSpec, PhysicalColumnChecker) + */ + @FunctionalInterface + public interface PhysicalColumnChecker + { + boolean check(String projectionName, String columnName); + } + + public static final class ProjectionMatch + { + private final CursorBuildSpec cursorBuildSpec; + private final Map remapColumns; + + public ProjectionMatch(CursorBuildSpec cursorBuildSpec, Map remapColumns) + { + this.cursorBuildSpec = cursorBuildSpec; + this.remapColumns = remapColumns; + } + + public CursorBuildSpec getCursorBuildSpec() + { + return cursorBuildSpec; + } + + public Map getRemapColumns() + { + return remapColumns; + } + } + + public static final class ProjectionMatchBuilder + { + private final Set referencedVirtualColumns; + private final Map remapColumns; + private final List combiningFactories; + + public ProjectionMatchBuilder() + { + this.referencedVirtualColumns = new HashSet<>(); + this.remapColumns = new HashMap<>(); + this.combiningFactories = new ArrayList<>(); + } + + /** + * Map a query column name to a projection column name + */ + public ProjectionMatchBuilder remapColumn(String queryColumn, String projectionColumn) + { + remapColumns.put(queryColumn, projectionColumn); + return this; + } + + /** + * Add a query virtual column that can use projection physical columns as inputs to the match builder, which will + * later be added to {@link ProjectionMatch#getCursorBuildSpec()} if the projection matches + */ + public ProjectionMatchBuilder addReferenceedVirtualColumn(VirtualColumn virtualColumn) + { + referencedVirtualColumns.add(virtualColumn); + return this; + } + + /** + * Add a query {@link AggregatorFactory#substituteCombiningFactory(AggregatorFactory)} which can combine the inputs + * of a selector created by a projection {@link AggregatorFactory} + * + */ + public ProjectionMatchBuilder addPreAggregatedAggregator(AggregatorFactory aggregator) + { + combiningFactories.add(aggregator); + return this; + } + + public ProjectionMatch build(CursorBuildSpec queryCursorBuildSpec) + { + return new ProjectionMatch( + CursorBuildSpec.builder(queryCursorBuildSpec) + .setVirtualColumns(VirtualColumns.fromIterable(referencedVirtualColumns)) + .setAggregators(combiningFactories) + .build(), + remapColumns + ); + } + } + + private static class ConstantTimeColumn implements NumericColumn + { + private final int numRows; + private final long constant; + + private ConstantTimeColumn(int numRows, long constant) + { + this.numRows = numRows; + this.constant = constant; + } + + @Override + public int length() + { + return numRows; + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + return constant; + } + + @Override + public void close() + { + // nothing to close + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return new ConstantExprEvalSelector(ExprEval.ofLong(constant)); + } + + @Override + public VectorValueSelector makeVectorValueSelector(ReadableVectorOffset offset) + { + return ConstantVectorSelectors.vectorValueSelector(offset, constant); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/projections/QueryableProjection.java b/processing/src/main/java/org/apache/druid/segment/projections/QueryableProjection.java new file mode 100644 index 00000000000..b32ea829f34 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/projections/QueryableProjection.java @@ -0,0 +1,119 @@ +/* + * 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.projections; + +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.RemapColumnSelectorFactory; +import org.apache.druid.segment.vector.RemapVectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import java.util.Map; + +/** + * Represents a projection of some base table available to use to build a {@link org.apache.druid.segment.CursorHolder} + * by a {@link org.apache.druid.segment.CursorFactory}. + *

+ * Projections are a type of invisible materialized view stored inside of a {@link org.apache.druid.segment.Segment} + * which can be automatically used if they match the {@link CursorBuildSpec} argument passed to + * {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)}. + *

+ * In the most basic sense, a projection consists of: + * - the actual underlying projection rows ({@link #rowSelector}) + * - a mapping of {@link CursorBuildSpec} columns to underlying projection columns ({@link #remapColumns}) + * - and a modified {@link CursorBuildSpec} ({@link #cursorBuildSpec}) + *

+ * The {@link #getRowSelector()} and {@link #getCursorBuildSpec()} methods can be used by a + * {@link org.apache.druid.segment.CursorFactory} to build a {@link org.apache.druid.segment.CursorHolder} for the + * projection instead of the base table, and {@link #wrapColumnSelectorFactory(ColumnSelectorFactory)} and + * {@link #wrapVectorColumnSelectorFactory(VectorColumnSelectorFactory)} can be used to decorate the selector factories + * constructed by that {@link org.apache.druid.segment.CursorHolder} whenever it builds a + * {@link org.apache.druid.segment.Cursor} or {@link org.apache.druid.segment.vector.VectorCursor} to ensure that all + * the selectors needed to satisfy the original {@link CursorBuildSpec} are available at the correct names. + * + * @see org.apache.druid.segment.QueryableIndexCursorFactory#makeCursorHolder(CursorBuildSpec) + * @see org.apache.druid.segment.incremental.IncrementalIndexCursorFactory#makeCursorHolder(CursorBuildSpec) + */ +public class QueryableProjection +{ + private final CursorBuildSpec cursorBuildSpec; + private final Map remapColumns; + private final T rowSelector; + + public QueryableProjection( + CursorBuildSpec cursorBuildSpec, + Map remapColumns, + T rowSelector + ) + { + this.cursorBuildSpec = cursorBuildSpec; + this.remapColumns = remapColumns; + this.rowSelector = rowSelector; + } + + /** + * The original {@link CursorBuildSpec} of a query can be modified if a projection matches the query, such as removing + * virtual columns which have already been pre-computed. + */ + public CursorBuildSpec getCursorBuildSpec() + { + return cursorBuildSpec; + } + + /** + * The projection can contain pre-computed virtual columns or pre-aggregated aggregation columns. At query time, + * these are remapped to match the desired names for all equivalent components of the {@link CursorBuildSpec}. + *

+ * For example, if the original {@link CursorBuildSpec} has a sum aggregator named 'sum_x' which takes a field 'x' + * as input, and an equivalent sum aggregation exists on the projection with the name 'xsum' built from the base table + * column 'x', the wrapped column selector factory will make 'xsum' available as 'sum_x', allowing the query to + * use the combining aggregator instead of processing the base table for column 'x'. + */ + public ColumnSelectorFactory wrapColumnSelectorFactory(ColumnSelectorFactory selectorFactory) + { + return new RemapColumnSelectorFactory( + selectorFactory, + remapColumns + ); + } + + /** + * The projection can contain pre-computed virtual columns or pre-aggregated aggregation columns. At query time, + * these are remapped to match the desired names for all equivalent components of the {@link CursorBuildSpec} + *

+ * For example, if the original {@link CursorBuildSpec} has a sum aggregator named 'sum_x' which takes a field 'x' + * as input, and an equivalent sum aggregation exists on the projection with the name 'xsum' built from the base table + * column 'x', the wrapped column selector factory will make 'xsum' available as 'sum_x', allowing the query to + * use the combining aggregator instead of processing the base table for column 'x'. + */ + public VectorColumnSelectorFactory wrapVectorColumnSelectorFactory(VectorColumnSelectorFactory selectorFactory) + { + return new RemapVectorColumnSelectorFactory(selectorFactory, remapColumns); + } + + /** + * Backing storage for the rows of the projection as is appropriate for the type of + * {@link org.apache.druid.segment.CursorFactory} + */ + public T getRowSelector() + { + return rowSelector; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java index 19008a36526..f48b46fbbdd 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -54,6 +55,11 @@ public interface ColumnPartSerde interface Deserializer { - void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); + void read( + ByteBuffer buffer, + ColumnBuilder builder, + ColumnConfig columnConfig, + @Nullable ColumnHolder parent + ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java index 651d2c6a5d9..f8220d40046 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java @@ -77,7 +77,7 @@ public class ComplexColumnPartSerde implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { // we don't currently know if complex column can have nulls (or can be multi-valued, but not making that change // since it isn't supported anywhere in the query engines) // longer term this needs to be captured by making the serde provide this information, and then this should diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 41aef13f576..cf9b7c70d56 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -29,8 +29,10 @@ import org.apache.druid.collections.spatial.ImmutableRTree; import org.apache.druid.io.Channels; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerde; @@ -292,7 +294,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde return new Deserializer() { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read( + ByteBuffer buffer, + ColumnBuilder builder, + ColumnConfig columnConfig, + @Nullable ColumnHolder parent + ) { final VERSION rVersion = VERSION.fromByte(buffer.get()); final int rFlags; @@ -309,12 +316,17 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde builder.setType(ValueType.STRING); - final Supplier> dictionarySupplier = - StringEncodingStrategies.getStringDictionarySupplier( - builder.getFileMapper(), - buffer, - byteOrder - ); + final Supplier> dictionarySupplier; + if (parent != null) { + final Supplier parentSupplier = parent.getColumnSupplier(); + dictionarySupplier = ((StringUtf8DictionaryEncodedColumnSupplier) parentSupplier).getDictionary(); + } else { + dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + builder.getFileMapper(), + buffer, + byteOrder + ); + } final WritableSupplier rSingleValuedColumn; final WritableSupplier rMultiValuedColumn; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java index 864e893fd19..012b3ed77b0 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java @@ -96,7 +96,7 @@ public class DoubleNumericColumnPartSerde implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( buffer, byteOrder diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java index 5eda4f84ae9..0fa42f2187a 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java @@ -143,7 +143,7 @@ public class DoubleNumericColumnPartSerdeV2 implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { int offset = buffer.getInt(); int initialPos = buffer.position(); final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( diff --git a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java index 02ac221be32..441f774c7d1 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java @@ -22,13 +22,10 @@ package org.apache.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.column.ColumnBuilder; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.CompressedColumnarFloatsSupplier; import javax.annotation.Nullable; -import java.nio.ByteBuffer; import java.nio.ByteOrder; /** @@ -99,23 +96,18 @@ public class FloatNumericColumnPartSerde implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return new Deserializer() - { - @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) - { - final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( - buffer, - byteOrder - ); - FloatNumericColumnSupplier columnSupplier = new FloatNumericColumnSupplier( - column, - IndexIO.LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap() - ); - builder.setType(ValueType.FLOAT) - .setHasMultipleValues(false) - .setNumericColumnSupplier(columnSupplier); - } + return (buffer, builder, columnConfig, parent) -> { + final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( + buffer, + byteOrder + ); + FloatNumericColumnSupplier columnSupplier = new FloatNumericColumnSupplier( + column, + IndexIO.LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap() + ); + builder.setType(ValueType.FLOAT) + .setHasMultipleValues(false) + .setNumericColumnSupplier(columnSupplier); }; } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java index 8e79bc7a3fa..457612c4f81 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java @@ -141,7 +141,7 @@ public class FloatNumericColumnPartSerdeV2 implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { int offset = buffer.getInt(); int initialPos = buffer.position(); final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( diff --git a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java index 3ad77a45e62..fa94b91ecd4 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java @@ -96,7 +96,7 @@ public class LongNumericColumnPartSerde implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( buffer, byteOrder diff --git a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java index 7f145b36a41..ac81a225de4 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java @@ -143,7 +143,7 @@ public class LongNumericColumnPartSerdeV2 implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { int offset = buffer.getInt(); int initialPos = buffer.position(); final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java index e09855cdab2..d923de51d09 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; @@ -189,13 +190,14 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde private class StringColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( byteOrder, bitmapSerdeFactory, buffer, - builder + builder, + parent == null ? null : (ScalarStringColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); capabilitiesBuilder.setDictionaryEncoded(true); @@ -205,21 +207,26 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde builder.setHasNulls(hasNulls); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue(), + enforceLogicalType + )); } } private class LongColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( byteOrder, bitmapSerdeFactory, buffer, builder, - columnConfig + columnConfig, + parent == null ? null : (ScalarLongColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn @@ -228,21 +235,26 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde builder.setHasNulls(hasNulls); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue(), + enforceLogicalType + )); } } private class DoubleColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( byteOrder, bitmapSerdeFactory, buffer, builder, - columnConfig + columnConfig, + parent == null ? null : (ScalarDoubleColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn @@ -251,21 +263,26 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde builder.setHasNulls(hasNulls); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue(), + enforceLogicalType + )); } } private class VariantColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read( logicalType, byteOrder, bitmapSerdeFactory, buffer, - builder + builder, + parent == null ? null : (VariantColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // if we are a mixed type, don't call ourself dictionary encoded for now so we don't end up doing the wrong thing @@ -291,7 +308,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde private class NestedColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( logicalType, @@ -300,7 +317,8 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde builder, columnConfig, bitmapSerdeFactory, - byteOrder + byteOrder, + parent == null ? null : (NestedDataColumnSupplier) parent.getColumnSupplier() ); ColumnType simpleType = supplier.getLogicalType(); ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java index ea8af5e8541..b3489ef0179 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java @@ -105,7 +105,7 @@ public class NullColumnPartSerde implements ColumnPartSerde @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { builder.setHasMultipleValues(false) .setHasNulls(true) // this is a bit sneaky, we set supplier to null here to act like a null column instead of a column diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java index c0a8a0885e4..33de1869e27 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java @@ -50,6 +50,11 @@ public class StringUtf8DictionaryEncodedColumnSupplier getDictionary() + { + return utf8Dictionary; + } + @Override public DictionaryEncodedColumn get() { diff --git a/processing/src/main/java/org/apache/druid/segment/vector/RemapVectorColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/vector/RemapVectorColumnSelectorFactory.java new file mode 100644 index 00000000000..0c6bd7b05ec --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/vector/RemapVectorColumnSelectorFactory.java @@ -0,0 +1,102 @@ +/* + * 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.vector; + +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.DeferExpressionDimensions; +import org.apache.druid.query.groupby.epinephelinae.vector.GroupByVectorColumnSelector; +import org.apache.druid.segment.column.ColumnCapabilities; + +import javax.annotation.Nullable; +import java.util.Map; + +public class RemapVectorColumnSelectorFactory implements VectorColumnSelectorFactory +{ + private final VectorColumnSelectorFactory delegate; + private final Map remap; + + public RemapVectorColumnSelectorFactory(VectorColumnSelectorFactory delegate, Map remap) + { + this.delegate = delegate; + this.remap = remap; + } + + @Override + public ReadableVectorInspector getReadableVectorInspector() + { + return delegate.getReadableVectorInspector(); + } + + @Override + public int getMaxVectorSize() + { + return delegate.getMaxVectorSize(); + } + + @Override + public SingleValueDimensionVectorSelector makeSingleValueDimensionSelector(DimensionSpec dimensionSpec) + { + DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension())); + return delegate.makeSingleValueDimensionSelector(remapDimensionSpec); + } + + @Override + public MultiValueDimensionVectorSelector makeMultiValueDimensionSelector(DimensionSpec dimensionSpec) + { + DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension())); + return delegate.makeMultiValueDimensionSelector(remapDimensionSpec); + } + + @Override + public VectorValueSelector makeValueSelector(String column) + { + return delegate.makeValueSelector(remap.getOrDefault(column, column)); + } + + @Override + public VectorObjectSelector makeObjectSelector(String column) + { + return delegate.makeObjectSelector(remap.getOrDefault(column, column)); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(remap.getOrDefault(column, column)); + } + + @Nullable + @Override + public ExpressionType getType(String name) + { + return delegate.getType(remap.getOrDefault(name, name)); + } + + @Override + public GroupByVectorColumnSelector makeGroupByVectorColumnSelector( + String column, + DeferExpressionDimensions deferExpressionDimensions + ) + { + return delegate.makeGroupByVectorColumnSelector(remap.getOrDefault(column, column), deferExpressionDimensions); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java index 8bb62128f9d..91c9c9057f4 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java @@ -63,6 +63,7 @@ public class ExpressionVirtualColumn implements VirtualColumn private final String name; private final Expression expression; private final Supplier parsedExpression; + private final Supplier expressionAnalysis; private final Supplier cacheKey; /** @@ -126,6 +127,7 @@ public class ExpressionVirtualColumn implements VirtualColumn this.name = Preconditions.checkNotNull(name, "name"); this.expression = new Expression(Preconditions.checkNotNull(expression, "expression"), outputType); this.parsedExpression = parsedExpression; + this.expressionAnalysis = Suppliers.memoize(parsedExpression.get()::analyzeInputs); this.cacheKey = makeCacheKeySupplier(); } @@ -332,7 +334,7 @@ public class ExpressionVirtualColumn implements VirtualColumn @Override public List requiredColumns() { - return parsedExpression.get().analyzeInputs().getRequiredBindingsList(); + return expressionAnalysis.get().getRequiredBindingsList(); } @Override diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/AggregateProjectionSpecTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/AggregateProjectionSpecTest.java new file mode 100644 index 00000000000..60895ceefb0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/data/input/impl/AggregateProjectionSpecTest.java @@ -0,0 +1,105 @@ +/* + * 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.data.input.impl; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +public class AggregateProjectionSpecTest extends InitializedNullHandlingTest +{ + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + AggregateProjectionSpec spec = new AggregateProjectionSpec( + "some_projection", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "time") + ), + Arrays.asList( + new StringDimensionSchema("a"), + new LongDimensionSchema("b"), + new LongDimensionSchema("time"), + new FloatDimensionSchema("c"), + new DoubleDimensionSchema("d") + ), + new AggregatorFactory[] { + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("e", "e") + } + ); + Assert.assertEquals(spec, JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(spec), AggregateProjectionSpec.class)); + } + + @Test + public void testInvalidGrouping() + { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new AggregateProjectionSpec( + "other_projection", + null, + null, + null + ) + ); + Assert.assertEquals("groupingColumns must not be null or empty", t.getMessage()); + + t = Assert.assertThrows( + DruidException.class, + () -> new AggregateProjectionSpec( + "other_projection", + null, + Collections.emptyList(), + null + ) + ); + Assert.assertEquals("groupingColumns must not be null or empty", t.getMessage()); + } + + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(AggregateProjectionSpec.class) + .usingGetClass() + .withPrefabValues( + DimensionSchema.class, + new StringDimensionSchema("a"), + new DoubleDimensionSchema("d") + ) + .withIgnoredFields("timeColumnName") + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java index c916a458564..acc2572664b 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java @@ -90,7 +90,7 @@ public class FrameFileTest extends InitializedNullHandlingTest @Override int getRowCount() { - return TestIndex.getNoRollupIncrementalTestIndex().size(); + return TestIndex.getNoRollupIncrementalTestIndex().numRows(); } }, MMAP { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java index 4ed2c610525..000de90a83d 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java @@ -145,11 +145,11 @@ public class FrameProcessorExecutorTest final ListenableFuture blasterFuture = exec.runFully(blaster, null); final ListenableFuture muxerFuture = exec.runFully(muxer, null); - Assert.assertEquals(index.size(), (long) blasterFuture.get()); - Assert.assertEquals(index.size() * 2, (long) muxerFuture.get()); + Assert.assertEquals(index.numRows(), (long) blasterFuture.get()); + Assert.assertEquals(index.numRows() * 2, (long) muxerFuture.get()); Assert.assertEquals( - index.size() * 2, + index.numRows() * 2, FrameTestUtil.readRowsFromFrameChannel( new ReadableFileFrameChannel(FrameFile.open(outFile, null)), FrameReader.create(cursorFactory.getRowSignature()) diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java index 41834d448fb..a885b2f6df1 100644 --- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java +++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java @@ -33,8 +33,11 @@ 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.granularity.GranularityType; import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; @@ -55,7 +58,7 @@ import java.util.TimeZone; /** */ -public class QueryGranularityTest +public class QueryGranularityTest extends InitializedNullHandlingTest { @Test public void testIterableNone() @@ -1056,6 +1059,54 @@ public class QueryGranularityTest Assert.assertEquals("timestamp_floor(__time,'PT15M')", column.getExpression()); } + @Test + public void testFromVirtualColumn() + { + ExpressionVirtualColumn hourly = Granularities.toVirtualColumn(Granularities.HOUR, "v0"); + ExpressionVirtualColumn day = Granularities.toVirtualColumn(Granularities.DAY, "v0"); + ExpressionVirtualColumn hourlyNonstandardTime = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(__gran, 'PT1H')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn ceilHour = new ExpressionVirtualColumn( + "v0", + "timestamp_ceil(__time, 'PT1M')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn floorWithExpression = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(timestamp_parse(timestamp,null,'UTC'), 'PT1M')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn floorWithTimezone = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(__time, 'PT1M', null,'America/Los_Angeles')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn floorWithOriginTimezone = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(__time, 'PT1M', '2012-01-02T05:00:00.000-08:00','America/Los_Angeles')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + Assert.assertEquals(Granularities.HOUR, Granularities.fromVirtualColumn(hourly)); + Assert.assertEquals(Granularities.DAY, Granularities.fromVirtualColumn(day)); + Assert.assertEquals(Granularities.HOUR, Granularities.fromVirtualColumn(hourlyNonstandardTime)); + Assert.assertNull(Granularities.fromVirtualColumn(ceilHour)); + Assert.assertNull(Granularities.fromVirtualColumn(floorWithExpression)); + final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00"); + final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles"); + final Granularity minuteWithTz = new PeriodGranularity(new Period("PT1M"), null, tz); + final Granularity minuteWithOrigin = new PeriodGranularity(new Period("PT1M"), origin, tz); + Assert.assertEquals(minuteWithTz, Granularities.fromVirtualColumn(floorWithTimezone)); + Assert.assertEquals(minuteWithOrigin, Granularities.fromVirtualColumn(floorWithOriginTimezone)); + } + private void assertBucketStart(final Granularity granularity, final DateTime in, final DateTime expectedInProperTz) { Assert.assertEquals( diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index 8413c7ea0ed..4113191a50d 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -88,7 +88,7 @@ public class TimeseriesQueryRunnerBonusTest results = runTimeseriesCount(oneRowIndex); - Assert.assertEquals("index size", 1, oneRowIndex.size()); + Assert.assertEquals("index size", 1, oneRowIndex.numRows()); Assert.assertEquals("result size", 1, results.size()); Assert.assertEquals("result timestamp", DateTimes.of("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); Assert.assertEquals("result count metric", 1, (long) results.get(0).getValue().getLongMetric("rows")); @@ -103,7 +103,7 @@ public class TimeseriesQueryRunnerBonusTest results = runTimeseriesCount(oneRowIndex); - Assert.assertEquals("index size", 2, oneRowIndex.size()); + Assert.assertEquals("index size", 2, oneRowIndex.numRows()); Assert.assertEquals("result size", 1, results.size()); Assert.assertEquals("result timestamp", DateTimes.of("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); Assert.assertEquals("result count metric", 2, (long) results.get(0).getValue().getLongMetric("rows")); diff --git a/processing/src/test/java/org/apache/druid/segment/AggregateProjectionMetadataTest.java b/processing/src/test/java/org/apache/druid/segment/AggregateProjectionMetadataTest.java new file mode 100644 index 00000000000..49b38da340e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/AggregateProjectionMetadataTest.java @@ -0,0 +1,149 @@ +/* + * 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; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.SortedSet; + +public class AggregateProjectionMetadataTest extends InitializedNullHandlingTest +{ + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + AggregateProjectionMetadata spec = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "time", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "time") + ), + Arrays.asList("a", "b", "time", "c", "d"), + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("e", "e") + }, + Arrays.asList( + OrderBy.ascending("a"), + OrderBy.ascending("b"), + OrderBy.ascending("time"), + OrderBy.ascending("c"), + OrderBy.ascending("d") + ) + ), + 12345 + ); + Assert.assertEquals( + spec, + JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(spec), AggregateProjectionMetadata.class) + ); + } + + + @Test + public void testComparator() + { + SortedSet metadataBest = new ObjectAVLTreeSet<>(AggregateProjectionMetadata.COMPARATOR); + AggregateProjectionMetadata good = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "good", + "theTime", + VirtualColumns.create(Granularities.toVirtualColumn(Granularities.HOUR, "theTime")), + Arrays.asList("theTime", "a", "b", "c"), + new AggregatorFactory[] { + new CountAggregatorFactory("chocula") + }, + Arrays.asList( + OrderBy.ascending("theTime"), + OrderBy.ascending("a"), + OrderBy.ascending("b"), + OrderBy.ascending("c") + ) + ), + 123 + ); + // same row count, but more aggs more better + AggregateProjectionMetadata better = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "better", + "theTime", + VirtualColumns.create(Granularities.toVirtualColumn(Granularities.HOUR, "theTime")), + Arrays.asList("c", "d", "theTime"), + new AggregatorFactory[] { + new CountAggregatorFactory("chocula"), + new LongSumAggregatorFactory("e", "e") + }, + Arrays.asList( + OrderBy.ascending("c"), + OrderBy.ascending("d"), + OrderBy.ascending("theTime") + ) + ), + 123 + ); + + // small rows is best + AggregateProjectionMetadata best = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "better", + null, + VirtualColumns.EMPTY, + Arrays.asList("f", "g"), + new AggregatorFactory[0], + Arrays.asList(OrderBy.ascending("f"), OrderBy.ascending("g")) + ), + 10 + ); + metadataBest.add(good); + metadataBest.add(better); + metadataBest.add(best); + Assert.assertEquals(best, metadataBest.first()); + Assert.assertEquals(good, metadataBest.last()); + } + + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(AggregateProjectionMetadata.class).usingGetClass().verify(); + } + + @Test + public void testEqualsAndHashcodeSchema() + { + EqualsVerifier.forClass(AggregateProjectionMetadata.Schema.class) + .withIgnoredFields("orderingWithTimeSubstitution", "timeColumnPosition", "granularity") + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java new file mode 100644 index 00000000000..446e714f997 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java @@ -0,0 +1,1238 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.Hash; +import it.unimi.dsi.fastutil.objects.ObjectOpenCustomHashSet; +import org.apache.druid.collections.CloseableDefaultBlockingPool; +import org.apache.druid.collections.CloseableStupidPool; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.ListBasedInputRow; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.DoubleDimensionSchema; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.firstlast.last.LongLastAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; +import org.apache.druid.query.groupby.GroupingEngine; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.joda.time.DateTime; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +@RunWith(Parameterized.class) +public class CursorFactoryProjectionTest extends InitializedNullHandlingTest +{ + private static final Closer CLOSER = Closer.create(); + private static final DateTime TIMESTAMP = Granularities.DAY.bucket(DateTimes.nowUtc()).getStart(); + + private static final RowSignature ROW_SIGNATURE = RowSignature.builder() + .add("a", ColumnType.STRING) + .add("b", ColumnType.STRING) + .add("c", ColumnType.LONG) + .add("d", ColumnType.DOUBLE) + .build(); + private static final List ROWS = Arrays.asList( + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP, + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "aa", 1L, 1.0) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(2), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "bb", 1L, 1.1, 1.1f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(4), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "cc", 2L, 2.2, 2.2f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(6), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("b", "aa", 3L, 3.3, 3.3f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(8), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("b", "aa", 4L, 4.4, 4.4f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(10), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("b", "bb", 5L, 5.5, 5.5f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusHours(1), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "aa", 1L, 1.1, 1.1f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusHours(1).plusMinutes(1), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "dd", 2L, 2.2, 2.2f) + ) + ); + + private static final List PROJECTIONS = Arrays.asList( + new AggregateProjectionSpec( + "ab_hourly_cd_sum", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("b"), + new LongDimensionSchema("__gran") + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("_c_sum", "c"), + new DoubleSumAggregatorFactory("d", "d") + } + ), + new AggregateProjectionSpec( + "a_hourly_c_sum_with_count_latest", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList( + new LongDimensionSchema("__gran"), + new StringDimensionSchema("a") + ), + new AggregatorFactory[]{ + new CountAggregatorFactory("chocula"), + new LongSumAggregatorFactory("_c_sum", "c"), + new LongLastAggregatorFactory("_c_last", "c", null) + } + ), + new AggregateProjectionSpec( + "bf_daily_c_sum", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.DAY, "__gran") + ), + Arrays.asList( + new LongDimensionSchema("__gran"), + new StringDimensionSchema("b"), + new FloatDimensionSchema("e") + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("_c_sum", "c") + } + ), + new AggregateProjectionSpec( + "ab_daily", + null, + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("b") + ), + null + ), + new AggregateProjectionSpec( + "abfoo_daily", + VirtualColumns.create( + new ExpressionVirtualColumn( + "bfoo", + "concat(b, 'foo')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ), + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("bfoo") + ), + null + ), + // cannot really make an 'all' granularity projection, but can do something like floor time to the segment + // granularity interval resulting in a single row + new AggregateProjectionSpec( + "c_sum", + VirtualColumns.create(Granularities.toVirtualColumn(Granularities.DAY, "__gran")), + Collections.singletonList(new LongDimensionSchema("__gran")), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("_c_sum", "c") + } + ) + ); + + private static final List AUTO_PROJECTIONS = PROJECTIONS.stream().map(projection -> { + return new AggregateProjectionSpec( + projection.getName(), + projection.getVirtualColumns(), + projection.getGroupingColumns() + .stream() + .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .collect(Collectors.toList()), + projection.getAggregators() + ); + }).collect(Collectors.toList()); + + @Parameterized.Parameters(name = "name: {0}, sortByDim: {3}, autoSchema: {4}") + public static Collection constructorFeeder() + { + final List constructors = new ArrayList<>(); + final DimensionsSpec.Builder dimensionsBuilder = + DimensionsSpec.builder() + .setDimensions( + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("b"), + new LongDimensionSchema("c"), + new DoubleDimensionSchema("d"), + new FloatDimensionSchema("e") + ) + ); + DimensionsSpec dimsTimeOrdered = dimensionsBuilder.build(); + DimensionsSpec dimsOrdered = dimensionsBuilder.setForceSegmentSortByTime(false).build(); + + + List autoDims = dimsOrdered.getDimensions() + .stream() + .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .collect(Collectors.toList()); + for (boolean incremental : new boolean[]{true, false}) { + for (boolean sortByDim : new boolean[]{true, false}) { + for (boolean autoSchema : new boolean[]{true, false}) { + final DimensionsSpec dims; + if (sortByDim) { + if (autoSchema) { + dims = dimsOrdered.withDimensions(autoDims); + } else { + dims = dimsOrdered; + } + } else { + if (autoSchema) { + dims = dimsTimeOrdered.withDimensions(autoDims); + } else { + dims = dimsTimeOrdered; + } + } + if (incremental) { + IncrementalIndex index = CLOSER.register(makeBuilder(dims, autoSchema).buildIncrementalIndex()); + constructors.add(new Object[]{ + "incrementalIndex", + new IncrementalIndexCursorFactory(index), + new IncrementalIndexTimeBoundaryInspector(index), + sortByDim, + autoSchema + }); + } else { + QueryableIndex index = CLOSER.register(makeBuilder(dims, autoSchema).buildMMappedIndex()); + constructors.add(new Object[]{ + "queryableIndex", + new QueryableIndexCursorFactory(index), + QueryableIndexTimeBoundaryInspector.create(index), + sortByDim, + autoSchema + }); + } + } + } + } + return constructors; + } + + @AfterClass + public static void cleanup() throws IOException + { + CLOSER.close(); + } + + + public final CursorFactory projectionsCursorFactory; + public final TimeBoundaryInspector projectionsTimeBoundaryInspector; + + private final GroupingEngine groupingEngine; + private final TimeseriesQueryEngine timeseriesEngine; + + private final NonBlockingPool nonBlockingPool; + public final boolean sortByDim; + public final boolean autoSchema; + + @Rule + public final CloserRule closer = new CloserRule(false); + + public CursorFactoryProjectionTest( + String name, + CursorFactory projectionsCursorFactory, + TimeBoundaryInspector projectionsTimeBoundaryInspector, + boolean sortByDim, + boolean autoSchema + ) + { + this.projectionsCursorFactory = projectionsCursorFactory; + this.projectionsTimeBoundaryInspector = projectionsTimeBoundaryInspector; + this.sortByDim = sortByDim; + this.autoSchema = autoSchema; + this.nonBlockingPool = closer.closeLater( + new CloseableStupidPool<>( + "GroupByQueryEngine-bufferPool", + () -> ByteBuffer.allocate(50000) + ) + ); + this.groupingEngine = new GroupingEngine( + new DruidProcessingConfig(), + GroupByQueryConfig::new, + new GroupByResourcesReservationPool( + closer.closeLater( + new CloseableDefaultBlockingPool<>( + () -> ByteBuffer.allocate(50000), + 5 + ) + ), + new GroupByQueryConfig() + ), + TestHelper.makeJsonMapper(), + TestHelper.makeSmileMapper(), + (query, future) -> { + } + ); + this.timeseriesEngine = new TimeseriesQueryEngine(nonBlockingPool); + } + + @Test + public void testProjectionSelectionTwoDims() + { + // this query can use the projection with 2 dims, which has 7 rows instead of the total of 8 + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(6, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(6, results.size()); + if (projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + if (autoSchema) { + Assert.assertArrayEquals(new Object[]{"b", "bb"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa"}, results.get(5).getArray()); + } else { + Assert.assertArrayEquals(new Object[]{"a", "dd"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb"}, results.get(5).getArray()); + } + } else { + Assert.assertArrayEquals(new Object[]{"a", "aa"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb"}, results.get(5).getArray()); + } + } + + @Test + public void testProjectionSelectionTwoDimsVirtual() + { + // this query can use the projection with 2 dims, which has 7 rows instead of the total of 8 + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("v0") + .setVirtualColumns( + new ExpressionVirtualColumn( + "v0", + "concat(b, 'foo')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Arrays.asList( + new OrderByColumnSpec("a", OrderByColumnSpec.Direction.ASCENDING, StringComparators.LEXICOGRAPHIC), + new OrderByColumnSpec("v0", OrderByColumnSpec.Direction.ASCENDING, StringComparators.LEXICOGRAPHIC) + ), + 10 + ) + ) + .setContext(ImmutableMap.of(QueryContexts.USE_PROJECTION, "abfoo_daily")) + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(6, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(6, results.size()); + if (projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + // testing ordering of stuff is kind of tricky at this level... + if (autoSchema) { + Assert.assertArrayEquals(new Object[]{"b", "bbfoo"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ddfoo"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aafoo"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ccfoo"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bbfoo"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aafoo"}, results.get(5).getArray()); + } else { + Assert.assertArrayEquals(new Object[]{"a", "ddfoo"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aafoo"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aafoo"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ccfoo"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bbfoo"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bbfoo"}, results.get(5).getArray()); + } + } else { + Assert.assertArrayEquals(new Object[]{"a", "aafoo"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bbfoo"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ccfoo"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ddfoo"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aafoo"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bbfoo"}, results.get(5).getArray()); + } + } + + @Test + public void testProjectionSelectionTwoDimsCount() + { + // cannot use a projection since count is not defined + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .addAggregator(new CountAggregatorFactory("count")) + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(6, results.size()); + if (projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + if (autoSchema) { + Assert.assertArrayEquals(new Object[]{"b", "aa", 2L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc", 1L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb", 1L}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 1L}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd", 1L}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa", 2L}, results.get(5).getArray()); + } else { + Assert.assertArrayEquals(new Object[]{"a", "dd", 1L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa", 2L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb", 1L}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa", 2L}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc", 1L}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 1L}, results.get(5).getArray()); + } + } else { + Assert.assertArrayEquals(new Object[]{"a", "aa", 2L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb", 1L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc", 1L}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa", 2L}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 1L}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd", 1L}, results.get(5).getArray()); + } + } + + @Test + public void testProjectionSelectionTwoDimsCountForce() + { + // cannot use a projection since count is not defined + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .addAggregator(new CountAggregatorFactory("count")) + .setContext(ImmutableMap.of(QueryContexts.FORCE_PROJECTION, true)) + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + Throwable t = Assert.assertThrows( + DruidException.class, + () -> projectionsCursorFactory.makeCursorHolder(buildSpec) + ); + Assert.assertEquals("Force projections specified, but none satisfy query", t.getMessage()); + } + + @Test + public void testProjectionSkipContext() + { + // setting context flag to skip projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongLastAggregatorFactory("c_last", "c", null)) + .setContext(ImmutableMap.of(QueryContexts.NO_PROJECTIONS, true)) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + // has to scan full 8 rows because context ensures projections not used + Assert.assertEquals(8, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals( + new Object[]{"a", 7L, Pair.of(TIMESTAMP.plusHours(1).plusMinutes(1).getMillis(), 2L)}, + results.get(0).getArray() + ); + Assert.assertArrayEquals( + new Object[]{"b", 12L, Pair.of(TIMESTAMP.plusMinutes(10).getMillis(), 5L)}, + results.get(1).getArray() + ); + } + + @Test + public void testProjectionSingleDim() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongLastAggregatorFactory("c_last", "c", null)) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals( + new Object[]{"a", 7L, Pair.of(TIMESTAMP.plusHours(1).plusMinutes(1).getMillis(), 2L)}, + results.get(0).getArray() + ); + Assert.assertArrayEquals( + new Object[]{"b", 12L, Pair.of(TIMESTAMP.plusMinutes(10).getMillis(), 5L)}, + results.get(1).getArray() + ); + } + + @Test + public void testProjectionSingleDimFilter() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .setDimFilter(new EqualityFilter("a", ColumnType.STRING, "a", null)) + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongLastAggregatorFactory("c_last", "c", null)) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(2, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(1, results.size()); + Assert.assertArrayEquals( + new Object[]{"a", 7L, Pair.of(TIMESTAMP.plusHours(1).plusMinutes(1).getMillis(), 2L)}, + results.get(0).getArray() + ); + } + + @Test + public void testProjectionSingleDimCount() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new CountAggregatorFactory("cnt")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", 7L, 5L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", 12L, 3L}, results.get(1).getArray()); + } + + @Test + public void testProjectionSingleDimMultipleSameAggs() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongSumAggregatorFactory("c_sum_2", "c")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", 7L, 7L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", 12L, 12L}, results.get(1).getArray()); + } + + @Test + public void testQueryGranularityFinerThanProjectionGranularity() + { + final GroupByQuery.Builder queryBuilder = + GroupByQuery.builder() + .setDataSource("test") + .setInterval(Intervals.ETERNITY) + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")); + if (sortByDim) { + queryBuilder.setVirtualColumns(Granularities.toVirtualColumn(Granularities.MINUTE, "__gran")) + .setDimensions( + DefaultDimensionSpec.of("__gran", ColumnType.LONG), + DefaultDimensionSpec.of("a") + ) + .setGranularity(Granularities.ALL); + } else { + queryBuilder.addDimension("a") + .setGranularity(Granularities.MINUTE); + } + final GroupByQuery query = queryBuilder.build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(8, results.size()); + + if (sortByDim && projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + // this sorts funny when not time ordered + Set resultsInNoParticularOrder = makeArrayResultSet(); + resultsInNoParticularOrder.addAll( + ROWS.stream() + .map(x -> new Object[]{x.getTimestamp().getMillis(), x.getRaw("a"), x.getRaw("c")}) + .collect(Collectors.toList()) + ); + for (ResultRow row : results) { + Assert.assertTrue(resultsInNoParticularOrder.contains(row.getArray())); + } + } else { + Assert.assertArrayEquals( + new Object[]{ROWS.get(0).getTimestamp().getMillis(), "a", 1L}, + results.get(0).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(1).getTimestamp().getMillis(), "a", 1L}, + results.get(1).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(2).getTimestamp().getMillis(), "a", 2L}, + results.get(2).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(3).getTimestamp().getMillis(), "b", 3L}, + results.get(3).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(4).getTimestamp().getMillis(), "b", 4L}, + results.get(4).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(5).getTimestamp().getMillis(), "b", 5L}, + results.get(5).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(6).getTimestamp().getMillis(), "a", 1L}, + results.get(6).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(7).getTimestamp().getMillis(), "a", 2L}, + results.get(7).getArray() + ); + } + } + + @Test + public void testQueryGranularityFitsProjectionGranularity() + { + final GroupByQuery.Builder queryBuilder = + GroupByQuery.builder() + .setDataSource("test") + .setInterval(Intervals.ETERNITY) + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")); + if (sortByDim) { + queryBuilder.setGranularity(Granularities.ALL) + .setDimensions( + DefaultDimensionSpec.of("__gran", ColumnType.LONG), + DefaultDimensionSpec.of("a") + ) + .setVirtualColumns(Granularities.toVirtualColumn(Granularities.HOUR, "__gran")); + } else { + queryBuilder.addDimension("a") + .setGranularity(Granularities.HOUR); + } + final GroupByQuery query = queryBuilder.build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(3, results.size()); + if (sortByDim && projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + Set resultsInNoParticularOrder = makeArrayResultSet( + new Object[]{TIMESTAMP.getMillis(), "a", 4L}, + new Object[]{TIMESTAMP.getMillis(), "b", 12L}, + new Object[]{TIMESTAMP.plusHours(1).getMillis(), "a", 3L} + ); + for (ResultRow row : results) { + Assert.assertTrue(resultsInNoParticularOrder.contains(row.getArray())); + } + } else { + Assert.assertArrayEquals(new Object[]{TIMESTAMP.getMillis(), "a", 4L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.getMillis(), "b", 12L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1).getMillis(), "a", 3L}, results.get(2).getArray()); + } + } + + @Test + public void testProjectionSelectionMissingAggregatorButHasAggregatorInput() + { + // d is present as a column on the projection, but since its an aggregate projection we cannot use it + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("b") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new FloatSumAggregatorFactory("e_sum", "e")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(4, results.size()); + Assert.assertArrayEquals(new Object[]{"aa", 9L, NullHandling.defaultFloatValue()}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"bb", 6L, NullHandling.defaultFloatValue()}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"cc", 2L, NullHandling.defaultFloatValue()}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"dd", 2L, NullHandling.defaultFloatValue()}, results.get(3).getArray()); + } + + @Test + public void testProjectionSelectionMissingAggregatorAndAggregatorInput() + { + // since d isn't present on the smaller projection, cant use it, but can still use the larger projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new DoubleSumAggregatorFactory("d_sum", "d")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(7, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", 7L, 7.6000000000000005}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", 12L, 13.2}, results.get(1).getArray()); + } + + @Test + public void testProjectionSelectionMissingColumnOnBaseTableToo() + { + // since d isn't present on the smaller projection, cant use it, but can still use the larger projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("z") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new DoubleSumAggregatorFactory("d_sum", "d")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(7, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", null, 7L, 7.6000000000000005}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", null, 12L, 13.2}, results.get(1).getArray()); + } + + private static IndexBuilder makeBuilder(DimensionsSpec dimensionsSpec, boolean autoSchema) + { + File tmp = FileUtils.createTempDir(); + CLOSER.register(tmp::delete); + return IndexBuilder.create() + .tmpDir(tmp) + .schema( + IncrementalIndexSchema.builder() + .withDimensionsSpec(dimensionsSpec) + .withRollup(false) + .withMinTimestamp(TIMESTAMP.getMillis()) + .withProjections(autoSchema ? AUTO_PROJECTIONS : PROJECTIONS) + .build() + ) + .rows(ROWS); + } + + @Test + public void testTimeseriesQueryGranularityFitsProjectionGranularity() + { + Assume.assumeFalse(sortByDim); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.HOUR) + .aggregators(new LongSumAggregatorFactory("c_sum", "c")) + .build(); + + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + + final Sequence> resultRows = timeseriesEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + null + ); + + final List> results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES); + Assert.assertArrayEquals(new Object[]{TIMESTAMP, 16L}, getResultArray(results.get(0), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1), 3L}, getResultArray(results.get(1), querySignature)); + } + + @Test + public void testTimeseriesQueryGranularityAllFitsProjectionGranularity() + { + Assume.assumeFalse(sortByDim); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.ALL) + .aggregators(new LongSumAggregatorFactory("c_sum", "c")) + .context(ImmutableMap.of(QueryContexts.USE_PROJECTION, "c_sum")) + .build(); + + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(1, rowCount); + } + + final Sequence> resultRows = timeseriesEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + null + ); + + final List> results = resultRows.toList(); + Assert.assertEquals(1, results.size()); + final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES); + Assert.assertArrayEquals(new Object[]{TIMESTAMP, 19L}, getResultArray(results.get(0), querySignature)); + } + + @Test + public void testTimeseriesQueryGranularityFinerThanProjectionGranularity() + { + Assume.assumeFalse(sortByDim); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.MINUTE) + .aggregators(new LongSumAggregatorFactory("c_sum", "c")) + .context(ImmutableMap.of(TimeseriesQuery.SKIP_EMPTY_BUCKETS, true)) + .build(); + + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence> resultRows = timeseriesEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + null + ); + + final List> results = resultRows.toList(); + Assert.assertEquals(8, results.size()); + final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES); + Assert.assertArrayEquals(new Object[]{TIMESTAMP, 1L}, getResultArray(results.get(0), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(2), 1L}, getResultArray(results.get(1), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(4), 2L}, getResultArray(results.get(2), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(6), 3L}, getResultArray(results.get(3), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(8), 4L}, getResultArray(results.get(4), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(10), 5L}, getResultArray(results.get(5), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1), 1L}, getResultArray(results.get(6), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1).plusMinutes(1), 2L}, getResultArray(results.get(7), querySignature)); + } + + + private static Set makeArrayResultSet() + { + Set resultsInNoParticularOrder = new ObjectOpenCustomHashSet<>( + new Hash.Strategy() + { + @Override + public int hashCode(Object[] o) + { + return Arrays.hashCode(o); + } + + @Override + public boolean equals(Object[] a, Object[] b) + { + return Arrays.deepEquals(a, b); + } + } + ); + return resultsInNoParticularOrder; + } + + private static Set makeArrayResultSet(Object[]... values) + { + Set resultsInNoParticularOrder = makeArrayResultSet(); + resultsInNoParticularOrder.addAll(Arrays.asList(values)); + return resultsInNoParticularOrder; + } + + private static Object[] getResultArray(Result result, RowSignature rowSignature) + { + final Object[] rowArray = new Object[rowSignature.size()]; + for (int i = 0; i < rowSignature.size(); i++) { + if (i == 0) { + rowArray[i] = result.getTimestamp(); + } else { + rowArray[i] = result.getValue().getMetric(rowSignature.getColumnName(i)); + } + } + return rowArray; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java index 52c129379fc..262ecd50015 100644 --- a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java @@ -30,6 +30,7 @@ import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.Druids; import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; @@ -59,6 +60,7 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.nio.ByteBuffer; import java.util.List; +import java.util.stream.Collectors; public class CursorHolderPreaggTest extends InitializedNullHandlingTest { @@ -137,6 +139,15 @@ public class CursorHolderPreaggTest extends InitializedNullHandlingTest return true; } + @Nullable + @Override + public List getAggregatorsForPreAggregated() + { + return spec.getAggregators() + .stream().map(AggregatorFactory::getCombiningFactory) + .collect(Collectors.toList()); + } + @Override public void close() { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java index 703de4439e4..92bc826bb57 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java @@ -273,7 +273,7 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling ColumnDescriptor serde = mapper.readValue( IndexIO.SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class ); - return serde.read(byteBuffer, columnConfig, smooshedFiles); + return serde.read(byteBuffer, columnConfig, smooshedFiles, null); } } } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 5c519f88458..7faa31d8e53 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -410,7 +410,8 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest null, Granularities.NONE, Boolean.TRUE, - Cursors.ascendingTimeOrder() + Cursors.ascendingTimeOrder(), + null ), index.getMetadata() ); diff --git a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java index b158480b22d..5d06d2ab8c2 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -22,12 +22,17 @@ package org.apache.druid.segment; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.firstlast.last.LongLastAggregatorFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -40,7 +45,7 @@ import java.util.stream.Collectors; /** * */ -public class MetadataTest +public class MetadataTest extends InitializedNullHandlingTest { @Test public void testSerde() throws Exception @@ -57,6 +62,7 @@ public class MetadataTest null, Granularities.ALL, Boolean.FALSE, + null, null ); @@ -83,13 +89,32 @@ public class MetadataTest AggregatorFactory[] aggs = new AggregatorFactory[]{ new LongMaxAggregatorFactory("n", "f") }; + List projectionSpecs = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "__gran"), + new AggregatorFactory[]{ + new LongLastAggregatorFactory("atLongLast", "d", null) + }, + makeOrderBy("a", "b", "__gran") + ), + 1234 + ) + ); + final Metadata m1 = new Metadata( Collections.singletonMap("k", "v"), aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - null + null, + projectionSpecs ); final Metadata m2 = new Metadata( @@ -98,7 +123,8 @@ public class MetadataTest new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - null + null, + projectionSpecs ); final Metadata m3 = new Metadata( @@ -107,7 +133,8 @@ public class MetadataTest new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.TRUE, - null + null, + projectionSpecs ); final Metadata merged = new Metadata( @@ -118,7 +145,8 @@ public class MetadataTest new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - Cursors.ascendingTimeOrder() + Cursors.ascendingTimeOrder(), + projectionSpecs ); Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null)); @@ -129,7 +157,15 @@ public class MetadataTest metadataToBeMerged.add(null); final Metadata merged2 = - new Metadata(Collections.singletonMap("k", "v"), null, null, null, null, Cursors.ascendingTimeOrder()); + new Metadata( + Collections.singletonMap("k", "v"), + null, + null, + null, + null, + Cursors.ascendingTimeOrder(), + projectionSpecs + ); Assert.assertEquals(merged2, Metadata.merge(metadataToBeMerged, null)); @@ -139,7 +175,15 @@ public class MetadataTest }; final Metadata merged3 = - new Metadata(Collections.singletonMap("k", "v"), explicitAggs, null, null, null, Cursors.ascendingTimeOrder()); + new Metadata( + Collections.singletonMap("k", "v"), + explicitAggs, + null, + null, + null, + Cursors.ascendingTimeOrder(), + projectionSpecs + ); Assert.assertEquals( merged3, @@ -152,7 +196,8 @@ public class MetadataTest new TimestampSpec("ds", "auto", null), Granularities.ALL, null, - Cursors.ascendingTimeOrder() + Cursors.ascendingTimeOrder(), + projectionSpecs ); Assert.assertEquals( merged4, @@ -219,6 +264,107 @@ public class MetadataTest ); } + @Test + public void testMergeProjectionsUnexpectedMismatch() + { + List p1 = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "__gran"), + new AggregatorFactory[]{ + new LongLastAggregatorFactory("atLongLast", "d", null) + }, + makeOrderBy("a", "b", "__gran") + ), + 654321 + ) + ); + + List p2 = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "_gran"), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("longSum", "d") + }, + makeOrderBy("a", "b", "__gran") + ), + 1234 + ) + ); + + List p3 = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "__gran"), + new AggregatorFactory[]{ + new LongLastAggregatorFactory("atLongLast", "d", null) + }, + makeOrderBy("a", "b", "__gran") + ), + 12121 + ), + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection2", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.DAY, "__gran") + ), + Arrays.asList("__gran", "a"), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("longSum", "d") + }, + makeOrderBy("__gran", "a") + ), + 555 + ) + ); + + Throwable t = Assert.assertThrows( + DruidException.class, + () -> Metadata.validateProjections(Arrays.asList(p1, p2)) + ); + MatcherAssert.assertThat( + t.getMessage(), + CoreMatchers.startsWith("Unable to merge projections: mismatched projections") + ); + + t = Assert.assertThrows( + DruidException.class, + () -> Metadata.validateProjections(Arrays.asList(p1, p3)) + ); + + MatcherAssert.assertThat( + t.getMessage(), + CoreMatchers.startsWith("Unable to merge projections: mismatched projections count") + ); + + t = Assert.assertThrows( + DruidException.class, + () -> Metadata.validateProjections(Arrays.asList(p1, null)) + ); + MatcherAssert.assertThat( + t.getMessage(), + CoreMatchers.startsWith("Unable to merge projections: some projections were null") + ); + } + private static List makeOrderBy(final String... columnNames) { return Arrays.stream(columnNames).map(OrderBy::ascending).collect(Collectors.toList()); diff --git a/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java b/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java index 980b23e140c..69dbea9797b 100644 --- a/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java @@ -31,6 +31,7 @@ import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -128,7 +129,8 @@ public class V9IndexLoaderTest extends InitializedNullHandlingTest String columnName, ObjectMapper mapper, ByteBuffer byteBuffer, - SmooshedFileMapper smooshedFiles + SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parentColumn ) throws IOException { if (COUNT_COLUMN.equals(columnName)) { @@ -138,7 +140,8 @@ public class V9IndexLoaderTest extends InitializedNullHandlingTest columnName, mapper, byteBuffer, - smooshedFiles + smooshedFiles, + parentColumn ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java index 9c0e3d8f158..b29a571389c 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java @@ -233,7 +233,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest populateIndex(timestamp, index); Assert.assertEquals(Arrays.asList("__time", "dim1", "dim2"), index.getDimensionNames(true)); Assert.assertEquals(Arrays.asList("dim1", "dim2"), index.getDimensionNames(false)); - Assert.assertEquals(2, index.size()); + Assert.assertEquals(2, index.numRows()); final Iterator rows = index.iterator(); Row row = rows.next(); @@ -299,7 +299,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ), index.getMetricNames() ); - Assert.assertEquals(2, index.size()); + Assert.assertEquals(2, index.numRows()); final Iterator rows = index.iterator(); Row row = rows.next(); @@ -687,7 +687,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(2, index.size()); + Assert.assertEquals(2, index.numRows()); } @Test @@ -727,7 +727,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(index.isRollup() ? 1 : 4, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 4, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -795,7 +795,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(index.isRollup() ? 1 : 4, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 4, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -848,7 +848,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -901,7 +901,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -951,7 +951,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -999,7 +999,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 965b3517b24..a174bc9e667 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -543,7 +543,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final IncrementalIndex index = input.buildIncrementalIndex(); @@ -571,7 +572,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final QueryableIndex index = input.buildMMappedIndex(); @@ -600,7 +602,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ) // if 1 row per segment some of the columns have null values for the row which causes 'auto' @@ -677,7 +680,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED); @@ -700,7 +704,8 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR); diff --git a/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java index 0cb5f25ef73..dd88367e783 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java @@ -744,6 +744,6 @@ public class DataGeneratorTest extends InitializedNullHandlingTest .build(); dataGenerator.addToIndex(index, 100); - Assert.assertEquals(100, index.size()); + Assert.assertEquals(100, index.numRows()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactoryTest.java similarity index 83% rename from processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactoryTest.java index 87c60cb7420..36abb79be8c 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactoryTest.java @@ -24,8 +24,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; -import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; @@ -54,9 +55,9 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; @@ -70,6 +71,7 @@ import org.apache.druid.segment.Cursors; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; +import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; @@ -96,12 +98,19 @@ import java.util.List; import java.util.Set; /** + * */ @RunWith(Parameterized.class) -public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingTest +public class IncrementalIndexCursorFactoryTest extends InitializedNullHandlingTest { public final IncrementalIndexCreator indexCreator; + private final GroupingEngine groupingEngine; + private final TopNQueryEngine topnQueryEngine; + + private final NonBlockingPool nonBlockingPool; + + /** * If true, sort by [billy, __time]. If false, sort by [__time]. */ @@ -110,7 +119,8 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT @Rule public final CloserRule closer = new CloserRule(false); - public IncrementalIndexStorageAdapterTest(String indexType, boolean sortByDim) throws JsonProcessingException + public IncrementalIndexCursorFactoryTest(String indexType, boolean sortByDim) + throws JsonProcessingException { BuiltInTypesModule.registerHandlersAndSerde(); this.sortByDim = sortByDim; @@ -144,6 +154,31 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT } ) ); + + nonBlockingPool = closer.closeLater( + new CloseableStupidPool<>( + "GroupByQueryEngine-bufferPool", + () -> ByteBuffer.allocate(50000) + ) + ); + groupingEngine = new GroupingEngine( + new DruidProcessingConfig(), + GroupByQueryConfig::new, + new GroupByResourcesReservationPool( + closer.closeLater( + new CloseableDefaultBlockingPool<>( + () -> ByteBuffer.allocate(50000), + 5 + ) + ), + new GroupByQueryConfig() + ), + TestHelper.makeJsonMapper(), + TestHelper.makeSmileMapper(), + (query, future) -> { + } + ); + topnQueryEngine = new TopNQueryEngine(nonBlockingPool); } @Parameterized.Parameters(name = "{index}: {0}, sortByDim: {1}") @@ -183,37 +218,24 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .addOrderByColumn("billy") .build(); - final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "GroupByQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) - ); - ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) - ) { - final Sequence rows = GroupByQueryEngine.process( - query, - new IncrementalIndexTimeBoundaryInspector(index), - cursorHolder, - buildSpec, - processingBuffer.get(), - null, - new GroupByQueryConfig(), - new DruidProcessingConfig() - ); + final Sequence rows = groupingEngine.process( + query, + cursorFactory, + new IncrementalIndexTimeBoundaryInspector(index), + nonBlockingPool, + null + ); - final List results = rows.toList(); + final List results = rows.toList(); - Assert.assertEquals(2, results.size()); + Assert.assertEquals(2, results.size()); - ResultRow row = results.get(0); - Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); - row = results.get(1); - Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); - } + row = results.get(1); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } @Test @@ -260,39 +282,26 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .addOrderByColumn("billy") .build(); final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "GroupByQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) - ); - ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) - ) { - final Sequence rows = GroupByQueryEngine.process( - query, - new IncrementalIndexTimeBoundaryInspector(index), - cursorHolder, - buildSpec, - processingBuffer.get(), - null, - new GroupByQueryConfig(), - new DruidProcessingConfig() - ); + final Sequence rows = groupingEngine.process( + query, + cursorFactory, + new IncrementalIndexTimeBoundaryInspector(index), + nonBlockingPool, + null + ); - final List results = rows.toList(); + final List results = rows.toList(); - Assert.assertEquals(2, results.size()); + Assert.assertEquals(2, results.size()); - ResultRow row = results.get(0); - Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L, 2.0}, row.getArray()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L, 2.0}, row.getArray()); - row = results.get(1); - Assert.assertArrayEquals( - new Object[]{"hip", "hop", 1L, 6.0}, - row.getArray() - ); - } + row = results.get(1); + Assert.assertArrayEquals( + new Object[]{"hip", "hop", 1L, 6.0}, + row.getArray() + ); } @Test @@ -371,33 +380,24 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT ) ); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "TopNQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) + final Iterable> results = topnQueryEngine + .query( + new TopNQueryBuilder() + .dataSource("test") + .granularity(Granularities.ALL) + .intervals(Collections.singletonList(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))) + .dimension("sally") + .metric("cnt") + .threshold(10) + .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) + .build(), + new IncrementalIndexSegment(index, SegmentId.dummy("test")), + null ) - ) { - TopNQueryEngine engine = new TopNQueryEngine(pool); + .toList(); - final Iterable> results = engine - .query( - new TopNQueryBuilder() - .dataSource("test") - .granularity(Granularities.ALL) - .intervals(Collections.singletonList(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))) - .dimension("sally") - .metric("cnt") - .threshold(10) - .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) - .build(), - new IncrementalIndexSegment(index, SegmentId.dummy("test")), - null - ) - .toList(); - - Assert.assertEquals(1, Iterables.size(results)); - Assert.assertEquals(1, results.iterator().next().getValue().getValue().size()); - } + Assert.assertEquals(1, Iterables.size(results)); + Assert.assertEquals(1, results.iterator().next().getValue().getValue().size()); } @Test @@ -430,34 +430,21 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "GroupByQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) - ); - ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) - ) { - final Sequence rows = GroupByQueryEngine.process( - query, - new IncrementalIndexTimeBoundaryInspector(index), - cursorHolder, - buildSpec, - processingBuffer.get(), - null, - new GroupByQueryConfig(), - new DruidProcessingConfig() - ); + final Sequence rows = groupingEngine.process( + query, + cursorFactory, + new IncrementalIndexTimeBoundaryInspector(index), + nonBlockingPool, + null + ); - final List results = rows.toList(); + final List results = rows.toList(); - Assert.assertEquals(1, results.size()); + Assert.assertEquals(1, results.size()); - ResultRow row = results.get(0); - Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); - } + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } @Test @@ -701,6 +688,13 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT return super.hashCode(); } + @Override + public boolean equals(Object obj) + { + // Test code, hashcode and equals isn't important + return super.equals(obj); + } + private class DictionaryRaceTestFilterDruidPredicateFactory implements DruidPredicateFactory { @Override diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java index 067c078f5bc..7a99951c6bd 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java @@ -332,7 +332,7 @@ public class IncrementalIndexTest extends InitializedNullHandlingTest index.add(row); index.add(row); - Assert.assertEquals("rollup".equals(mode) ? 1 : 3, index.size()); + Assert.assertEquals("rollup".equals(mode) ? 1 : 3, index.numRows()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 1ebd1731c33..64ba2679f04 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -205,7 +205,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -244,7 +244,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest ); bob.setFileMapper(fileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE); + deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE, null); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -268,7 +268,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest ); bob.setFileMapper(arrayFileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(arrayBaseBuffer, bob, ColumnConfig.SELECTION_SIZE); + deserializer.read(arrayBaseBuffer, bob, ColumnConfig.SELECTION_SIZE, null); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -291,7 +291,8 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest bob, ColumnConfig.SELECTION_SIZE, bitmapSerdeFactory, - ByteOrder.nativeOrder() + ByteOrder.nativeOrder(), + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index 33eaa4a0312..aa42d58710d 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -225,7 +225,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest bob.setFileMapper(fileMapper); ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.createDeserializer(NestedDataComplexTypeSerde.TYPE_NAME); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE); + deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE, null); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index 3f237b9396f..f483e297be0 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -157,7 +157,7 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -192,7 +192,8 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); try (ScalarDoubleColumn column = (ScalarDoubleColumn) supplier.get()) { smokeTest(supplier, column); @@ -210,7 +211,8 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index 5fd8ddd299c..c8830f3aefd 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -157,7 +157,7 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -192,7 +192,8 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); try (ScalarLongColumn column = (ScalarLongColumn) supplier.get()) { smokeTest(supplier, column); @@ -210,7 +211,8 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index ee7ef05149e..d72970b3b12 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -157,7 +157,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -191,7 +191,8 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); try (StringUtf8DictionaryEncodedColumn column = (StringUtf8DictionaryEncodedColumn) supplier.get()) { smokeTest(supplier, column); @@ -208,7 +209,8 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 4a9423b87bc..6aea2ace234 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -282,7 +282,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -317,7 +317,8 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); try (VariantColumn column = (VariantColumn) supplier.get()) { smokeTest(supplier, column, data, expectedTypes); @@ -335,7 +336,8 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java index c6e6e9359b4..30aa6468dd7 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java @@ -66,7 +66,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); final ColumnCapabilities columnCapabilities = builder.build().getCapabilities(); Assert.assertTrue(Types.is(columnCapabilities, ValueType.DOUBLE)); Assert.assertTrue(columnCapabilities.hasNulls().isTrue()); @@ -82,7 +82,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -104,7 +104,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -132,7 +132,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -151,7 +151,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -172,7 +172,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -196,7 +196,7 @@ public class NullColumnPartSerdeTest extends InitializedNullHandlingTest { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); Assert.assertNull(holder.getIndexSupplier()); } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java index 14deba7725a..2e4cf91252f 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java @@ -53,6 +53,7 @@ public class CombinedDataSchema extends DataSchema granularitySpec, transformSpec, null, + null, null ); this.multiValuedDimensions = multiValuedDimensions; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index 22ee4ec4102..76a8c16802c 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -30,6 +30,7 @@ import com.google.common.base.Strings; import com.google.common.collect.Multiset; import com.google.common.collect.TreeMultiset; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; @@ -87,6 +88,7 @@ public class DataSchema // This is used for backward compatibility private InputRowParser inputRowParser; + private List projections; @JsonCreator public DataSchema( @@ -96,6 +98,7 @@ public class DataSchema @JsonProperty("metricsSpec") AggregatorFactory[] aggregators, @JsonProperty("granularitySpec") GranularitySpec granularitySpec, @JsonProperty("transformSpec") TransformSpec transformSpec, + @JsonProperty("projections") @Nullable List projections, @Deprecated @JsonProperty("parser") @Nullable Map parserMap, @JacksonInject ObjectMapper objectMapper ) @@ -120,6 +123,7 @@ public class DataSchema this.granularitySpec = granularitySpec; } this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; + this.projections = projections; this.parserMap = parserMap; this.objectMapper = objectMapper; @@ -351,6 +355,13 @@ public class DataSchema return transformSpec; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getProjections() + { + return projections; + } + @Deprecated @JsonProperty("parser") @Nullable @@ -410,6 +421,7 @@ public class DataSchema ", parserMap=" + parserMap + ", timestampSpec=" + timestampSpec + ", dimensionsSpec=" + dimensionsSpec + + ", projections=" + projections + ", inputRowParser=" + inputRowParser + '}'; } @@ -422,10 +434,9 @@ public class DataSchema private TransformSpec transformSpec; private Map parserMap; private ObjectMapper objectMapper; - - // The below fields can be initialized lazily from parser for backward compatibility. private TimestampSpec timestampSpec; private DimensionsSpec dimensionsSpec; + private List projections; public Builder() { @@ -435,13 +446,14 @@ public class DataSchema public Builder(DataSchema schema) { this.dataSource = schema.dataSource; - this.aggregators = schema.aggregators; - this.granularitySpec = schema.granularitySpec; - this.transformSpec = schema.transformSpec; - this.parserMap = schema.parserMap; - this.objectMapper = schema.objectMapper; this.timestampSpec = schema.timestampSpec; this.dimensionsSpec = schema.dimensionsSpec; + this.transformSpec = schema.transformSpec; + this.aggregators = schema.aggregators; + this.projections = schema.projections; + this.granularitySpec = schema.granularitySpec; + this.parserMap = schema.parserMap; + this.objectMapper = schema.objectMapper; } public Builder withDataSource(String dataSource) @@ -491,6 +503,12 @@ public class DataSchema return this; } + public Builder withProjections(List projections) + { + this.projections = projections; + return this; + } + @Deprecated public Builder withObjectMapper(ObjectMapper objectMapper) { @@ -514,6 +532,7 @@ public class DataSchema aggregators, granularitySpec, transformSpec, + projections, parserMap, objectMapper ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index bb5acec7b8f..16ac5ebfb96 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -1172,7 +1172,7 @@ public class BatchAppenderator implements Appenderator try { final long startTime = System.nanoTime(); - int numRows = indexToPersist.getIndex().size(); + int numRows = indexToPersist.getIndex().numRows(); // since the sink may have been persisted before it may have lost its // hydrant count, we remember that value in the sinks' metadata, so we have diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index ec63b589a79..2519cd1e921 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1647,7 +1647,7 @@ public class StreamAppenderator implements Appenderator try { final long startTime = System.nanoTime(); - int numRows = indexToPersist.getIndex().size(); + int numRows = indexToPersist.getIndex().numRows(); final File persistedFile; final File persistDir = createPersistDirIfNeeded(identifier); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java index 98c5e11fa0b..702bc3e77ad 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java @@ -231,7 +231,7 @@ public class Sink implements Iterable, Overshadowable public boolean swappable() { synchronized (hydrantLock) { - return writable && currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0; + return writable && currHydrant.getIndex() != null && currHydrant.getIndex().numRows() != 0; } } @@ -286,7 +286,7 @@ public class Sink implements Iterable, Overshadowable return 0; } - return index.size(); + return index.numRows(); } } @@ -327,6 +327,7 @@ public class Sink implements Iterable, Overshadowable .withDimensionsSpec(schema.getDimensionsSpec()) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) + .withProjections(schema.getProjections()) .build(); // Build the incremental-index according to the spec that was chosen by the user @@ -381,7 +382,7 @@ public class Sink implements Iterable, Overshadowable } currHydrant = new FireHydrant(newIndex, newCount, getSegment().getId()); if (old != null) { - numRowsExcludingCurrIndex.addAndGet(old.getIndex().size()); + numRowsExcludingCurrIndex.addAndGet(old.getIndex().numRows()); } hydrants.add(currHydrant); } else { @@ -448,7 +449,7 @@ public class Sink implements Iterable, Overshadowable public boolean apply(FireHydrant input) { final IncrementalIndex index = input.getIndex(); - return index == null || index.size() != 0; + return index == null || index.numRows() != 0; } } ); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java index b1050903624..326ad6dbed7 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java @@ -56,6 +56,7 @@ public class TestModifiedDataSchema extends DataSchema aggregators, granularitySpec, transformSpec, + null, parserMap, objectMapper );