Projections prototype (#17214)

This commit is contained in:
Clint Wylie 2024-10-05 04:38:57 -07:00 committed by GitHub
parent 04fe56835d
commit 0bd13bcd51
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
120 changed files with 5379 additions and 628 deletions

View File

@ -278,6 +278,26 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory
&& stringEncoding == that.stringEncoding; && 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 @Override
public int hashCode() public int hashCode()
{ {

View File

@ -231,6 +231,42 @@ public class HllSketchAggregatorFactoryTest
Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey()); 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 @Test
public void testToString() public void testToString()
{ {

View File

@ -296,7 +296,7 @@ public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest
frameReader = FrameReader.create(adapter.getRowSignature()); frameReader = FrameReader.create(adapter.getRowSignature());
frameList = FrameSequenceBuilder.fromCursorFactory(adapter) frameList = FrameSequenceBuilder.fromCursorFactory(adapter)
.frameType(FrameType.ROW_BASED) .frameType(FrameType.ROW_BASED)
.maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) .maxRowsPerFrame(IntMath.divide(index.numRows(), MAX_FRAMES, RoundingMode.CEILING))
.frames() .frames()
.toList(); .toList();
} }

View File

@ -165,7 +165,7 @@ public class DatasourceRecordReaderSegmentReaderTest
Assert.assertEquals(18, count); Assert.assertEquals(18, count);
// Check the index // Check the index
Assert.assertEquals(9, index.size()); Assert.assertEquals(9, index.numRows());
final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test")); final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test"));
final List<String> dimensions = index.getDimensionNames(false); final List<String> dimensions = index.getDimensionNames(false);
Assert.assertEquals(2, dimensions.size()); Assert.assertEquals(2, dimensions.size());

View File

@ -1913,6 +1913,7 @@ public class CompactionTaskTest
null, null,
null, null,
null, null,
null,
null null
); );
} }

View File

@ -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<DimensionSchema> groupingColumns;
private final VirtualColumns virtualColumns;
private final AggregatorFactory[] aggregators;
private final List<OrderBy> ordering;
@Nullable
private final String timeColumnName;
@JsonCreator
public AggregateProjectionSpec(
@JsonProperty("name") String name,
@JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns,
@JsonProperty("groupingColumns") @Nullable List<DimensionSchema> 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<DimensionSchema> getGroupingColumns()
{
return groupingColumns;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public AggregatorFactory[] getAggregators()
{
return aggregators;
}
@JsonProperty
public List<OrderBy> 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<DimensionSchema> groupingColumns)
{
final List<OrderBy> 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<OrderBy> ordering;
@Nullable
private final String timeColumnName;
private ProjectionOrdering(List<OrderBy> ordering, @Nullable String timeColumnName)
{
this.ordering = ordering;
this.timeColumnName = timeColumnName;
}
}
}

View File

@ -23,9 +23,11 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Query; import org.apache.druid.query.Query;
import org.apache.druid.query.expression.TimestampFloorExprMacro; import org.apache.druid.query.expression.TimestampFloorExprMacro;
import org.apache.druid.segment.AggregateProjectionMetadata;
import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.VirtualColumns;
@ -160,4 +162,32 @@ public class Granularities
ExprMacroTable.granularity() ExprMacroTable.granularity()
); );
} }
/**
* Converts a virtual column with a single input time column into a {@link Granularity} if it is a
* {@link TimestampFloorExprMacro.TimestampFloorExpr}.
* <p>
* 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()}.
* <p>
* 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;
}
} }

View File

@ -190,6 +190,12 @@ public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMet
setDimension("segment", segmentIdentifier); setDimension("segment", segmentIdentifier);
} }
@Override
public void projection(String projection)
{
setDimension("projection", projection);
}
@Override @Override
public void identity(String identity) public void identity(String identity)
{ {

View File

@ -88,6 +88,12 @@ public class QueryContexts
public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit"; public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit";
public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold"; public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold";
public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled"; 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 // Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the
// query's runtime // query's runtime
public static final String QUERY_RESOURCE_ID = "queryResourceId"; public static final String QUERY_RESOURCE_ID = "queryResourceId";

View File

@ -243,6 +243,11 @@ public interface QueryMetrics<QueryType extends Query<?>>
void segment(String segmentIdentifier); 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 * @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were
* used to construct {@link org.apache.druid.segment.BitmapOffset} or * used to construct {@link org.apache.druid.segment.BitmapOffset} or

View File

@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.query.PerSegmentQueryOptimizationContext;
import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory; 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.ColumnType;
import org.apache.druid.segment.column.ColumnTypeFactory; import org.apache.druid.segment.column.ColumnTypeFactory;
import org.apache.druid.segment.column.ValueType; 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()); 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.
* <p>
* 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 * Merges the list of AggregatorFactory[] (presumable from metadata of some segments being merged) and
* returns merged AggregatorFactory[] (for the metadata for merged segment). * returns merged AggregatorFactory[] (for the metadata for merged segment).

View File

@ -454,13 +454,4 @@ public class AggregatorUtil
} }
return false; return false;
} }
public static List<AggregatorFactory> getCombiningAggregators(List<AggregatorFactory> aggs)
{
List<AggregatorFactory> combining = new ArrayList<>(aggs.size());
for (AggregatorFactory agg : aggs) {
combining.add(agg.getCombiningFactory());
}
return combining;
}
} }

View File

@ -20,6 +20,7 @@
package org.apache.druid.query.groupby; package org.apache.druid.query.groupby;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.base.Suppliers; import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList; 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.ResourceLimitExceededException;
import org.apache.druid.query.ResultMergeQueryRunner; import org.apache.druid.query.ResultMergeQueryRunner;
import org.apache.druid.query.aggregation.AggregatorFactory; 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.aggregation.PostAggregator;
import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DefaultDimensionSpec;
@ -509,7 +509,7 @@ public class GroupingEngine
final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec)); final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec));
if (cursorHolder.isPreAggregated()) { if (cursorHolder.isPreAggregated()) {
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
} }
final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory); final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory);

View File

@ -151,6 +151,12 @@ public class DefaultSearchQueryMetrics implements SearchQueryMetrics
delegateQueryMetrics.segment(segmentIdentifier); delegateQueryMetrics.segment(segmentIdentifier);
} }
@Override
public void projection(String projection)
{
delegateQueryMetrics.projection(projection);
}
@Override @Override
public void filterBundle(FilterBundle.BundleInfo bundleInfo) public void filterBundle(FilterBundle.BundleInfo bundleInfo)
{ {

View File

@ -20,6 +20,7 @@
package org.apache.druid.query.timeseries; package org.apache.druid.query.timeseries;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.inject.Inject; import com.google.inject.Inject;
import org.apache.druid.collections.NonBlockingPool; 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.Aggregator;
import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory; 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.query.vector.VectorCursorGranularizer;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor; import org.apache.druid.segment.Cursor;
@ -104,7 +104,7 @@ public class TimeseriesQueryEngine
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics));
if (cursorHolder.isPreAggregated()) { if (cursorHolder.isPreAggregated()) {
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
} }
try { try {
final Sequence<Result<TimeseriesResultValue>> result; final Sequence<Result<TimeseriesResultValue>> result;

View File

@ -19,6 +19,7 @@
package org.apache.druid.query.topn; package org.apache.druid.query.topn;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicates; import com.google.common.base.Predicates;
import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ResourceHolder; 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.QueryMetrics;
import org.apache.druid.query.Result; import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.AggregatorFactory; 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.extraction.ExtractionFn;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory;
@ -89,7 +89,7 @@ public class TopNQueryEngine
final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics);
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec);
if (cursorHolder.isPreAggregated()) { if (cursorHolder.isPreAggregated()) {
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
} }
final Cursor cursor = cursorHolder.asCursor(); final Cursor cursor = cursorHolder.asCursor();
if (cursor == null) { if (cursor == null) {

View File

@ -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.
* <p>
* 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> SCHEMA_INTERNER = Interners.newWeakInterner();
public static final Comparator<AggregateProjectionMetadata> 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<Schema> 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<String> groupingColumns;
private final AggregatorFactory[] aggregators;
private final List<OrderBy> ordering;
private final List<OrderBy> 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<String> groupingColumns,
@JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators,
@JsonProperty("ordering") List<OrderBy> 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<String> getGroupingColumns()
{
return groupingColumns;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public AggregatorFactory[] getAggregators()
{
return aggregators;
}
@JsonProperty
public List<OrderBy> getOrdering()
{
return ordering;
}
@JsonIgnore
public List<OrderBy> 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<String> 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)}.
* <p>
* 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<String> 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
);
}
}
}

View File

@ -27,6 +27,7 @@ import org.apache.druid.common.guava.GuavaUtils;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE; 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.UOE;
import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Evals;
@ -360,8 +361,42 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
return rootLiteralSelector; return rootLiteralSelector;
} }
return new ObjectColumnSelector<StructuredData>() return new ColumnValueSelector<Object>()
{ {
@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 @Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector) public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{ {

View File

@ -75,6 +75,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
SimpleDictionaryMergingIterator.makePeekingComparator(); SimpleDictionaryMergingIterator.makePeekingComparator();
private final String name; private final String name;
private final String outputName;
private final IndexSpec indexSpec; private final IndexSpec indexSpec;
private final SegmentWriteOutMedium segmentWriteOutMedium; private final SegmentWriteOutMedium segmentWriteOutMedium;
private final Closer closer; private final Closer closer;
@ -84,9 +85,23 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
@Nullable @Nullable
private final ColumnType castToType; private final ColumnType castToType;
private boolean isVariantType = false; 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( public AutoTypeColumnMerger(
String name, String name,
String outputName,
@Nullable ColumnType castToType, @Nullable ColumnType castToType,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
@ -95,6 +110,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
{ {
this.name = name; this.name = name;
this.outputName = outputName;
this.castToType = castToType; this.castToType = castToType;
this.indexSpec = indexSpec; this.indexSpec = indexSpec;
this.segmentWriteOutMedium = segmentWriteOutMedium; this.segmentWriteOutMedium = segmentWriteOutMedium;
@ -165,7 +181,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) { if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) {
logicalType = ColumnType.STRING; logicalType = ColumnType.STRING;
serializer = new ScalarStringColumnSerializer( serializer = new ScalarStringColumnSerializer(
name, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
closer closer
@ -179,7 +195,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
switch (logicalType.getType()) { switch (logicalType.getType()) {
case LONG: case LONG:
serializer = new ScalarLongColumnSerializer( serializer = new ScalarLongColumnSerializer(
name, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
closer closer
@ -187,7 +203,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
break; break;
case DOUBLE: case DOUBLE:
serializer = new ScalarDoubleColumnSerializer( serializer = new ScalarDoubleColumnSerializer(
name, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
closer closer
@ -195,7 +211,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
break; break;
case STRING: case STRING:
serializer = new ScalarStringColumnSerializer( serializer = new ScalarStringColumnSerializer(
name, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
closer closer
@ -203,7 +219,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
break; break;
case ARRAY: case ARRAY:
serializer = new VariantColumnSerializer( serializer = new VariantColumnSerializer(
name, outputName,
logicalType, logicalType,
null, null,
indexSpec, indexSpec,
@ -222,6 +238,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
// mixed type column, but only root path, we can use VariantArrayColumnSerializer // mixed type column, but only root path, we can use VariantArrayColumnSerializer
// pick the least restrictive type for the logical type // pick the least restrictive type for the logical type
isVariantType = true; isVariantType = true;
variantTypeByte = rootTypes.getByteValue();
for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) { for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) {
logicalType = ColumnType.leastRestrictiveType(logicalType, type); logicalType = ColumnType.leastRestrictiveType(logicalType, type);
} }
@ -230,9 +247,9 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType); logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType);
} }
serializer = new VariantColumnSerializer( serializer = new VariantColumnSerializer(
name, outputName,
null, null,
rootTypes.getByteValue(), variantTypeByte,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
closer closer
@ -241,7 +258,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
// all the bells and whistles // all the bells and whistles
logicalType = ColumnType.NESTED_DATA; logicalType = ColumnType.NESTED_DATA;
serializer = new NestedDataColumnSerializer( serializer = new NestedDataColumnSerializer(
name, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
closer closer
@ -262,7 +279,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
sortedLookup.getSortedDoubles(), sortedLookup.getSortedDoubles(),
() -> new ArrayDictionaryMergingIterator( () -> new ArrayDictionaryMergingIterator(
sortedArrayLookups, sortedArrayLookups,
serializer.getGlobalLookup() serializer.getDictionaryIdLookup()
) )
); );
stringCardinality = sortedLookup.getStringCardinality(); stringCardinality = sortedLookup.getStringCardinality();
@ -284,7 +301,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
); );
final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator( final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
sortedArrayLookups, sortedArrayLookups,
serializer.getGlobalLookup() serializer.getDictionaryIdLookup()
); );
serializer.serializeDictionaries( serializer.serializeDictionaries(
() -> stringIterator, () -> stringIterator,
@ -367,6 +384,76 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
return descriptorBuilder.build(); return descriptorBuilder.build();
} }
protected DictionaryIdLookup getIdLookup()
{
return serializer.getDictionaryIdLookup();
}
@Override
public void attachParent(DimensionMergerV9 parent, List<IndexableAdapter> 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<int[]> public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
{ {
private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR = private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =

View File

@ -26,6 +26,7 @@ import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.Filter;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -54,10 +55,12 @@ public class CursorBuildSpec
private final VirtualColumns virtualColumns; private final VirtualColumns virtualColumns;
@Nullable @Nullable
private final List<AggregatorFactory> aggregators; private final List<AggregatorFactory> aggregators;
private final List<OrderBy> orderByColumns; private final List<OrderBy> preferredOrdering;
private final QueryContext queryContext; private final QueryContext queryContext;
private final boolean isAggregate;
@Nullable @Nullable
private final QueryMetrics<?> queryMetrics; private final QueryMetrics<?> queryMetrics;
@ -77,9 +80,10 @@ public class CursorBuildSpec
this.groupingColumns = groupingColumns; this.groupingColumns = groupingColumns;
this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns"); this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns");
this.aggregators = aggregators; this.aggregators = aggregators;
this.orderByColumns = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering"); this.preferredOrdering = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering");
this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext"); this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext");
this.queryMetrics = queryMetrics; this.queryMetrics = queryMetrics;
this.isAggregate = !CollectionUtils.isNullOrEmpty(groupingColumns) || !CollectionUtils.isNullOrEmpty(aggregators);
} }
/** /**
@ -142,7 +146,7 @@ public class CursorBuildSpec
*/ */
public List<OrderBy> getPreferredOrdering() public List<OrderBy> getPreferredOrdering()
{ {
return orderByColumns; return preferredOrdering;
} }
/** /**
@ -166,6 +170,39 @@ public class CursorBuildSpec
return queryMetrics; 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<OrderBy> 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 public static class CursorBuildSpecBuilder
{ {
@Nullable @Nullable
@ -195,7 +232,7 @@ public class CursorBuildSpec
this.groupingColumns = buildSpec.groupingColumns; this.groupingColumns = buildSpec.groupingColumns;
this.virtualColumns = buildSpec.virtualColumns; this.virtualColumns = buildSpec.virtualColumns;
this.aggregators = buildSpec.aggregators; this.aggregators = buildSpec.aggregators;
this.preferredOrdering = buildSpec.orderByColumns; this.preferredOrdering = buildSpec.preferredOrdering;
this.queryContext = buildSpec.queryContext; this.queryContext = buildSpec.queryContext;
this.queryMetrics = buildSpec.queryMetrics; this.queryMetrics = buildSpec.queryMetrics;
} }

View File

@ -75,6 +75,18 @@ public interface CursorHolder extends Closeable
return false; 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<AggregatorFactory> getAggregatorsForPreAggregated()
{
return null;
}
/** /**
* Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns * Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns
* an empty list then the cursor has no defined ordering. * an empty list then the cursor has no defined ordering.

View File

@ -70,6 +70,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class); private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class);
protected final String dimensionName; protected final String dimensionName;
protected final String outputName;
protected final ProgressIndicator progress; protected final ProgressIndicator progress;
protected final Closer closer; protected final Closer closer;
protected final IndexSpec indexSpec; protected final IndexSpec indexSpec;
@ -81,6 +82,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
protected int rowCount = 0; protected int rowCount = 0;
protected int cardinality = 0; protected int cardinality = 0;
protected boolean hasNull = false; protected boolean hasNull = false;
protected boolean writeDictionary = true;
@Nullable @Nullable
protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter; protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
@ -102,6 +104,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
public DictionaryEncodedColumnMerger( public DictionaryEncodedColumnMerger(
String dimensionName, String dimensionName,
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -110,6 +113,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
) )
{ {
this.dimensionName = dimensionName; this.dimensionName = dimensionName;
this.outputName = outputName;
this.indexSpec = indexSpec; this.indexSpec = indexSpec;
this.capabilities = capabilities; this.capabilities = capabilities;
this.segmentWriteOutMedium = segmentWriteOutMedium; this.segmentWriteOutMedium = segmentWriteOutMedium;
@ -171,8 +175,9 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
numMergeIndex++; numMergeIndex++;
} }
String dictFilename = StringUtils.format("%s.dim_values", dimensionName); String dictFilename = StringUtils.format("%s.dim_values", outputName);
dictionaryWriter = makeDictionaryWriter(dictFilename); dictionaryWriter = makeDictionaryWriter(dictFilename);
firstDictionaryValue = null; firstDictionaryValue = null;
dictionarySize = 0; dictionarySize = 0;
dictionaryWriter.open(); dictionaryWriter.open();
@ -338,7 +343,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
long dimStartTime = System.currentTimeMillis(); long dimStartTime = System.currentTimeMillis();
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
String bmpFilename = StringUtils.format("%s.inverted", dimensionName); String bmpFilename = StringUtils.format("%s.inverted", outputName);
bitmapWriter = new GenericIndexedWriter<>( bitmapWriter = new GenericIndexedWriter<>(
segmentWriteOutMedium, segmentWriteOutMedium,
bmpFilename, bmpFilename,
@ -402,11 +407,11 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
{ {
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); 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 (capabilities.hasMultipleValues().isTrue()) {
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create( encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create(
dimensionName, outputName,
segmentWriteOutMedium, segmentWriteOutMedium,
filenameBase, filenameBase,
cardinality, cardinality,
@ -414,12 +419,12 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
); );
} else { } else {
encodedValueSerializer = encodedValueSerializer =
new VSizeColumnarMultiIntsSerializer(dimensionName, segmentWriteOutMedium, cardinality); new VSizeColumnarMultiIntsSerializer(outputName, segmentWriteOutMedium, cardinality);
} }
} else { } else {
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create(
dimensionName, outputName,
segmentWriteOutMedium, segmentWriteOutMedium,
filenameBase, filenameBase,
cardinality, cardinality,

View File

@ -106,13 +106,17 @@ public interface DimensionHandler
* *
* See {@link DimensionMergerV9} interface for more information. * See {@link DimensionMergerV9} interface for more information.
* *
* @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 indexSpec Specification object for the index merge
* @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if needed * @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 capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler
* @param progress ProgressIndicator used by the merging process * @param progress ProgressIndicator used by the merging process
* @return A new DimensionMergerV9 object. * @return A new DimensionMergerV9 object.
*/ */
DimensionMergerV9 makeMerger( DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,

View File

@ -21,6 +21,9 @@ package org.apache.druid.segment;
import org.apache.druid.segment.column.ColumnDescriptor; import org.apache.druid.segment.column.ColumnDescriptor;
import java.io.IOException;
import java.util.List;
/** /**
* Processing related interface * Processing related interface
* *
@ -35,4 +38,13 @@ public interface DimensionMergerV9 extends DimensionMerger
* @return ColumnDescriptor that IndexMergerV9 will use to build a column. * @return ColumnDescriptor that IndexMergerV9 will use to build a column.
*/ */
ColumnDescriptor makeColumnDescriptor(); 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<IndexableAdapter> projectionAdapters) throws IOException
{
// do nothing
}
} }

View File

@ -77,6 +77,7 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
@Override @Override
public DimensionMergerV9 makeMerger( public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -85,7 +86,7 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
) )
{ {
return new DoubleDimensionMergerV9( return new DoubleDimensionMergerV9(
dimensionName, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium segmentWriteOutMedium
); );

View File

@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9 public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9
{ {
DoubleDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) DoubleDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
{ {
super(dimensionName, indexSpec, segmentWriteOutMedium); super(outputName, indexSpec, segmentWriteOutMedium);
} }
@Override @Override
GenericColumnSerializer setupEncodedValueWriter() GenericColumnSerializer setupEncodedValueWriter()
{ {
return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
} }
@Override @Override

View File

@ -77,6 +77,7 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
@Override @Override
public DimensionMergerV9 makeMerger( public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -85,7 +86,7 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
) )
{ {
return new FloatDimensionMergerV9( return new FloatDimensionMergerV9(
dimensionName, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium segmentWriteOutMedium
); );

View File

@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
public class FloatDimensionMergerV9 extends NumericDimensionMergerV9 public class FloatDimensionMergerV9 extends NumericDimensionMergerV9
{ {
FloatDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) FloatDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
{ {
super(dimensionName, indexSpec, segmentWriteOutMedium); super(outputName, indexSpec, segmentWriteOutMedium);
} }
@Override @Override
GenericColumnSerializer setupEncodedValueWriter() GenericColumnSerializer setupEncodedValueWriter()
{ {
return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
} }
@Override @Override

View File

@ -48,6 +48,7 @@ import org.apache.druid.java.util.common.io.smoosh.Smoosh;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnConfig;
@ -64,6 +65,7 @@ import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.data.IndexedIterable; import org.apache.druid.segment.data.IndexedIterable;
import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.data.ListIndexed;
import org.apache.druid.segment.data.VSizeColumnarMultiInts; import org.apache.druid.segment.data.VSizeColumnarMultiInts;
import org.apache.druid.segment.projections.Projections;
import org.apache.druid.segment.serde.ComplexColumnPartSupplier; import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
import org.apache.druid.segment.serde.FloatNumericColumnSupplier; import org.apache.druid.segment.serde.FloatNumericColumnSupplier;
import org.apache.druid.segment.serde.LongNumericColumnSupplier; import org.apache.druid.segment.serde.LongNumericColumnSupplier;
@ -621,6 +623,7 @@ public class IndexIO
mapper, mapper,
timeBuffer, timeBuffer,
smooshedFiles, smooshedFiles,
null,
loadFailed loadFailed
); );
@ -643,6 +646,22 @@ public class IndexIO
smooshedFiles, smooshedFiles,
loadFailed loadFailed
); );
final Map<String, Map<String, Supplier<ColumnHolder>>> projectionsColumns = new LinkedHashMap<>();
final Metadata metadata = getMetdata(smooshedFiles, mapper, inDir);
if (metadata != null && metadata.getProjections() != null) {
for (AggregateProjectionMetadata projectionSpec : metadata.getProjections()) {
final Map<String, Supplier<ColumnHolder>> projectionColumns = readProjectionColumns(
mapper,
loadFailed,
projectionSpec,
smooshedFiles,
columns,
dataInterval
);
projectionsColumns.put(projectionSpec.getSchema().getName(), projectionColumns);
}
}
final QueryableIndex index = new SimpleQueryableIndex( final QueryableIndex index = new SimpleQueryableIndex(
dataInterval, dataInterval,
@ -650,11 +669,83 @@ public class IndexIO
segmentBitmapSerdeFactory.getBitmapFactory(), segmentBitmapSerdeFactory.getBitmapFactory(),
columns, columns,
smooshedFiles, smooshedFiles,
lazy lazy,
metadata,
projectionsColumns
) )
{ {
@Override @Override
public Metadata getMetadata() public Metadata getMetadata()
{
return getMetdata(smooshedFiles, mapper, inDir);
}
};
log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
return index;
}
private Map<String, Supplier<ColumnHolder>> readProjectionColumns(
ObjectMapper mapper,
SegmentLazyLoadFailCallback loadFailed,
AggregateProjectionMetadata projectionSpec,
SmooshedFileMapper smooshedFiles,
Map<String, Supplier<ColumnHolder>> columns,
Interval dataInterval
) throws IOException
{
final Map<String, Supplier<ColumnHolder>> 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 { try {
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
@ -675,12 +766,6 @@ public class IndexIO
} }
return null; return null;
} }
};
log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
return index;
}
/** /**
* Return a list of columns that contains given inputs merged. The returned column names are in * Return a list of columns that contains given inputs merged. The returned column names are in
@ -740,6 +825,7 @@ public class IndexIO
mapper, mapper,
colBuffer, colBuffer,
smooshedFiles, smooshedFiles,
null,
loadFailed loadFailed
); );
} }
@ -752,6 +838,7 @@ public class IndexIO
ObjectMapper mapper, ObjectMapper mapper,
ByteBuffer colBuffer, ByteBuffer colBuffer,
SmooshedFileMapper smooshedFiles, SmooshedFileMapper smooshedFiles,
@Nullable ColumnHolder parentColumn,
SegmentLazyLoadFailCallback loadFailed SegmentLazyLoadFailCallback loadFailed
) throws IOException ) throws IOException
{ {
@ -767,7 +854,8 @@ public class IndexIO
internedColumnName, internedColumnName,
mapper, mapper,
colBuffer, colBuffer,
smooshedFiles smooshedFiles,
parentColumn
); );
} }
catch (IOException | RuntimeException e) { catch (IOException | RuntimeException e) {
@ -782,7 +870,8 @@ public class IndexIO
internedColumnName, internedColumnName,
mapper, mapper,
colBuffer, colBuffer,
smooshedFiles smooshedFiles,
parentColumn
); );
columns.put(internedColumnName, () -> columnHolder); columns.put(internedColumnName, () -> columnHolder);
} }
@ -797,11 +886,12 @@ public class IndexIO
String columnName, // columnName is not used in this method, but used in tests. String columnName, // columnName is not used in this method, but used in tests.
ObjectMapper mapper, ObjectMapper mapper,
ByteBuffer byteBuffer, ByteBuffer byteBuffer,
SmooshedFileMapper smooshedFiles SmooshedFileMapper smooshedFiles,
@Nullable ColumnHolder parentColumn
) throws IOException ) throws IOException
{ {
ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class); ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class);
return serde.read(byteBuffer, columnConfig, smooshedFiles); return serde.read(byteBuffer, columnConfig, smooshedFiles, parentColumn);
} }
} }

View File

@ -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" // 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. // encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9.
//noinspection ObjectEquality //noinspection ObjectEquality
anySelectorChanged |= convertedDimensionSelector != sourceDimensionSelector; anySelectorChanged = anySelectorChanged || convertedDimensionSelector != sourceDimensionSelector;
convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues( convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues(
indexNumber, indexNumber,

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import com.google.inject.Inject; 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.incremental.IncrementalIndexAdapter;
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
import org.apache.druid.segment.loading.SegmentizerFactory; 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.ColumnPartSerde;
import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.serde.ComplexColumnPartSerde;
import org.apache.druid.segment.serde.ComplexMetricSerde; 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.serde.NullColumnPartSerde;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -138,7 +141,6 @@ public class IndexMergerV9 implements IndexMerger
final DimensionsSpecInspector dimensionsSpecInspector, final DimensionsSpecInspector dimensionsSpecInspector,
final List<String> mergedMetrics, final List<String> mergedMetrics,
final Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn, final Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn,
final boolean fillRowNumConversions,
final IndexSpec indexSpec, final IndexSpec indexSpec,
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException ) throws IOException
@ -154,7 +156,7 @@ public class IndexMergerV9 implements IndexMerger
.filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim)) .filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim))
.collect(Collectors.toList()); .collect(Collectors.toList());
final Metadata segmentMetadata; Metadata segmentMetadata;
if (metricAggs != null) { if (metricAggs != null) {
AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length]; AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length];
for (int i = 0; i < metricAggs.length; i++) { for (int i = 0; i < metricAggs.length; i++) {
@ -215,18 +217,20 @@ public class IndexMergerV9 implements IndexMerger
mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats); mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats);
final Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimFormats); final Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimFormats);
final Map<String, DimensionMergerV9> mergersMap = Maps.newHashMapWithExpectedSize(mergedDimensions.size());
final List<DimensionMergerV9> mergers = new ArrayList<>(); final List<DimensionMergerV9> mergers = new ArrayList<>();
for (int i = 0; i < mergedDimensions.size(); i++) { for (int i = 0; i < mergedDimensions.size(); i++) {
DimensionHandler handler = handlers.get(mergedDimensions.get(i)); DimensionHandler handler = handlers.get(mergedDimensions.get(i));
mergers.add( DimensionMergerV9 merger = handler.makeMerger(
handler.makeMerger( mergedDimensions.get(i),
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
dimFormats.get(i).toColumnCapabilities(), dimFormats.get(i).toColumnCapabilities(),
progress, progress,
closer closer
)
); );
mergers.add(merger);
mergersMap.put(mergedDimensions.get(i), merger);
} }
/************* Setup Dim Conversions **************/ /************* Setup Dim Conversions **************/
@ -255,8 +259,7 @@ public class IndexMergerV9 implements IndexMerger
timeAndDimsIterator, timeAndDimsIterator,
timeWriter, timeWriter,
metricWriters, metricWriters,
mergers, mergers
fillRowNumConversions
); );
/************ Create Inverted Indexes and Finalize Build Columns *************/ /************ Create Inverted Indexes and Finalize Build Columns *************/
@ -291,6 +294,20 @@ public class IndexMergerV9 implements IndexMerger
progress.stopSection(section); 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 **************/ /************* Make index.drd & metadata.drd files **************/
progress.progress(); progress.progress();
makeIndexBinary( makeIndexBinary(
@ -332,6 +349,193 @@ public class IndexMergerV9 implements IndexMerger
} }
} }
private Metadata makeProjections(
final FileSmoosher smoosher,
final List<AggregateProjectionMetadata> projections,
final List<IndexableAdapter> adapters,
final IndexSpec indexSpec,
final SegmentWriteOutMedium segmentWriteOutMedium,
final ProgressIndicator progress,
final Closer closer,
final Map<String, DimensionMergerV9> parentMergers,
final Metadata segmentMetadata
) throws IOException
{
final List<AggregateProjectionMetadata> projectionMetadata = Lists.newArrayListWithCapacity(projections.size());
for (AggregateProjectionMetadata spec : projections) {
final List<IndexableAdapter> 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<String> dimensions = projectionAdapters.get(0).getDimensionNames(false);
final List<String> metrics = Arrays.stream(projectionSchema.getAggregators())
.map(AggregatorFactory::getName)
.collect(Collectors.toList());
final List<DimensionMergerV9> mergers = new ArrayList<>();
final Map<String, ColumnFormat> 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<GenericColumnSerializer> metricWriters =
setupMetricsWriters(
segmentWriteOutMedium,
metrics,
columnFormats,
indexSpec,
Projections.getProjectionSmooshV9Prefix(spec)
);
Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn =
rowIterators -> new RowCombiningTimeAndDimsIterator(rowIterators, projectionSchema.getAggregators(), metrics);
List<TransformableRowIterator> 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<IntBuffer> 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( private void makeIndexBinary(
final FileSmoosher v9Smoosher, final FileSmoosher v9Smoosher,
final List<IndexableAdapter> adapters, final List<IndexableAdapter> adapters,
@ -348,7 +552,7 @@ public class IndexMergerV9 implements IndexMerger
columnSet.addAll(mergedMetrics); columnSet.addAll(mergedMetrics);
Preconditions.checkState( Preconditions.checkState(
columnSet.size() == mergedDimensions.size() + mergedMetrics.size(), 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, mergedDimensions,
mergedMetrics mergedMetrics
); );
@ -444,6 +648,18 @@ public class IndexMergerV9 implements IndexMerger
final List<GenericColumnSerializer> metWriters, final List<GenericColumnSerializer> metWriters,
final IndexSpec indexSpec final IndexSpec indexSpec
) throws IOException ) throws IOException
{
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsTypes, metWriters, indexSpec, "");
}
private void makeMetricsColumns(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final List<String> mergedMetrics,
final Map<String, ColumnFormat> metricsTypes,
final List<GenericColumnSerializer> metWriters,
final IndexSpec indexSpec,
final String namePrefix
) throws IOException
{ {
final String section = "make metric columns"; final String section = "make metric columns";
progress.startSection(section); progress.startSection(section);
@ -483,8 +699,9 @@ public class IndexMergerV9 implements IndexMerger
default: default:
throw new ISE("Unknown type[%s]", type); throw new ISE("Unknown type[%s]", type);
} }
makeColumn(v9Smoosher, metric, builder.build()); final String columnName = namePrefix + metric;
log.debug("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime); 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); log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section); progress.stopSection(section);
@ -547,6 +764,17 @@ public class IndexMergerV9 implements IndexMerger
final GenericColumnSerializer timeWriter, final GenericColumnSerializer timeWriter,
final IndexSpec indexSpec final IndexSpec indexSpec
) throws IOException ) 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"; final String section = "make time column";
progress.startSection(section); progress.startSection(section);
@ -557,7 +785,7 @@ public class IndexMergerV9 implements IndexMerger
.setValueType(ValueType.LONG) .setValueType(ValueType.LONG)
.addSerde(createLongColumnPartSerde(timeWriter, indexSpec)) .addSerde(createLongColumnPartSerde(timeWriter, indexSpec))
.build(); .build();
makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator); makeColumn(v9Smoosher, name, serdeficator);
log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section); progress.stopSection(section);
} }
@ -601,24 +829,20 @@ public class IndexMergerV9 implements IndexMerger
final TimeAndDimsIterator timeAndDimsIterator, final TimeAndDimsIterator timeAndDimsIterator,
final GenericColumnSerializer timeWriter, final GenericColumnSerializer timeWriter,
final ArrayList<GenericColumnSerializer> metricWriters, final ArrayList<GenericColumnSerializer> metricWriters,
final List<DimensionMergerV9> mergers, final List<DimensionMergerV9> mergers
final boolean fillRowNumConversions
) throws IOException ) throws IOException
{ {
final String section = "walk through and merge rows"; final String section = "walk through and merge rows";
progress.startSection(section); progress.startSection(section);
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
List<IntBuffer> rowNumConversions = null;
int rowCount = 0; int rowCount = 0;
if (fillRowNumConversions) { List<IntBuffer> rowNumConversions = new ArrayList<>(adapters.size());
rowNumConversions = new ArrayList<>(adapters.size());
for (IndexableAdapter adapter : adapters) { for (IndexableAdapter adapter : adapters) {
int[] arr = new int[adapter.getNumRows()]; int[] arr = new int[adapter.getNumRows()];
Arrays.fill(arr, INVALID_ROW); Arrays.fill(arr, INVALID_ROW);
rowNumConversions.add(IntBuffer.wrap(arr)); rowNumConversions.add(IntBuffer.wrap(arr));
} }
}
long time = System.currentTimeMillis(); long time = System.currentTimeMillis();
while (timeAndDimsIterator.moveToNext()) { while (timeAndDimsIterator.moveToNext()) {
@ -656,9 +880,7 @@ public class IndexMergerV9 implements IndexMerger
} }
conversionBuffer.put(rowCount); conversionBuffer.put(rowCount);
} }
} }
} else if (timeAndDimsIterator instanceof MergingRowIterator) { } else if (timeAndDimsIterator instanceof MergingRowIterator) {
RowPointer rowPointer = (RowPointer) timeAndDims; RowPointer rowPointer = (RowPointer) timeAndDims;
IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum()); IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum());
@ -668,23 +890,19 @@ public class IndexMergerV9 implements IndexMerger
} }
conversionBuffer.put(rowCount); conversionBuffer.put(rowCount);
} else { } else {
if (fillRowNumConversions) {
throw new IllegalStateException( throw new IllegalStateException(
"Filling row num conversions is supported only with RowCombining and Merging iterators" "Filling row num conversions is supported only with RowCombining and Merging iterators"
); );
} }
}
if ((++rowCount % 500000) == 0) { if ((++rowCount % 500000) == 0) {
log.debug("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time); log.debug("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time);
time = System.currentTimeMillis(); time = System.currentTimeMillis();
} }
} }
if (rowNumConversions != null) {
for (IntBuffer rowNumConversion : rowNumConversions) { for (IntBuffer rowNumConversion : rowNumConversions) {
rowNumConversion.rewind(); rowNumConversion.rewind();
} }
}
log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime);
progress.stopSection(section); progress.stopSection(section);
return new IndexMergeResult(rowNumConversions, rowCount); return new IndexMergeResult(rowNumConversions, rowCount);
@ -711,28 +929,40 @@ public class IndexMergerV9 implements IndexMerger
final Map<String, ColumnFormat> metricsTypes, final Map<String, ColumnFormat> metricsTypes,
final IndexSpec indexSpec final IndexSpec indexSpec
) throws IOException ) throws IOException
{
return setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsTypes, indexSpec, "");
}
private ArrayList<GenericColumnSerializer> setupMetricsWriters(
final SegmentWriteOutMedium segmentWriteOutMedium,
final List<String> mergedMetrics,
final Map<String, ColumnFormat> metricsTypes,
final IndexSpec indexSpec,
final String prefix
) throws IOException
{ {
ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
for (String metric : mergedMetrics) { for (String metric : mergedMetrics) {
TypeSignature<ValueType> type = metricsTypes.get(metric).getLogicalType(); TypeSignature<ValueType> type = metricsTypes.get(metric).getLogicalType();
final String outputName = prefix + metric;
GenericColumnSerializer writer; GenericColumnSerializer writer;
switch (type.getType()) { switch (type.getType()) {
case LONG: case LONG:
writer = createLongColumnSerializer(segmentWriteOutMedium, metric, indexSpec); writer = createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
break; break;
case FLOAT: case FLOAT:
writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, indexSpec); writer = createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
break; break;
case DOUBLE: case DOUBLE:
writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, indexSpec); writer = createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
break; break;
case COMPLEX: case COMPLEX:
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName()); ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName());
if (serde == null) { if (serde == null) {
throw new ISE("Unknown type[%s]", type.getComplexTypeName()); throw new ISE("Unknown type[%s]", type.getComplexTypeName());
} }
writer = serde.getSerializer(segmentWriteOutMedium, metric, indexSpec); writer = serde.getSerializer(segmentWriteOutMedium, outputName, indexSpec);
break; break;
default: default:
throw new ISE("Unknown type[%s]", type); throw new ISE("Unknown type[%s]", type);
@ -891,7 +1121,7 @@ public class IndexMergerV9 implements IndexMerger
FileUtils.mkdirp(outDir); 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( return multiphaseMerge(
Collections.singletonList( Collections.singletonList(
new IncrementalIndexAdapter( new IncrementalIndexAdapter(
@ -1179,7 +1409,6 @@ public class IndexMergerV9 implements IndexMerger
new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec), new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec),
mergedMetrics, mergedMetrics,
rowMergerFn, rowMergerFn,
true,
indexSpec, indexSpec,
segmentWriteOutMediumFactory segmentWriteOutMediumFactory
); );

View File

@ -64,6 +64,8 @@ public interface IndexableAdapter
TransformableRowIterator getRows(); TransformableRowIterator getRows();
IndexableAdapter getProjectionAdapter(String projection);
BitmapValues getBitmapValues(String dimension, int dictId); BitmapValues getBitmapValues(String dimension, int dictId);
ColumnCapabilities getCapabilities(String column); ColumnCapabilities getCapabilities(String column);

View File

@ -77,6 +77,7 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
@Override @Override
public DimensionMergerV9 makeMerger( public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -85,7 +86,7 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
) )
{ {
return new LongDimensionMergerV9( return new LongDimensionMergerV9(
dimensionName, outputName,
indexSpec, indexSpec,
segmentWriteOutMedium segmentWriteOutMedium
); );

View File

@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
public class LongDimensionMergerV9 extends NumericDimensionMergerV9 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 @Override
GenericColumnSerializer setupEncodedValueWriter() GenericColumnSerializer setupEncodedValueWriter()
{ {
return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
} }
@Override @Override

View File

@ -22,6 +22,7 @@ package org.apache.druid.segment;
import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.data.input.impl.TimestampSpec; 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.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
@ -56,6 +57,8 @@ public class Metadata
private final Boolean rollup; private final Boolean rollup;
@Nullable @Nullable
private final List<OrderBy> ordering; private final List<OrderBy> ordering;
@Nullable
private final List<AggregateProjectionMetadata> projections;
public Metadata( public Metadata(
@JsonProperty("container") @Nullable Map<String, Object> container, @JsonProperty("container") @Nullable Map<String, Object> container,
@ -63,7 +66,8 @@ public class Metadata
@JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec, @JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec,
@JsonProperty("queryGranularity") @Nullable Granularity queryGranularity, @JsonProperty("queryGranularity") @Nullable Granularity queryGranularity,
@JsonProperty("rollup") @Nullable Boolean rollup, @JsonProperty("rollup") @Nullable Boolean rollup,
@JsonProperty("ordering") @Nullable List<OrderBy> ordering @JsonProperty("ordering") @Nullable List<OrderBy> ordering,
@JsonProperty("projections") @Nullable List<AggregateProjectionMetadata> projections
) )
{ {
this.container = container == null ? new ConcurrentHashMap<>() : container; this.container = container == null ? new ConcurrentHashMap<>() : container;
@ -72,6 +76,7 @@ public class Metadata
this.queryGranularity = queryGranularity; this.queryGranularity = queryGranularity;
this.rollup = rollup; this.rollup = rollup;
this.ordering = ordering; this.ordering = ordering;
this.projections = projections;
} }
@JsonProperty @JsonProperty
@ -124,6 +129,27 @@ public class Metadata
return ordering; return ordering;
} }
@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_EMPTY)
public List<AggregateProjectionMetadata> getProjections()
{
return projections;
}
public Metadata withProjections(List<AggregateProjectionMetadata> projections)
{
return new Metadata(
container,
aggregators,
timestampSpec,
queryGranularity,
rollup,
ordering,
projections
);
}
public Metadata putAll(@Nullable Map<String, Object> other) public Metadata putAll(@Nullable Map<String, Object> other)
{ {
if (other != null) { if (other != null) {
@ -155,6 +181,7 @@ public class Metadata
List<Granularity> gransToMerge = new ArrayList<>(); List<Granularity> gransToMerge = new ArrayList<>();
List<Boolean> rollupToMerge = new ArrayList<>(); List<Boolean> rollupToMerge = new ArrayList<>();
List<List<OrderBy>> orderingsToMerge = new ArrayList<>(); List<List<OrderBy>> orderingsToMerge = new ArrayList<>();
List<List<AggregateProjectionMetadata>> projectionsToMerge = new ArrayList<>();
for (Metadata metadata : toBeMerged) { for (Metadata metadata : toBeMerged) {
if (metadata != null) { if (metadata != null) {
@ -176,6 +203,7 @@ public class Metadata
} }
orderingsToMerge.add(metadata.getOrdering()); orderingsToMerge.add(metadata.getOrdering());
projectionsToMerge.add(metadata.getProjections());
mergedContainer.putAll(metadata.container); mergedContainer.putAll(metadata.container);
} else { } else {
//if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then //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); Granularity.mergeGranularities(gransToMerge);
final List<OrderBy> mergedOrdering = mergeOrderings(orderingsToMerge); final List<OrderBy> mergedOrdering = mergeOrderings(orderingsToMerge);
validateProjections(projectionsToMerge);
Boolean rollup = null; Boolean rollup = null;
if (rollupToMerge != null && !rollupToMerge.isEmpty()) { if (rollupToMerge != null && !rollupToMerge.isEmpty()) {
@ -227,7 +256,8 @@ public class Metadata
mergedTimestampSpec, mergedTimestampSpec,
mergedGranularity, mergedGranularity,
rollup, 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(timestampSpec, metadata.timestampSpec) &&
Objects.equals(queryGranularity, metadata.queryGranularity) && Objects.equals(queryGranularity, metadata.queryGranularity) &&
Objects.equals(rollup, metadata.rollup) && Objects.equals(rollup, metadata.rollup) &&
Objects.equals(ordering, metadata.ordering); Objects.equals(ordering, metadata.ordering) &&
Objects.equals(projections, metadata.projections);
} }
@Override @Override
public int hashCode() 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 @Override
@ -265,6 +296,7 @@ public class Metadata
", queryGranularity=" + queryGranularity + ", queryGranularity=" + queryGranularity +
", rollup=" + rollup + ", rollup=" + rollup +
", ordering=" + ordering + ", ordering=" + ordering +
", projections=" + projections +
'}'; '}';
} }
@ -308,4 +340,34 @@ public class Metadata
mergedOrdering.add(orderBy); mergedOrdering.add(orderBy);
} }
} }
public static void validateProjections(List<List<AggregateProjectionMetadata>> projectionsToMerge)
{
final Map<String, AggregateProjectionMetadata> projectionsMap = new HashMap<>();
// dedupe by name, fail if somehow incompatible projections are defined
int nullCount = 0;
int expectedSize = -1;
for (List<AggregateProjectionMetadata> 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");
}
}
}
} }

View File

@ -77,6 +77,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler<Structu
@Override @Override
public DimensionMergerV9 makeMerger( public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -84,7 +85,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler<Structu
Closer closer Closer closer
) )
{ {
return new AutoTypeColumnMerger(name, castTo, indexSpec, segmentWriteOutMedium, closer); return new AutoTypeColumnMerger(name, outputName, castTo, indexSpec, segmentWriteOutMedium, closer);
} }
@Override @Override

View File

@ -73,6 +73,7 @@ public class NestedDataColumnHandlerV4 implements DimensionHandler<StructuredDat
@Override @Override
public DimensionMergerV9 makeMerger( public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -80,7 +81,7 @@ public class NestedDataColumnHandlerV4 implements DimensionHandler<StructuredDat
Closer closer Closer closer
) )
{ {
return new NestedDataColumnMergerV4(name, indexSpec, segmentWriteOutMedium, closer); return new NestedDataColumnMergerV4(outputName, indexSpec, segmentWriteOutMedium, closer);
} }
@Override @Override

View File

@ -31,19 +31,19 @@ import java.util.List;
*/ */
public abstract class NumericDimensionMergerV9 implements DimensionMergerV9 public abstract class NumericDimensionMergerV9 implements DimensionMergerV9
{ {
protected final String dimensionName; protected final String outputName;
protected final IndexSpec indexSpec; protected final IndexSpec indexSpec;
protected final SegmentWriteOutMedium segmentWriteOutMedium; protected final SegmentWriteOutMedium segmentWriteOutMedium;
protected final GenericColumnSerializer serializer; protected final GenericColumnSerializer serializer;
NumericDimensionMergerV9( NumericDimensionMergerV9(
String dimensionName, String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium SegmentWriteOutMedium segmentWriteOutMedium
) )
{ {
this.dimensionName = dimensionName; this.outputName = outputName;
this.indexSpec = indexSpec; this.indexSpec = indexSpec;
this.segmentWriteOutMedium = segmentWriteOutMedium; this.segmentWriteOutMedium = segmentWriteOutMedium;

View File

@ -24,6 +24,7 @@ import org.apache.druid.query.OrderBy;
import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.projections.QueryableProjection;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -85,4 +86,16 @@ public interface QueryableIndex extends Closeable, ColumnInspector
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed //@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
@Override @Override
void close(); void close();
@Nullable
default QueryableProjection<QueryableIndex> getProjection(CursorBuildSpec cursorBuildSpec)
{
return null;
}
@Nullable
default QueryableIndex getProjectionQueryableIndex(String name)
{
return null;
}
} }

View File

@ -20,13 +20,19 @@
package org.apache.druid.segment; package org.apache.druid.segment;
import org.apache.druid.query.OrderBy; 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.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature; 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 javax.annotation.Nullable;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List;
public class QueryableIndexCursorFactory implements CursorFactory public class QueryableIndexCursorFactory implements CursorFactory
{ {
@ -40,6 +46,46 @@ public class QueryableIndexCursorFactory implements CursorFactory
@Override @Override
public CursorHolder makeCursorHolder(CursorBuildSpec spec) public CursorHolder makeCursorHolder(CursorBuildSpec spec)
{ {
QueryableProjection<QueryableIndex> 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<AggregatorFactory> getAggregatorsForPreAggregated()
{
return projection.getCursorBuildSpec().getAggregators();
}
};
}
return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build()); return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build());
} }

View File

@ -214,11 +214,9 @@ public class QueryableIndexCursorHolder implements CursorHolder
} }
final Offset baseCursorOffset = offset.clone(); final Offset baseCursorOffset = offset.clone();
final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( final ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactoryForOffset(
virtualColumns, columnCache,
Cursors.getTimeOrdering(ordering), baseCursorOffset
baseCursorOffset.getBaseReadableOffset(),
columnCache
); );
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter
// needs to use a value matcher // 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, ColumnCache columnCache,
VectorOffset baseOffset VectorOffset baseOffset
) )

View File

@ -262,6 +262,14 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
return new RowIteratorImpl(); 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 * 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 * SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same

View File

@ -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<String, String> remap;
public RemapColumnSelectorFactory(ColumnSelectorFactory delegate, Map<String, String> 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));
}
}

View File

@ -24,32 +24,45 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.base.Suppliers; import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList; 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 com.google.common.collect.Maps;
import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet;
import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import org.apache.druid.query.OrderBy; import org.apache.druid.query.OrderBy;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed; 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 org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.Collections;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.SortedSet;
/** /**
* *
*/ */
public abstract class SimpleQueryableIndex implements QueryableIndex public abstract class SimpleQueryableIndex implements QueryableIndex
{ {
public static final Interner<List<OrderBy>> ORDERING_INTERNER = Interners.newWeakInterner();
private final Interval dataInterval; private final Interval dataInterval;
private final List<String> columnNames; private final List<String> columnNames;
private final Indexed<String> availableDimensions; private final Indexed<String> availableDimensions;
private final BitmapFactory bitmapFactory; private final BitmapFactory bitmapFactory;
private final Map<String, Supplier<ColumnHolder>> columns; private final Map<String, Supplier<ColumnHolder>> columns;
private final List<OrderBy> ordering;
private final Map<String, AggregateProjectionMetadata> projectionsMap;
private final SortedSet<AggregateProjectionMetadata> projections;
private final Map<String, Map<String, Supplier<ColumnHolder>>> projectionColumns;
private final SmooshedFileMapper fileMapper; private final SmooshedFileMapper fileMapper;
private final Supplier<Map<String, DimensionHandler>> dimensionHandlers; private final Supplier<Map<String, DimensionHandler>> dimensionHandlers;
private final List<OrderBy> ordering;
public SimpleQueryableIndex( public SimpleQueryableIndex(
Interval dataInterval, Interval dataInterval,
@ -59,6 +72,20 @@ public abstract class SimpleQueryableIndex implements QueryableIndex
SmooshedFileMapper fileMapper, SmooshedFileMapper fileMapper,
boolean lazy boolean lazy
) )
{
this(dataInterval, dimNames, bitmapFactory, columns, fileMapper, lazy, null, null);
}
public SimpleQueryableIndex(
Interval dataInterval,
Indexed<String> dimNames,
BitmapFactory bitmapFactory,
Map<String, Supplier<ColumnHolder>> columns,
SmooshedFileMapper fileMapper,
boolean lazy,
@Nullable Metadata metadata,
@Nullable Map<String, Map<String, Supplier<ColumnHolder>>> projectionColumns
)
{ {
Preconditions.checkNotNull(columns.get(ColumnHolder.TIME_COLUMN_NAME)); Preconditions.checkNotNull(columns.get(ColumnHolder.TIME_COLUMN_NAME));
this.dataInterval = Preconditions.checkNotNull(dataInterval, "dataInterval"); this.dataInterval = Preconditions.checkNotNull(dataInterval, "dataInterval");
@ -79,18 +106,35 @@ public abstract class SimpleQueryableIndex implements QueryableIndex
this.columns = columns; this.columns = columns;
this.fileMapper = fileMapper; this.fileMapper = fileMapper;
this.projectionColumns = projectionColumns == null ? Collections.emptyMap() : projectionColumns;
if (lazy) { if (lazy) {
this.dimensionHandlers = Suppliers.memoize(() -> initDimensionHandlers(availableDimensions)); this.dimensionHandlers = Suppliers.memoize(() -> initDimensionHandlers(availableDimensions));
} else { } else {
this.dimensionHandlers = () -> initDimensionHandlers(availableDimensions); this.dimensionHandlers = () -> initDimensionHandlers(availableDimensions);
} }
if (metadata != null) {
final Metadata metadata = getMetadata(); if (metadata.getOrdering() != null) {
if (metadata != null && metadata.getOrdering() != null) { this.ordering = ORDERING_INTERNER.intern(metadata.getOrdering());
this.ordering = 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 { } else {
// When sort order isn't set in metadata.drd, assume the segment is sorted by __time. // When sort order isn't set in metadata.drd, assume the segment is sorted by __time.
this.ordering = Cursors.ascendingTimeOrder(); this.ordering = Cursors.ascendingTimeOrder();
this.projections = Collections.emptySortedSet();
this.projectionsMap = Collections.emptyMap();
} }
} }
@ -177,4 +221,54 @@ public abstract class SimpleQueryableIndex implements QueryableIndex
} }
return dimensionHandlerMap; return dimensionHandlerMap;
} }
@Nullable
@Override
public QueryableProjection<QueryableIndex> 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<OrderBy> getOrdering()
{
// return ordering with projection time column substituted with __time so query engines can treat it equivalently
return projectionSpec.getSchema().getOrderingWithTimeColumnSubstitution();
}
};
}
}

View File

@ -164,6 +164,7 @@ public class StringDimensionHandler implements DimensionHandler<Integer, int[],
@Override @Override
public DimensionMergerV9 makeMerger( public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -182,6 +183,7 @@ public class StringDimensionHandler implements DimensionHandler<Integer, int[],
return new StringDimensionMergerV9( return new StringDimensionMergerV9(
dimensionName, dimensionName,
outputName,
indexSpec, indexSpec,
segmentWriteOutMedium, segmentWriteOutMedium,
capabilities, capabilities,

View File

@ -28,6 +28,7 @@ import org.apache.druid.collections.spatial.ImmutableRTree;
import org.apache.druid.collections.spatial.RTree; import org.apache.druid.collections.spatial.RTree;
import org.apache.druid.collections.spatial.split.LinearGutmanSplitStrategy; import org.apache.druid.collections.spatial.split.LinearGutmanSplitStrategy;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilities;
@ -63,8 +64,22 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger<Strin
@Nullable @Nullable
private ByteBufferWriter<ImmutableRTree> spatialWriter; private ByteBufferWriter<ImmutableRTree> 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( public StringDimensionMergerV9(
String dimensionName, String dimensionName,
String outputName,
IndexSpec indexSpec, IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium, SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities, ColumnCapabilities capabilities,
@ -72,7 +87,7 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger<Strin
Closer closer Closer closer
) )
{ {
super(dimensionName, indexSpec, segmentWriteOutMedium, capabilities, progress, closer); super(dimensionName, outputName, indexSpec, segmentWriteOutMedium, capabilities, progress, closer);
} }
@Override @Override
@ -113,8 +128,11 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger<Strin
@Override @Override
protected ExtendedIndexesMerger getExtendedIndexesMerger() protected ExtendedIndexesMerger getExtendedIndexesMerger()
{ {
if (capabilities.hasSpatialIndexes()) {
return new SpatialIndexesMerger(); return new SpatialIndexesMerger();
} }
return null;
}
@Override @Override
public ColumnDescriptor makeColumnDescriptor() public ColumnDescriptor makeColumnDescriptor()
@ -127,9 +145,8 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger<Strin
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
builder.setValueType(ValueType.STRING); builder.setValueType(ValueType.STRING);
builder.setHasMultipleValues(hasMultiValue); builder.setHasMultipleValues(hasMultiValue);
final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde
.serializerBuilder() .serializerBuilder()
.withDictionary(dictionaryWriter)
.withValue( .withValue(
encodedValueSerializer, encodedValueSerializer,
hasMultiValue, hasMultiValue,
@ -140,11 +157,33 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger<Strin
.withSpatialIndex(spatialWriter) .withSpatialIndex(spatialWriter)
.withByteOrder(IndexIO.BYTE_ORDER); .withByteOrder(IndexIO.BYTE_ORDER);
if (writeDictionary) {
partBuilder = partBuilder.withDictionary(dictionaryWriter);
}
return builder return builder
.addSerde(partBuilder.build()) .addSerde(partBuilder.build())
.build(); .build();
} }
@Override
public void attachParent(DimensionMergerV9 parent, List<IndexableAdapter> 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 * Write spatial indexes for string columns that have them
*/ */

View File

@ -94,8 +94,8 @@ public class TimeAndDimsPointer implements Comparable<TimeAndDimsPointer>
this.timestampSelector = timestampSelector; this.timestampSelector = timestampSelector;
this.timePosition = timePosition; this.timePosition = timePosition;
Preconditions.checkArgument( Preconditions.checkArgument(
timePosition >= 0 && timePosition <= dimensionSelectors.length, timePosition <= dimensionSelectors.length,
"timePosition[%s] is out of range 0 .. [%s]", "timePosition[%s] must be less than [%s]",
timePosition, timePosition,
dimensionSelectors.length dimensionSelectors.length
); );

View File

@ -99,7 +99,8 @@ public class ColumnDescriptor implements Serializer
public ColumnHolder read( public ColumnHolder read(
ByteBuffer buffer, ByteBuffer buffer,
ColumnConfig columnConfig, ColumnConfig columnConfig,
SmooshedFileMapper smooshedFiles SmooshedFileMapper smooshedFiles,
@Nullable ColumnHolder parent
) )
{ {
final ColumnBuilder builder = new ColumnBuilder() final ColumnBuilder builder = new ColumnBuilder()
@ -108,9 +109,8 @@ public class ColumnDescriptor implements Serializer
.setFileMapper(smooshedFiles); .setFileMapper(smooshedFiles);
for (ColumnPartSerde part : parts) { for (ColumnPartSerde part : parts) {
part.getDeserializer().read(buffer, builder, columnConfig); part.getDeserializer().read(buffer, builder, columnConfig, parent);
} }
return builder.build(); return builder.build();
} }

View File

@ -19,6 +19,7 @@
package org.apache.druid.segment.column; package org.apache.druid.segment.column;
import com.google.common.base.Supplier;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector;
@ -45,8 +46,15 @@ public interface ColumnHolder
} }
int getLength(); int getLength();
BaseColumn getColumn(); BaseColumn getColumn();
@Nullable
default Supplier<? extends BaseColumn> getColumnSupplier()
{
return this::getColumn;
}
@Nullable @Nullable
ColumnIndexSupplier getIndexSupplier(); ColumnIndexSupplier getIndexSupplier();

View File

@ -99,6 +99,12 @@ class SimpleColumnHolder implements ColumnHolder
return columnSupplier == null ? UnknownTypeComplexColumn.instance() : columnSupplier.get(); return columnSupplier == null ? UnknownTypeComplexColumn.instance() : columnSupplier.get();
} }
@Override
public Supplier<? extends BaseColumn> getColumnSupplier()
{
return columnSupplier;
}
@Nullable @Nullable
@Override @Override
public ColumnIndexSupplier getIndexSupplier() public ColumnIndexSupplier getIndexSupplier()

View File

@ -52,6 +52,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandler;
import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionIndexer; 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.ColumnType;
import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.column.ValueType; 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.ComplexMetricExtractor;
import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetricSerde;
import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.serde.ComplexMetrics;
@ -85,6 +87,7 @@ import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.Closeable; import java.io.Closeable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
@ -363,10 +366,16 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
incrementalIndexSchema.getTimestampSpec(), incrementalIndexSchema.getTimestampSpec(),
this.queryGranularity, this.queryGranularity,
this.rollup, this.rollup,
getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()) getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()),
Collections.emptyList()
); );
} }
@Nullable
public abstract QueryableProjection<IncrementalIndexRowSelector> getProjection(CursorBuildSpec buildSpec);
public abstract IncrementalIndexRowSelector getProjection(String name);
public abstract boolean canAppendRow(); public abstract boolean canAppendRow();
public abstract String getOutOfRowsReason(); public abstract String getOutOfRowsReason();
@ -436,6 +445,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
} }
@Nullable @Nullable
@Override
public ColumnFormat getColumnFormat(String columnName) public ColumnFormat getColumnFormat(String columnName)
{ {
if (timeAndMetricsColumnFormats.containsKey(columnName)) { if (timeAndMetricsColumnFormats.containsKey(columnName)) {
@ -670,7 +680,8 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
return numEntries.get() == 0; return numEntries.get() == 0;
} }
public int size() @Override
public int numRows()
{ {
return numEntries.get(); return numEntries.get();
} }
@ -718,6 +729,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
* *
* @param includeTime whether to include {@link ColumnHolder#TIME_COLUMN_NAME}. * @param includeTime whether to include {@link ColumnHolder#TIME_COLUMN_NAME}.
*/ */
@Override
public List<String> getDimensionNames(final boolean includeTime) public List<String> getDimensionNames(final boolean includeTime)
{ {
synchronized (dimensionDescs) { 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}. * Returns a descriptor for each dimension. Does not inclue {@link ColumnHolder#TIME_COLUMN_NAME}.
*/ */
@Override
public List<DimensionDesc> getDimensions() public List<DimensionDesc> getDimensions()
{ {
synchronized (dimensionDescs) { synchronized (dimensionDescs) {
@ -777,6 +790,12 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
return metadata.getOrdering(); return metadata.getOrdering();
} }
@Override
public int getTimePosition()
{
return timePosition;
}
public static ColumnValueSelector<?> makeMetricColumnValueSelector( public static ColumnValueSelector<?> makeMetricColumnValueSelector(
IncrementalIndexRowSelector rowSelector, IncrementalIndexRowSelector rowSelector,
IncrementalIndexRowHolder currEntry, IncrementalIndexRowHolder currEntry,
@ -861,7 +880,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
) )
{ {
synchronized (dimensionDescs) { synchronized (dimensionDescs) {
if (size() != 0) { if (numRows() != 0) {
throw new ISE("Cannot load dimension order[%s] when existing index is not empty.", dimensionDescs.keySet()); throw new ISE("Cannot load dimension order[%s] when existing index is not empty.", dimensionDescs.keySet());
} }
for (String dim : oldDimensionOrder) { for (String dim : oldDimensionOrder) {
@ -888,6 +907,7 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates); return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates);
} }
@Override
public List<String> getMetricNames() public List<String> getMetricNames()
{ {
return ImmutableList.copyOf(metricDescs.keySet()); return ImmutableList.copyOf(metricDescs.keySet());
@ -932,7 +952,6 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); return new IncrementalIndexRowComparator(timePosition, dimensionDescsList);
} }
private static String getSimplifiedEventStringFromRow(InputRow inputRow) private static String getSimplifiedEventStringFromRow(InputRow inputRow)
{ {
if (inputRow instanceof MapBasedInputRow) { if (inputRow instanceof MapBasedInputRow) {
@ -985,6 +1004,14 @@ public abstract class IncrementalIndex implements IncrementalIndexRowSelector, C
this.indexer = handler.makeIndexer(useMaxMemoryEstimates); 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() public int getIndex()
{ {
return index; return index;

View File

@ -22,6 +22,7 @@ package org.apache.druid.segment.incremental;
import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterator;
import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.collections.bitmap.MutableBitmap;
import org.apache.druid.error.DruidException;
import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.AutoTypeColumnIndexer;
import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.DimensionIndexer;
import org.apache.druid.segment.IndexableAdapter; import org.apache.druid.segment.IndexableAdapter;
@ -46,13 +47,15 @@ import java.util.stream.Collectors;
public class IncrementalIndexAdapter implements IndexableAdapter public class IncrementalIndexAdapter implements IndexableAdapter
{ {
private final Interval dataInterval; private final Interval dataInterval;
private final IncrementalIndex index; private final IncrementalIndexRowSelector index;
private final Map<String, DimensionAccessor> accessors; private final Map<String, DimensionAccessor> 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.dataInterval = dataInterval;
this.index = index; this.index = index;
this.bitmapFactory = bitmapFactory;
final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions(); final List<IncrementalIndex.DimensionDesc> dimensions = index.getDimensions();
accessors = dimensions accessors = dimensions
@ -73,7 +76,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
* a null value. * a null value.
*/ */
private void processRows( private void processRows(
IncrementalIndex index, IncrementalIndexRowSelector index,
BitmapFactory bitmapFactory, BitmapFactory bitmapFactory,
List<IncrementalIndex.DimensionDesc> dimensions List<IncrementalIndex.DimensionDesc> dimensions
) )
@ -112,7 +115,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter
@Override @Override
public int getNumRows() public int getNumRows()
{ {
return index.size(); return index.numRows();
} }
@Override @Override
@ -182,6 +185,16 @@ public class IncrementalIndexAdapter implements IndexableAdapter
return new IncrementalIndexRowIterator(index); 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 @Override
public BitmapValues getBitmapValues(String dimension, int index) public BitmapValues getBitmapValues(String dimension, int index)
{ {
@ -225,7 +238,10 @@ public class IncrementalIndexAdapter implements IndexableAdapter
@Override @Override
public Metadata getMetadata() 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 static class MutableBitmapValues implements BitmapValues

View File

@ -50,15 +50,15 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro
IncrementalIndexColumnSelectorFactory( IncrementalIndexColumnSelectorFactory(
IncrementalIndexRowSelector rowSelector, IncrementalIndexRowSelector rowSelector,
IncrementalIndexRowHolder rowHolder,
VirtualColumns virtualColumns, VirtualColumns virtualColumns,
Order timeOrder, Order timeOrder
IncrementalIndexRowHolder rowHolder
) )
{ {
this.rowSelector = rowSelector;
this.virtualColumns = virtualColumns; this.virtualColumns = virtualColumns;
this.timeOrder = timeOrder; this.timeOrder = timeOrder;
this.rowHolder = rowHolder; this.rowHolder = rowHolder;
this.rowSelector = rowSelector;
this.snapshotColumnInspector = new ColumnInspector() this.snapshotColumnInspector = new ColumnInspector()
{ {
@Nullable @Nullable
@ -85,12 +85,8 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro
final String dimension = dimensionSpec.getDimension(); final String dimension = dimensionSpec.getDimension();
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrder != Order.NONE) { if (isTimeColumn(dimension) && timeOrder != Order.NONE) {
return new SingleScanTimeDimensionSelector( return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, timeOrder);
makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
extractionFn,
timeOrder
);
} }
final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(dimensionSpec.getDimension()); final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(dimensionSpec.getDimension());
@ -122,7 +118,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro
if (virtualColumns.exists(columnName)) { if (virtualColumns.exists(columnName)) {
return virtualColumns.makeColumnValueSelector(columnName, this); return virtualColumns.makeColumnValueSelector(columnName, this);
} }
if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { if (isTimeColumn(columnName)) {
return rowHolder; return rowHolder;
} }
@ -140,6 +136,9 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro
public ColumnCapabilities getColumnCapabilities(String columnName) public ColumnCapabilities getColumnCapabilities(String columnName)
{ {
// Use snapshotColumnInspector instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater) // 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); return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, columnName);
} }
@ -155,4 +154,9 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro
{ {
return rowHolder.get().getRowIndex(); return rowHolder.get().getRowIndex();
} }
private boolean isTimeColumn(String columnName)
{
return ColumnHolder.TIME_COLUMN_NAME.equals(columnName);
}
} }

View File

@ -20,6 +20,8 @@
package org.apache.druid.segment.incremental; package org.apache.druid.segment.incremental;
import com.google.common.collect.Iterables; 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.CursorBuildSpec;
import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder; 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.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.projections.QueryableProjection;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.List;
public class IncrementalIndexCursorFactory implements CursorFactory public class IncrementalIndexCursorFactory implements CursorFactory
{ {
@ -77,7 +81,35 @@ public class IncrementalIndexCursorFactory implements CursorFactory
@Override @Override
public CursorHolder makeCursorHolder(CursorBuildSpec spec) public CursorHolder makeCursorHolder(CursorBuildSpec spec)
{ {
final QueryableProjection<IncrementalIndexRowSelector> projection = index.getProjection(spec);
if (projection == null) {
return new IncrementalIndexCursorHolder(index, spec); 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<AggregatorFactory> getAggregatorsForPreAggregated()
{
return projection.getCursorBuildSpec().getAggregators();
}
};
}
} }
@Override @Override

View File

@ -29,7 +29,9 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.Cursors; import org.apache.druid.segment.Cursors;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.filter.ValueMatchers;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -70,10 +72,13 @@ public class IncrementalIndexCursorHolder implements CursorHolder
spec.getQueryMetrics().vectorized(false); spec.getQueryMetrics().vectorized(false);
} }
IncrementalIndexRowHolder currentRow = new IncrementalIndexRowHolder();
return new IncrementalIndexCursor( return new IncrementalIndexCursor(
rowSelector, rowSelector,
currentRow,
makeSelectorFactory(spec, currentRow),
spec, spec,
Cursors.getTimeOrdering(ordering) getTimeOrder(ordering)
); );
} }
@ -83,47 +88,50 @@ public class IncrementalIndexCursorHolder implements CursorHolder
return ordering; return ordering;
} }
public ColumnSelectorFactory makeSelectorFactory(CursorBuildSpec buildSpec, IncrementalIndexRowHolder currEntry)
{
return new IncrementalIndexColumnSelectorFactory(
rowSelector,
currEntry,
buildSpec.getVirtualColumns(),
getTimeOrder()
);
}
static class IncrementalIndexCursor implements Cursor static class IncrementalIndexCursor implements Cursor
{ {
private final IncrementalIndexRowSelector rowSelector; private final Iterable<IncrementalIndexRow> cursorIterable;
private final IncrementalIndexRowHolder currEntry; private final IncrementalIndexRowHolder currEntry;
private final ColumnSelectorFactory columnSelectorFactory; private final ColumnSelectorFactory columnSelectorFactory;
private final ValueMatcher filterMatcher; private final ValueMatcher filterMatcher;
private final int maxRowIndex; private final int maxRowIndex;
@MonotonicNonNull
private Iterator<IncrementalIndexRow> baseIter; private Iterator<IncrementalIndexRow> baseIter;
private Iterable<IncrementalIndexRow> cursorIterable;
private boolean emptyRange;
private int numAdvanced; private int numAdvanced;
private boolean done; private boolean done;
IncrementalIndexCursor( IncrementalIndexCursor(
IncrementalIndexRowSelector index, IncrementalIndexRowSelector rowSelector,
IncrementalIndexRowHolder currentRow,
ColumnSelectorFactory selectorFactory,
CursorBuildSpec buildSpec, CursorBuildSpec buildSpec,
Order timeOrder Order timeOrder
) )
{ {
currEntry = new IncrementalIndexRowHolder(); currEntry = currentRow;
columnSelectorFactory = selectorFactory;
// Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340
maxRowIndex = index.getLastRowIndex(); maxRowIndex = rowSelector.getLastRowIndex();
numAdvanced = -1; numAdvanced = -1;
rowSelector = index;
cursorIterable = rowSelector.getFacts().timeRangeIterable( cursorIterable = rowSelector.getFacts().timeRangeIterable(
timeOrder == Order.DESCENDING, timeOrder == Order.DESCENDING,
buildSpec.getInterval().getStartMillis(), buildSpec.getInterval().getStartMillis(),
buildSpec.getInterval().getEndMillis() buildSpec.getInterval().getEndMillis()
); );
columnSelectorFactory = new IncrementalIndexColumnSelectorFactory(
rowSelector,
buildSpec.getVirtualColumns(),
timeOrder,
currEntry
);
filterMatcher = buildSpec.getFilter() == null filterMatcher = buildSpec.getFilter() == null
? ValueMatchers.allTrue() ? ValueMatchers.allTrue()
: buildSpec.getFilter().makeMatcher(columnSelectorFactory); : buildSpec.getFilter().makeMatcher(columnSelectorFactory);
emptyRange = !cursorIterable.iterator().hasNext();
reset(); reset();
} }
@ -228,7 +236,7 @@ public class IncrementalIndexCursorHolder implements CursorHolder
numAdvanced++; numAdvanced++;
} }
done = !foundMatched && (emptyRange || !baseIter.hasNext()); done = !foundMatched;
} }
private boolean beyondMaxRowIndex(int rowIndex) private boolean beyondMaxRowIndex(int rowIndex)
@ -239,4 +247,13 @@ public class IncrementalIndexCursorHolder implements CursorHolder
return rowIndex > maxRowIndex; return rowIndex > maxRowIndex;
} }
} }
private static Order getTimeOrder(List<OrderBy> ordering)
{
if (!ordering.isEmpty() && ColumnHolder.TIME_COLUMN_NAME.equals(ordering.get(0).getColumnName())) {
return ordering.get(0).getOrder();
} else {
return Order.NONE;
}
}
} }

View File

@ -134,6 +134,6 @@ public class IncrementalIndexPhysicalSegmentInspector implements PhysicalSegment
@Override @Override
public int getNumRows() public int getNumRows()
{ {
return index.size(); return index.numRows();
} }
} }

View File

@ -49,7 +49,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator
private final RowPointer currentRowPointer; private final RowPointer currentRowPointer;
private final TimeAndDimsPointer markedRowPointer; private final TimeAndDimsPointer markedRowPointer;
IncrementalIndexRowIterator(IncrementalIndex incrementalIndex) IncrementalIndexRowIterator(IncrementalIndexRowSelector incrementalIndex)
{ {
this.timeAndDimsIterator = incrementalIndex.getFacts().persistIterable().iterator(); this.timeAndDimsIterator = incrementalIndex.getFacts().persistIterable().iterator();
this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter); this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter);
@ -59,7 +59,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator
} }
private static RowPointer makeRowPointer( private static RowPointer makeRowPointer(
IncrementalIndex incrementalIndex, IncrementalIndexRowSelector incrementalIndex,
IncrementalIndexRowHolder rowHolder, IncrementalIndexRowHolder rowHolder,
RowNumCounter rowNumCounter RowNumCounter rowNumCounter
) )
@ -67,9 +67,9 @@ class IncrementalIndexRowIterator implements TransformableRowIterator
ColumnSelectorFactory columnSelectorFactory = ColumnSelectorFactory columnSelectorFactory =
new IncrementalIndexColumnSelectorFactory( new IncrementalIndexColumnSelectorFactory(
incrementalIndex, incrementalIndex,
rowHolder,
VirtualColumns.EMPTY, VirtualColumns.EMPTY,
incrementalIndex.timePosition == 0 ? Order.ASCENDING : Order.NONE, incrementalIndex.getTimePosition() == 0 ? Order.ASCENDING : Order.NONE
rowHolder
); );
ColumnValueSelector[] dimensionSelectors = incrementalIndex ColumnValueSelector[] dimensionSelectors = incrementalIndex
.getDimensions() .getDimensions()
@ -92,7 +92,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator
return new RowPointer( return new RowPointer(
rowHolder, rowHolder,
incrementalIndex.timePosition, incrementalIndex.getTimePosition(),
dimensionSelectors, dimensionSelectors,
dimensionHandlers, dimensionHandlers,
metricSelectors, metricSelectors,

View File

@ -21,6 +21,7 @@ package org.apache.druid.segment.incremental;
import org.apache.druid.query.OrderBy; import org.apache.druid.query.OrderBy;
import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.column.ColumnFormat;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.List; import java.util.List;
@ -30,6 +31,22 @@ import java.util.List;
*/ */
public interface IncrementalIndexRowSelector extends ColumnInspector public interface IncrementalIndexRowSelector extends ColumnInspector
{ {
/**
* Returns list of {@link IncrementalIndex.DimensionDesc} for the row selector
*/
List<IncrementalIndex.DimensionDesc> 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<String> getDimensionNames(boolean includeTime);
/**
* Returns list of all metric column names for the row selector
*/
List<String> getMetricNames();
/** /**
* get {@link IncrementalIndex.DimensionDesc} for the specified column, if available, which provides access to things * 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 * 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 @Nullable
IncrementalIndex.MetricDesc getMetric(String s); 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 * Ordering for the data in the facts table
*/ */
List<OrderBy> getOrdering(); List<OrderBy> getOrdering();
/**
* Position of the time column in {@link #getOrdering()}
*/
int getTimePosition();
/** /**
* Are there any {@link IncrementalIndexRow} stored in the {@link FactsHolder}? * Are there any {@link IncrementalIndexRow} stored in the {@link FactsHolder}?
*/ */
boolean isEmpty(); boolean isEmpty();
/**
* Number of rows in {@link FactsHolder}
*/
int numRows();
/** /**
* Get the {@link FactsHolder} containing all of the {@link IncrementalIndexRow} backing this selector * Get the {@link FactsHolder} containing all of the {@link IncrementalIndexRow} backing this selector
*/ */

View File

@ -19,6 +19,7 @@
package org.apache.druid.segment.incremental; 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.DimensionsSpec;
import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec; 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 org.apache.druid.segment.VirtualColumns;
import javax.annotation.Nullable; 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 AggregatorFactory[] metrics;
private final boolean rollup; private final boolean rollup;
private final List<AggregateProjectionSpec> projections;
public IncrementalIndexSchema( public IncrementalIndexSchema(
long minTimestamp, long minTimestamp,
TimestampSpec timestampSpec, TimestampSpec timestampSpec,
@ -53,7 +58,8 @@ public class IncrementalIndexSchema
VirtualColumns virtualColumns, VirtualColumns virtualColumns,
DimensionsSpec dimensionsSpec, DimensionsSpec dimensionsSpec,
AggregatorFactory[] metrics, AggregatorFactory[] metrics,
boolean rollup boolean rollup,
List<AggregateProjectionSpec> projections
) )
{ {
this.minTimestamp = minTimestamp; this.minTimestamp = minTimestamp;
@ -63,6 +69,7 @@ public class IncrementalIndexSchema
this.dimensionsSpec = dimensionsSpec; this.dimensionsSpec = dimensionsSpec;
this.metrics = metrics; this.metrics = metrics;
this.rollup = rollup; this.rollup = rollup;
this.projections = projections;
} }
public long getMinTimestamp() public long getMinTimestamp()
@ -100,6 +107,11 @@ public class IncrementalIndexSchema
return rollup; return rollup;
} }
public List<AggregateProjectionSpec> getProjections()
{
return projections;
}
public static class Builder public static class Builder
{ {
private long minTimestamp; private long minTimestamp;
@ -109,6 +121,7 @@ public class IncrementalIndexSchema
private DimensionsSpec dimensionsSpec; private DimensionsSpec dimensionsSpec;
private AggregatorFactory[] metrics; private AggregatorFactory[] metrics;
private boolean rollup; private boolean rollup;
private List<AggregateProjectionSpec> projections;
public Builder() public Builder()
{ {
@ -118,6 +131,7 @@ public class IncrementalIndexSchema
this.dimensionsSpec = DimensionsSpec.EMPTY; this.dimensionsSpec = DimensionsSpec.EMPTY;
this.metrics = new AggregatorFactory[]{}; this.metrics = new AggregatorFactory[]{};
this.rollup = true; this.rollup = true;
this.projections = Collections.emptyList();
} }
public Builder withMinTimestamp(long minTimestamp) public Builder withMinTimestamp(long minTimestamp)
@ -176,6 +190,12 @@ public class IncrementalIndexSchema
return this; return this;
} }
public Builder withProjections(@Nullable List<AggregateProjectionSpec> projections)
{
this.projections = projections == null ? Collections.emptyList() : projections;
return this;
}
public IncrementalIndexSchema build() public IncrementalIndexSchema build()
{ {
return new IncrementalIndexSchema( return new IncrementalIndexSchema(
@ -185,7 +205,8 @@ public class IncrementalIndexSchema
virtualColumns, virtualColumns,
dimensionsSpec, dimensionsSpec,
metrics, metrics,
rollup rollup,
projections
); );
} }
} }

View File

@ -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<IncrementalIndex.DimensionDesc> dimensions;
private final int[] parentDimensionIndex;
private final AggregatorFactory[] aggregatorFactories;
private final Map<String, IncrementalIndex.DimensionDesc> dimensionsMap;
private final Map<String, IncrementalIndex.MetricDesc> aggregatorsMap;
private final Map<String, ColumnFormat> columnFormats;
private final FactsHolder factsHolder;
private final IncrementalIndex.InputRowHolder inputRowHolder = new IncrementalIndex.InputRowHolder();
private final ConcurrentHashMap<Integer, Aggregator[]> aggregators = new ConcurrentHashMap<>();
private final ColumnSelectorFactory virtualSelectorFactory;
private final Map<String, ColumnSelectorFactory> 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<IncrementalIndex.DimensionDesc> dimensions,
Map<String, IncrementalIndex.DimensionDesc> 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<String> 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<IncrementalIndex.DimensionDesc> getDimensions()
{
return dimensions;
}
@Override
public List<String> 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<OrderBy> 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<String> getDimensionNames(boolean includeTime)
{
synchronized (dimensionsMap) {
if (includeTime && projectionSchema.getTimeColumnName() != null) {
final ImmutableList.Builder<String> 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<String, IncrementalIndex.DimensionDesc> 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;
}
}

View File

@ -24,9 +24,13 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators; import com.google.common.collect.Iterators;
import com.google.common.collect.Maps; 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.MapBasedRow;
import org.apache.druid.data.input.Row; 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.DruidException;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.io.Closer; 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.AggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.dimension.DimensionSpec; 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.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandler;
import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.DimensionIndexer;
import org.apache.druid.segment.DimensionSelector; 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.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ValueType; 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 org.apache.druid.utils.JvmUtils;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayDeque; import java.util.ArrayDeque;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator; import java.util.Comparator;
import java.util.Deque; import java.util.Deque;
@ -58,6 +70,7 @@ import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.SortedSet;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentLinkedDeque;
import java.util.concurrent.ConcurrentMap; 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.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function; import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
/** /**
@ -86,7 +100,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex
/** /**
* overhead per {@link ConcurrentSkipListMap.Node} object in facts table * 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<Integer, Aggregator[]> aggregators = new ConcurrentHashMap<>(); private final ConcurrentHashMap<Integer, Aggregator[]> aggregators = new ConcurrentHashMap<>();
private final FactsHolder facts; private final FactsHolder facts;
private final AtomicInteger indexIncrement = new AtomicInteger(0); private final AtomicInteger indexIncrement = new AtomicInteger(0);
@ -135,6 +150,9 @@ public class OnheapIncrementalIndex extends IncrementalIndex
@Nullable @Nullable
private String outOfRowsReason = null; private String outOfRowsReason = null;
private final SortedSet<AggregateProjectionMetadata> aggregateProjections;
private final HashMap<String, OnHeapAggregateProjection> projections;
OnheapIncrementalIndex( OnheapIncrementalIndex(
IncrementalIndexSchema incrementalIndexSchema, IncrementalIndexSchema incrementalIndexSchema,
int maxRowCount, int maxRowCount,
@ -158,6 +176,80 @@ public class OnheapIncrementalIndex extends IncrementalIndex
this.maxBytesPerRowForAggregators = this.maxBytesPerRowForAggregators =
useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0; useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0;
this.useMaxMemoryEstimates = useMaxMemoryEstimates; 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<DimensionDesc> 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<String, DimensionDesc> 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; return facts;
} }
@Override
public Metadata getMetadata()
{
if (aggregateProjections.isEmpty()) {
return super.getMetadata();
}
final List<AggregateProjectionMetadata> projectionMetadata = projections.values()
.stream()
.map(OnHeapAggregateProjection::toMetadata)
.collect(Collectors.toList());
return super.getMetadata().withProjections(projectionMetadata);
}
@Override @Override
protected void initAggs( protected void initAggs(
final AggregatorFactory[] metrics, final AggregatorFactory[] metrics,
@ -254,6 +359,12 @@ public class OnheapIncrementalIndex extends IncrementalIndex
final List<String> parseExceptionMessages = new ArrayList<>(); final List<String> parseExceptionMessages = new ArrayList<>();
final AtomicLong totalSizeInBytes = getBytesInMemory(); 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); final int priorIndex = facts.getPriorIndex(key);
Aggregator[] aggs; Aggregator[] aggs;
@ -336,11 +447,10 @@ public class OnheapIncrementalIndex extends IncrementalIndex
} }
// Creates aggregators to combine already aggregated field // Creates aggregators to combine already aggregated field
if (preserveExistingMetrics) { if (preserveExistingMetrics) {
if (useMaxMemoryEstimates) {
AggregatorFactory combiningAgg = agg.getCombiningFactory(); AggregatorFactory combiningAgg = agg.getCombiningFactory();
if (useMaxMemoryEstimates) {
aggs[i + metrics.length] = combiningAgg.factorize(combiningAggSelectors.get(combiningAgg.getName())); aggs[i + metrics.length] = combiningAgg.factorize(combiningAggSelectors.get(combiningAgg.getName()));
} else { } else {
AggregatorFactory combiningAgg = agg.getCombiningFactory();
AggregatorAndSize aggregatorAndSize = combiningAgg.factorizeWithSize(combiningAggSelectors.get(combiningAgg.getName())); AggregatorAndSize aggregatorAndSize = combiningAgg.factorizeWithSize(combiningAggSelectors.get(combiningAgg.getName()));
aggs[i + metrics.length] = aggregatorAndSize.getAggregator(); aggs[i + metrics.length] = aggregatorAndSize.getAggregator();
totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes();
@ -368,7 +478,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex
return doAggregate(metrics, aggs, inputRowHolder, parseExceptionsHolder, useMaxMemoryEstimates, preserveExistingMetrics); return doAggregate(metrics, aggs, inputRowHolder, parseExceptionsHolder, useMaxMemoryEstimates, preserveExistingMetrics);
} }
private static long doAggregate( static long doAggregate(
AggregatorFactory[] metrics, AggregatorFactory[] metrics,
Aggregator[] aggs, Aggregator[] aggs,
InputRowHolder inputRowHolder, InputRowHolder inputRowHolder,
@ -431,10 +541,29 @@ public class OnheapIncrementalIndex extends IncrementalIndex
} }
} }
@Nullable
@Override
public QueryableProjection<IncrementalIndexRowSelector> 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 @Override
public boolean canAppendRow() public boolean canAppendRow()
{ {
final boolean countCheck = size() < maxRowCount; final boolean countCheck = numRows() < maxRowCount;
// if maxBytesInMemory = -1, then ignore sizeCheck // if maxBytesInMemory = -1, then ignore sizeCheck
final boolean sizeCheck = maxBytesInMemory <= 0 || getBytesInMemory().get() < maxBytesInMemory; final boolean sizeCheck = maxBytesInMemory <= 0 || getBytesInMemory().get() < maxBytesInMemory;
final boolean canAdd = countCheck && sizeCheck; final boolean canAdd = countCheck && sizeCheck;

View File

@ -100,6 +100,12 @@ public final class DictionaryIdLookup implements Closeable
this.arrayDictionaryWriter = arrayDictionaryWriter; this.arrayDictionaryWriter = arrayDictionaryWriter;
} }
public int[] getArrayValue(int id)
{
ensureArrayDictionaryLoaded();
return arrayDictionary.get(id - arrayOffset());
}
@Nullable @Nullable
public Object getDictionaryValue(int id) 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();
}
} }

View File

@ -69,7 +69,9 @@ public abstract class NestedCommonFormatColumnSerializer implements GenericColum
public abstract String getColumnName(); public abstract String getColumnName();
public abstract DictionaryIdLookup getGlobalLookup(); public abstract DictionaryIdLookup getDictionaryIdLookup();
public abstract void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup);
public abstract boolean hasNulls(); public abstract boolean hasNulls();

View File

@ -156,6 +156,7 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
private Map<String, GlobalDictionaryEncodedFieldColumnWriter<?>> fieldWriters; private Map<String, GlobalDictionaryEncodedFieldColumnWriter<?>> fieldWriters;
private int rowCount = 0; private int rowCount = 0;
private boolean closedForWrite = false; private boolean closedForWrite = false;
private boolean writeDictionary = true;
private boolean dictionarySerialized = false; private boolean dictionarySerialized = false;
private ByteBuffer columnNameBytes = null; private ByteBuffer columnNameBytes = null;
@ -180,11 +181,19 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
} }
@Override @Override
public DictionaryIdLookup getGlobalLookup() public DictionaryIdLookup getDictionaryIdLookup()
{ {
return globalDictionaryIdLookup; return globalDictionaryIdLookup;
} }
@Override
public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup)
{
this.globalDictionaryIdLookup = dictionaryIdLookup;
this.writeDictionary = false;
this.dictionarySerialized = true;
}
@Override @Override
public boolean hasNulls() public boolean hasNulls()
{ {
@ -421,13 +430,15 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
) throws IOException ) throws IOException
{ {
Preconditions.checkState(closedForWrite, "Not closed yet!"); Preconditions.checkState(closedForWrite, "Not closed yet!");
if (writeDictionary) {
Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?");
}
writeV0Header(channel, columnNameBytes); writeV0Header(channel, columnNameBytes);
fieldsWriter.writeTo(channel, smoosher); fieldsWriter.writeTo(channel, smoosher);
fieldsInfoWriter.writeTo(channel, smoosher); fieldsInfoWriter.writeTo(channel, smoosher);
if (writeDictionary) {
if (globalDictionaryIdLookup.getStringBufferMapper() != null) { if (globalDictionaryIdLookup.getStringBufferMapper() != null) {
SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper(); SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper();
for (String internalName : fileMapper.getInternalFilenames()) { for (String internalName : fileMapper.getInternalFilenames()) {
@ -437,20 +448,33 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME);
} }
if (globalDictionaryIdLookup.getLongBuffer() != null) { if (globalDictionaryIdLookup.getLongBuffer() != null) {
writeInternal(smoosher, globalDictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); writeInternal(
smoosher,
globalDictionaryIdLookup.getLongBuffer(),
ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME
);
} else { } else {
writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME);
} }
if (globalDictionaryIdLookup.getDoubleBuffer() != null) { if (globalDictionaryIdLookup.getDoubleBuffer() != null) {
writeInternal(smoosher, globalDictionaryIdLookup.getDoubleBuffer(), ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); writeInternal(
smoosher,
globalDictionaryIdLookup.getDoubleBuffer(),
ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME
);
} else { } else {
writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME);
} }
if (globalDictionaryIdLookup.getArrayBuffer() != null) { if (globalDictionaryIdLookup.getArrayBuffer() != null) {
writeInternal(smoosher, globalDictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); writeInternal(
smoosher,
globalDictionaryIdLookup.getArrayBuffer(),
ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME
);
} else { } else {
writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME);
} }
}
writeInternal(smoosher, rawWriter, RAW_FILE_NAME); writeInternal(smoosher, rawWriter, RAW_FILE_NAME);
if (!nullRowsBitmap.isEmpty()) { if (!nullRowsBitmap.isEmpty()) {
writeInternal(smoosher, nullBitmapWriter, ColumnSerializerUtils.NULL_BITMAP_FILE_NAME); writeInternal(smoosher, nullBitmapWriter, ColumnSerializerUtils.NULL_BITMAP_FILE_NAME);

View File

@ -55,7 +55,8 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
ColumnBuilder columnBuilder, ColumnBuilder columnBuilder,
ColumnConfig columnConfig, ColumnConfig columnConfig,
BitmapSerdeFactory bitmapSerdeFactory, BitmapSerdeFactory bitmapSerdeFactory,
ByteOrder byteOrder ByteOrder byteOrder,
NestedDataColumnSupplier parent
) )
{ {
final byte version = bb.get(); final byte version = bb.get();
@ -77,49 +78,57 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper);
fieldInfo = FieldTypeInfo.read(bb, fields.size()); fieldInfo = FieldTypeInfo.read(bb, fields.size());
if (parent != null) {
stringDictionarySupplier = parent.stringDictionarySupplier;
longDictionarySupplier = parent.longDictionarySupplier;
doubleDictionarySupplier = parent.doubleDictionarySupplier;
arrayDictionarySupplier = parent.arrayDictionarySupplier;
} else {
final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME 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
);
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
mapper, mapper,
stringDictionaryBuffer, stringDictionaryBuffer,
byteOrder byteOrder
); );
final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME
);
longDictionarySupplier = FixedIndexed.read( longDictionarySupplier = FixedIndexed.read(
longDictionaryBuffer, longDictionaryBuffer,
ColumnType.LONG.getStrategy(), ColumnType.LONG.getStrategy(),
byteOrder, byteOrder,
Long.BYTES Long.BYTES
); );
final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME
);
doubleDictionarySupplier = FixedIndexed.read( doubleDictionarySupplier = FixedIndexed.read(
doubleDictionaryBuffer, doubleDictionaryBuffer,
ColumnType.DOUBLE.getStrategy(), ColumnType.DOUBLE.getStrategy(),
byteOrder, byteOrder,
Double.BYTES Double.BYTES
); );
final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME
);
arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( arrayDictionarySupplier = FrontCodedIntArrayIndexed.read(
arrayDictionarybuffer, arrayDictionarybuffer,
byteOrder byteOrder
); );
}
final ByteBuffer rawBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer rawBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,

View File

@ -91,7 +91,8 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
BitmapSerdeFactory bitmapSerdeFactory, BitmapSerdeFactory bitmapSerdeFactory,
ByteBuffer bb, ByteBuffer bb,
ColumnBuilder columnBuilder, ColumnBuilder columnBuilder,
ColumnConfig columnConfig ColumnConfig columnConfig,
@Nullable ScalarDoubleColumnAndIndexSupplier parent
) )
{ {
final byte version = bb.get(); final byte version = bb.get();
@ -103,23 +104,29 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME
);
final ByteBuffer doublesValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer doublesValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
ColumnSerializerUtils.DOUBLE_VALUE_COLUMN_FILE_NAME ColumnSerializerUtils.DOUBLE_VALUE_COLUMN_FILE_NAME
); );
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier = FixedIndexed.read( final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
if (parent != null) {
doubleDictionarySupplier = parent.doubleDictionarySupplier;
} else {
final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME
);
doubleDictionarySupplier = FixedIndexed.read(
doubleDictionaryBuffer, doubleDictionaryBuffer,
ColumnType.DOUBLE.getStrategy(), ColumnType.DOUBLE.getStrategy(),
byteOrder, byteOrder,
Double.BYTES Double.BYTES
); );
}
final Supplier<ColumnarDoubles> doubles = CompressedColumnarDoublesSuppliers.fromByteBuffer( final Supplier<ColumnarDoubles> doubles = CompressedColumnarDoublesSuppliers.fromByteBuffer(
doublesValueColumn, doublesValueColumn,

View File

@ -142,4 +142,13 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME);
} }
} }
@Override
public int getCardinality()
{
if (writeDictionary) {
return dictionaryWriter.getCardinality();
}
return dictionaryIdLookup.getDoubleCardinality();
}
} }

View File

@ -90,7 +90,8 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
BitmapSerdeFactory bitmapSerdeFactory, BitmapSerdeFactory bitmapSerdeFactory,
ByteBuffer bb, ByteBuffer bb,
ColumnBuilder columnBuilder, ColumnBuilder columnBuilder,
ColumnConfig columnConfig ColumnConfig columnConfig,
@Nullable ScalarLongColumnAndIndexSupplier parent
) )
{ {
final byte version = bb.get(); final byte version = bb.get();
@ -102,11 +103,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME
);
final ByteBuffer longsValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer longsValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
@ -123,12 +119,23 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
columnBuilder.getFileMapper() columnBuilder.getFileMapper()
); );
final Supplier<FixedIndexed<Long>> longDictionarySupplier = FixedIndexed.read(
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
if (parent != null) {
longDictionarySupplier = parent.longDictionarySupplier;
} else {
final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME
);
longDictionarySupplier = FixedIndexed.read(
longDictionaryBuffer, longDictionaryBuffer,
ColumnType.LONG.getStrategy(), ColumnType.LONG.getStrategy(),
byteOrder, byteOrder,
Long.BYTES Long.BYTES
); );
}
final Supplier<ColumnarLongs> longs = CompressedColumnarLongsSupplier.fromByteBuffer( final Supplier<ColumnarLongs> longs = CompressedColumnarLongsSupplier.fromByteBuffer(
longsValueColumn, longsValueColumn,

View File

@ -142,4 +142,13 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME);
} }
} }
@Override
public int getCardinality()
{
if (writeDictionary) {
return dictionaryWriter.getCardinality();
}
return dictionaryIdLookup.getLongCardinality();
}
} }

View File

@ -62,6 +62,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
protected ByteBuffer columnNameBytes = null; protected ByteBuffer columnNameBytes = null;
protected boolean hasNulls; protected boolean hasNulls;
protected boolean writeDictionary = true;
public ScalarNestedCommonFormatColumnSerializer( public ScalarNestedCommonFormatColumnSerializer(
@ -98,6 +99,8 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
protected abstract void writeDictionaryFile(FileSmoosher smoosher) throws IOException; protected abstract void writeDictionaryFile(FileSmoosher smoosher) throws IOException;
public abstract int getCardinality();
@Override @Override
public String getColumnName() public String getColumnName()
{ {
@ -105,11 +108,19 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
} }
@Override @Override
public DictionaryIdLookup getGlobalLookup() public DictionaryIdLookup getDictionaryIdLookup()
{ {
return dictionaryIdLookup; return dictionaryIdLookup;
} }
@Override
public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup)
{
this.dictionaryIdLookup = dictionaryIdLookup;
this.writeDictionary = false;
this.dictionarySerialized = true;
}
@Override @Override
public boolean hasNulls() public boolean hasNulls()
{ {
@ -166,7 +177,9 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
) throws IOException ) throws IOException
{ {
Preconditions.checkState(closedForWrite, "Not closed yet!"); Preconditions.checkState(closedForWrite, "Not closed yet!");
if (writeDictionary) {
Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?");
}
// write out compressed dictionaryId int column and bitmap indexes by iterating intermediate value column // 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 // 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<T> extends Nested
name, name,
segmentWriteOutMedium, segmentWriteOutMedium,
filenameBase, filenameBase,
dictionaryWriter.getCardinality(), getCardinality(),
compressionToUse, compressionToUse,
segmentWriteOutMedium.getCloser() segmentWriteOutMedium.getCloser()
); );
@ -198,7 +211,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
bitmapIndexWriter.open(); bitmapIndexWriter.open();
bitmapIndexWriter.setObjectsNotSorted(); bitmapIndexWriter.setObjectsNotSorted();
final MutableBitmap[] bitmaps; final MutableBitmap[] bitmaps;
bitmaps = new MutableBitmap[dictionaryWriter.getCardinality()]; bitmaps = new MutableBitmap[getCardinality()];
for (int i = 0; i < bitmaps.length; i++) { for (int i = 0; i < bitmaps.length; i++) {
bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
} }
@ -220,7 +233,9 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
} }
writeV0Header(channel, columnNameBytes); writeV0Header(channel, columnNameBytes);
if (writeDictionary) {
writeDictionaryFile(smoosher); writeDictionaryFile(smoosher);
}
writeInternal(smoosher, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME); writeInternal(smoosher, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME);
writeValueColumn(smoosher); writeValueColumn(smoosher);
writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME); writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME);

View File

@ -49,7 +49,8 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
ByteOrder byteOrder, ByteOrder byteOrder,
BitmapSerdeFactory bitmapSerdeFactory, BitmapSerdeFactory bitmapSerdeFactory,
ByteBuffer bb, ByteBuffer bb,
ColumnBuilder columnBuilder ColumnBuilder columnBuilder,
@Nullable ScalarStringColumnAndIndexSupplier parent
) )
{ {
final byte version = bb.get(); final byte version = bb.get();
@ -61,6 +62,9 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier; final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
if (parent != null) {
dictionarySupplier = parent.dictionarySupplier;
} else {
final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
@ -72,6 +76,8 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
stringDictionaryBuffer, stringDictionaryBuffer,
byteOrder byteOrder
); );
}
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
@ -106,6 +112,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
} }
} }
private final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier; private final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
private final Supplier<ColumnarInts> encodedColumnSupplier; private final Supplier<ColumnarInts> encodedColumnSupplier;
private final GenericIndexed<ImmutableBitmap> valueIndexes; private final GenericIndexed<ImmutableBitmap> valueIndexes;

View File

@ -131,4 +131,13 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME);
} }
} }
@Override
public int getCardinality()
{
if (writeDictionary) {
return dictionaryWriter.getCardinality();
}
return dictionaryIdLookup.getStringCardinality();
}
} }

View File

@ -68,7 +68,8 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
ByteOrder byteOrder, ByteOrder byteOrder,
BitmapSerdeFactory bitmapSerdeFactory, BitmapSerdeFactory bitmapSerdeFactory,
ByteBuffer bb, ByteBuffer bb,
ColumnBuilder columnBuilder ColumnBuilder columnBuilder,
@Nullable VariantColumnAndIndexSupplier parent
) )
{ {
final byte version = bb.get(); final byte version = bb.get();
@ -95,26 +96,18 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier; final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier; final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier;
if (parent != null) {
stringDictionarySupplier = parent.stringDictionarySupplier;
longDictionarySupplier = parent.longDictionarySupplier;
doubleDictionarySupplier = parent.doubleDictionarySupplier;
arrayDictionarySupplier = parent.arrayDictionarySupplier;
arrayElementDictionarySupplier = parent.arrayElementDictionarySupplier;
} else {
final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME
); );
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
mapper,
stringDictionaryBuffer,
byteOrder
);
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME
);
final CompressedVSizeColumnarIntsSupplier ints = CompressedVSizeColumnarIntsSupplier.fromByteBuffer(
encodedValueColumn,
byteOrder
);
final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
@ -125,11 +118,55 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
columnName, columnName,
ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME
); );
final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME
);
final ByteBuffer arrayElementDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer arrayElementDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME 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
);
}
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME
);
final CompressedVSizeColumnarIntsSupplier ints = CompressedVSizeColumnarIntsSupplier.fromByteBuffer(
encodedValueColumn,
byteOrder
);
final ByteBuffer valueIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( final ByteBuffer valueIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper, mapper,
columnName, columnName,
@ -151,34 +188,6 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
columnBuilder.getFileMapper() columnBuilder.getFileMapper()
); );
longDictionarySupplier = FixedIndexed.read(
longDictionaryBuffer,
ColumnType.LONG.getStrategy(),
byteOrder,
Long.BYTES
);
doubleDictionarySupplier = FixedIndexed.read(
doubleDictionaryBuffer,
ColumnType.DOUBLE.getStrategy(),
byteOrder,
Double.BYTES
);
final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME
);
arrayDictionarySupplier = FrontCodedIntArrayIndexed.read(
arrayDictionarybuffer,
byteOrder
);
arrayElementDictionarySupplier = FixedIndexed.read(
arrayElementDictionaryBuffer,
CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY,
byteOrder,
Integer.BYTES
);
return new VariantColumnAndIndexSupplier( return new VariantColumnAndIndexSupplier(
logicalType, logicalType,
variantTypeByte, variantTypeByte,

View File

@ -83,6 +83,7 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
private ByteBuffer columnNameBytes = null; private ByteBuffer columnNameBytes = null;
private boolean hasNulls; private boolean hasNulls;
private boolean writeDictionary = true;
@Nullable @Nullable
private final ExpressionType expectedExpressionType; private final ExpressionType expectedExpressionType;
@Nullable @Nullable
@ -112,11 +113,19 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
} }
@Override @Override
public DictionaryIdLookup getGlobalLookup() public DictionaryIdLookup getDictionaryIdLookup()
{ {
return dictionaryIdLookup; return dictionaryIdLookup;
} }
@Override
public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup)
{
this.dictionaryIdLookup = dictionaryIdLookup;
this.writeDictionary = false;
this.dictionarySerialized = true;
}
@Override @Override
public boolean hasNulls() public boolean hasNulls()
{ {
@ -319,17 +328,19 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
) throws IOException ) throws IOException
{ {
Preconditions.checkState(closedForWrite, "Not closed yet!"); Preconditions.checkState(closedForWrite, "Not closed yet!");
if (writeDictionary) {
Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?");
}
// write out compressed dictionaryId int column, bitmap indexes, and array element bitmap indexes // write out compressed dictionaryId int column, bitmap indexes, and array element bitmap indexes
// by iterating intermediate value column the intermediate value column should be replaced someday by a cooler // by iterating intermediate value column the intermediate value column should be replaced someday by a cooler
// compressed int column writer that allows easy iteration of the values it writes out, so that we could just // compressed int column writer that allows easy iteration of the values it writes out, so that we could just
// build the bitmap indexes here instead of doing both things // build the bitmap indexes here instead of doing both things
String filenameBase = StringUtils.format("%s.forward_dim", name); String filenameBase = StringUtils.format("%s.forward_dim", name);
final int cardinality = dictionaryWriter.getCardinality() final int cardinality = dictionaryIdLookup.getStringCardinality()
+ longDictionaryWriter.getCardinality() + dictionaryIdLookup.getLongCardinality()
+ doubleDictionaryWriter.getCardinality() + dictionaryIdLookup.getDoubleCardinality()
+ arrayDictionaryWriter.getCardinality(); + dictionaryIdLookup.getArrayCardinality();
final CompressionStrategy compression = indexSpec.getDimensionCompression(); final CompressionStrategy compression = indexSpec.getDimensionCompression();
final CompressionStrategy compressionToUse; final CompressionStrategy compressionToUse;
if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) {
@ -370,15 +381,15 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
final IntIterator rows = intermediateValueWriter.getIterator(); final IntIterator rows = intermediateValueWriter.getIterator();
int rowCount = 0; int rowCount = 0;
final int arrayBaseId = dictionaryWriter.getCardinality() final int arrayBaseId = dictionaryIdLookup.getStringCardinality()
+ longDictionaryWriter.getCardinality() + dictionaryIdLookup.getLongCardinality()
+ doubleDictionaryWriter.getCardinality(); + dictionaryIdLookup.getDoubleCardinality();
while (rows.hasNext()) { while (rows.hasNext()) {
final int dictId = rows.nextInt(); final int dictId = rows.nextInt();
encodedValueSerializer.addValue(dictId); encodedValueSerializer.addValue(dictId);
bitmaps[dictId].add(rowCount); bitmaps[dictId].add(rowCount);
if (dictId >= arrayBaseId) { if (dictId >= arrayBaseId) {
int[] array = arrayDictionaryWriter.get(dictId - arrayBaseId); int[] array = dictionaryIdLookup.getArrayValue(dictId);
for (int elementId : array) { for (int elementId : array) {
arrayElements.computeIfAbsent( arrayElements.computeIfAbsent(
elementId, elementId,
@ -396,18 +407,21 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
); );
bitmaps[i] = null; // Reclaim memory bitmaps[i] = null; // Reclaim memory
} }
if (writeDictionary) {
for (Int2ObjectMap.Entry<MutableBitmap> arrayElement : arrayElements.int2ObjectEntrySet()) { for (Int2ObjectMap.Entry<MutableBitmap> arrayElement : arrayElements.int2ObjectEntrySet()) {
arrayElementDictionaryWriter.write(arrayElement.getIntKey()); arrayElementDictionaryWriter.write(arrayElement.getIntKey());
arrayElementIndexWriter.write( arrayElementIndexWriter.write(
indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue())
); );
} }
}
writeV0Header(channel, columnNameBytes); writeV0Header(channel, columnNameBytes);
if (variantTypeSetByte != null) { if (variantTypeSetByte != null) {
channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte})); channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte}));
} }
if (writeDictionary) {
if (dictionaryIdLookup.getStringBufferMapper() != null) { if (dictionaryIdLookup.getStringBufferMapper() != null) {
SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper(); SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper();
for (String internalName : fileMapper.getInternalFilenames()) { for (String internalName : fileMapper.getInternalFilenames()) {
@ -422,7 +436,11 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME);
} }
if (dictionaryIdLookup.getDoubleBuffer() != null) { if (dictionaryIdLookup.getDoubleBuffer() != null) {
writeInternal(smoosher, dictionaryIdLookup.getDoubleBuffer(), ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); writeInternal(
smoosher,
dictionaryIdLookup.getDoubleBuffer(),
ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME
);
} else { } else {
writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME);
} }
@ -433,6 +451,7 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
} }
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, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME);
writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME); writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME);
writeInternal(smoosher, arrayElementIndexWriter, ColumnSerializerUtils.ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME); writeInternal(smoosher, arrayElementIndexWriter, ColumnSerializerUtils.ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME);

View File

@ -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<ColumnHolder> 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 <T> QueryableProjection<T> findMatchingProjection(
CursorBuildSpec cursorBuildSpec,
SortedSet<AggregateProjectionMetadata> projections,
PhysicalColumnChecker physicalChecker,
Function<String, T> 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<String, String> remapColumns;
public ProjectionMatch(CursorBuildSpec cursorBuildSpec, Map<String, String> remapColumns)
{
this.cursorBuildSpec = cursorBuildSpec;
this.remapColumns = remapColumns;
}
public CursorBuildSpec getCursorBuildSpec()
{
return cursorBuildSpec;
}
public Map<String, String> getRemapColumns()
{
return remapColumns;
}
}
public static final class ProjectionMatchBuilder
{
private final Set<VirtualColumn> referencedVirtualColumns;
private final Map<String, String> remapColumns;
private final List<AggregatorFactory> 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);
}
}
}

View File

@ -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}.
* <p>
* 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)}.
* <p>
* 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})
* <p>
* 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<T>
{
private final CursorBuildSpec cursorBuildSpec;
private final Map<String, String> remapColumns;
private final T rowSelector;
public QueryableProjection(
CursorBuildSpec cursorBuildSpec,
Map<String, String> 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}.
* <p>
* 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}
* <p>
* 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;
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -54,6 +55,11 @@ public interface ColumnPartSerde
interface Deserializer interface Deserializer
{ {
void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); void read(
ByteBuffer buffer,
ColumnBuilder builder,
ColumnConfig columnConfig,
@Nullable ColumnHolder parent
);
} }
} }

View File

@ -77,7 +77,7 @@ public class ComplexColumnPartSerde implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() 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 // 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) // 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 // longer term this needs to be captured by making the serde provide this information, and then this should

View File

@ -29,8 +29,10 @@ import org.apache.druid.collections.spatial.ImmutableRTree;
import org.apache.druid.io.Channels; import org.apache.druid.io.Channels;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; 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.ColumnBuilder;
import org.apache.druid.segment.column.ColumnConfig; 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.StringEncodingStrategies;
import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.BitmapSerde; import org.apache.druid.segment.data.BitmapSerde;
@ -292,7 +294,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
return new Deserializer() return new Deserializer()
{ {
@Override @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 VERSION rVersion = VERSION.fromByte(buffer.get());
final int rFlags; final int rFlags;
@ -309,12 +316,17 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
builder.setType(ValueType.STRING); builder.setType(ValueType.STRING);
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier = final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
StringEncodingStrategies.getStringDictionarySupplier( if (parent != null) {
final Supplier<? extends BaseColumn> parentSupplier = parent.getColumnSupplier();
dictionarySupplier = ((StringUtf8DictionaryEncodedColumnSupplier<?>) parentSupplier).getDictionary();
} else {
dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
builder.getFileMapper(), builder.getFileMapper(),
buffer, buffer,
byteOrder byteOrder
); );
}
final WritableSupplier<ColumnarInts> rSingleValuedColumn; final WritableSupplier<ColumnarInts> rSingleValuedColumn;
final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn; final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn;

View File

@ -96,7 +96,7 @@ public class DoubleNumericColumnPartSerde implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return (buffer, builder, columnConfig) -> { return (buffer, builder, columnConfig, parent) -> {
final Supplier<ColumnarDoubles> column = CompressedColumnarDoublesSuppliers.fromByteBuffer( final Supplier<ColumnarDoubles> column = CompressedColumnarDoublesSuppliers.fromByteBuffer(
buffer, buffer,
byteOrder byteOrder

View File

@ -143,7 +143,7 @@ public class DoubleNumericColumnPartSerdeV2 implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return (buffer, builder, columnConfig) -> { return (buffer, builder, columnConfig, parent) -> {
int offset = buffer.getInt(); int offset = buffer.getInt();
int initialPos = buffer.position(); int initialPos = buffer.position();
final Supplier<ColumnarDoubles> column = CompressedColumnarDoublesSuppliers.fromByteBuffer( final Supplier<ColumnarDoubles> column = CompressedColumnarDoublesSuppliers.fromByteBuffer(

View File

@ -22,13 +22,10 @@ package org.apache.druid.segment.serde;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.segment.IndexIO; 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.column.ValueType;
import org.apache.druid.segment.data.CompressedColumnarFloatsSupplier; import org.apache.druid.segment.data.CompressedColumnarFloatsSupplier;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
/** /**
@ -99,11 +96,7 @@ public class FloatNumericColumnPartSerde implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return new Deserializer() return (buffer, builder, columnConfig, parent) -> {
{
@Override
public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig)
{
final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer(
buffer, buffer,
byteOrder byteOrder
@ -115,7 +108,6 @@ public class FloatNumericColumnPartSerde implements ColumnPartSerde
builder.setType(ValueType.FLOAT) builder.setType(ValueType.FLOAT)
.setHasMultipleValues(false) .setHasMultipleValues(false)
.setNumericColumnSupplier(columnSupplier); .setNumericColumnSupplier(columnSupplier);
}
}; };
} }
} }

View File

@ -141,7 +141,7 @@ public class FloatNumericColumnPartSerdeV2 implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return (buffer, builder, columnConfig) -> { return (buffer, builder, columnConfig, parent) -> {
int offset = buffer.getInt(); int offset = buffer.getInt();
int initialPos = buffer.position(); int initialPos = buffer.position();
final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer(

View File

@ -96,7 +96,7 @@ public class LongNumericColumnPartSerde implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return (buffer, builder, columnConfig) -> { return (buffer, builder, columnConfig, parent) -> {
final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer(
buffer, buffer,
byteOrder byteOrder

View File

@ -143,7 +143,7 @@ public class LongNumericColumnPartSerdeV2 implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return (buffer, builder, columnConfig) -> { return (buffer, builder, columnConfig, parent) -> {
int offset = buffer.getInt(); int offset = buffer.getInt();
int initialPos = buffer.position(); int initialPos = buffer.position();
final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer(

View File

@ -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.ColumnBuilder;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnConfig; 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.ColumnType;
import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.BitmapSerdeFactory;
@ -189,13 +190,14 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
private class StringColumnDeserializer implements Deserializer private class StringColumnDeserializer implements Deserializer
{ {
@Override @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( ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read(
byteOrder, byteOrder,
bitmapSerdeFactory, bitmapSerdeFactory,
buffer, buffer,
builder builder,
parent == null ? null : (ScalarStringColumnAndIndexSupplier) parent.getColumnSupplier()
); );
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
capabilitiesBuilder.setDictionaryEncoded(true); capabilitiesBuilder.setDictionaryEncoded(true);
@ -205,21 +207,26 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
builder.setHasNulls(hasNulls); builder.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier); builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false); 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 private class LongColumnDeserializer implements Deserializer
{ {
@Override @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( ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read(
byteOrder, byteOrder,
bitmapSerdeFactory, bitmapSerdeFactory,
buffer, buffer,
builder, builder,
columnConfig columnConfig,
parent == null ? null : (ScalarLongColumnAndIndexSupplier) parent.getColumnSupplier()
); );
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
// technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn // 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.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier); builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false); 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 private class DoubleColumnDeserializer implements Deserializer
{ {
@Override @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( ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read(
byteOrder, byteOrder,
bitmapSerdeFactory, bitmapSerdeFactory,
buffer, buffer,
builder, builder,
columnConfig columnConfig,
parent == null ? null : (ScalarDoubleColumnAndIndexSupplier) parent.getColumnSupplier()
); );
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
// technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn // 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.setHasNulls(hasNulls);
builder.setNestedCommonFormatColumnSupplier(supplier); builder.setNestedCommonFormatColumnSupplier(supplier);
builder.setIndexSupplier(supplier, true, false); 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 private class VariantColumnDeserializer implements Deserializer
{ {
@Override @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( VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read(
logicalType, logicalType,
byteOrder, byteOrder,
bitmapSerdeFactory, bitmapSerdeFactory,
buffer, buffer,
builder builder,
parent == null ? null : (VariantColumnAndIndexSupplier) parent.getColumnSupplier()
); );
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); 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 // 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 private class NestedColumnDeserializer implements Deserializer
{ {
@Override @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( NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
logicalType, logicalType,
@ -300,7 +317,8 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
builder, builder,
columnConfig, columnConfig,
bitmapSerdeFactory, bitmapSerdeFactory,
byteOrder byteOrder,
parent == null ? null : (NestedDataColumnSupplier) parent.getColumnSupplier()
); );
ColumnType simpleType = supplier.getLogicalType(); ColumnType simpleType = supplier.getLogicalType();
ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType;

View File

@ -105,7 +105,7 @@ public class NullColumnPartSerde implements ColumnPartSerde
@Override @Override
public Deserializer getDeserializer() public Deserializer getDeserializer()
{ {
return (buffer, builder, columnConfig) -> { return (buffer, builder, columnConfig, parent) -> {
builder.setHasMultipleValues(false) builder.setHasMultipleValues(false)
.setHasNulls(true) .setHasNulls(true)
// this is a bit sneaky, we set supplier to null here to act like a null column instead of a column // this is a bit sneaky, we set supplier to null here to act like a null column instead of a column

View File

@ -50,6 +50,11 @@ public class StringUtf8DictionaryEncodedColumnSupplier<TIndexed extends Indexed<
this.multiValuedColumn = multiValuedColumn; this.multiValuedColumn = multiValuedColumn;
} }
public Supplier<TIndexed> getDictionary()
{
return utf8Dictionary;
}
@Override @Override
public DictionaryEncodedColumn<?> get() public DictionaryEncodedColumn<?> get()
{ {

View File

@ -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<String, String> remap;
public RemapVectorColumnSelectorFactory(VectorColumnSelectorFactory delegate, Map<String, String> 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);
}
}

View File

@ -63,6 +63,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
private final String name; private final String name;
private final Expression expression; private final Expression expression;
private final Supplier<Expr> parsedExpression; private final Supplier<Expr> parsedExpression;
private final Supplier<Expr.BindingAnalysis> expressionAnalysis;
private final Supplier<byte[]> cacheKey; private final Supplier<byte[]> cacheKey;
/** /**
@ -126,6 +127,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
this.name = Preconditions.checkNotNull(name, "name"); this.name = Preconditions.checkNotNull(name, "name");
this.expression = new Expression(Preconditions.checkNotNull(expression, "expression"), outputType); this.expression = new Expression(Preconditions.checkNotNull(expression, "expression"), outputType);
this.parsedExpression = parsedExpression; this.parsedExpression = parsedExpression;
this.expressionAnalysis = Suppliers.memoize(parsedExpression.get()::analyzeInputs);
this.cacheKey = makeCacheKeySupplier(); this.cacheKey = makeCacheKeySupplier();
} }
@ -332,7 +334,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
@Override @Override
public List<String> requiredColumns() public List<String> requiredColumns()
{ {
return parsedExpression.get().analyzeInputs().getRequiredBindingsList(); return expressionAnalysis.get().getRequiredBindingsList();
} }
@Override @Override

View File

@ -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();
}
}

View File

@ -90,7 +90,7 @@ public class FrameFileTest extends InitializedNullHandlingTest
@Override @Override
int getRowCount() int getRowCount()
{ {
return TestIndex.getNoRollupIncrementalTestIndex().size(); return TestIndex.getNoRollupIncrementalTestIndex().numRows();
} }
}, },
MMAP { MMAP {

View File

@ -145,11 +145,11 @@ public class FrameProcessorExecutorTest
final ListenableFuture<Long> blasterFuture = exec.runFully(blaster, null); final ListenableFuture<Long> blasterFuture = exec.runFully(blaster, null);
final ListenableFuture<Long> muxerFuture = exec.runFully(muxer, null); final ListenableFuture<Long> muxerFuture = exec.runFully(muxer, null);
Assert.assertEquals(index.size(), (long) blasterFuture.get()); Assert.assertEquals(index.numRows(), (long) blasterFuture.get());
Assert.assertEquals(index.size() * 2, (long) muxerFuture.get()); Assert.assertEquals(index.numRows() * 2, (long) muxerFuture.get());
Assert.assertEquals( Assert.assertEquals(
index.size() * 2, index.numRows() * 2,
FrameTestUtil.readRowsFromFrameChannel( FrameTestUtil.readRowsFromFrameChannel(
new ReadableFileFrameChannel(FrameFile.open(outFile, null)), new ReadableFileFrameChannel(FrameFile.open(outFile, null)),
FrameReader.create(cursorFactory.getRowSignature()) FrameReader.create(cursorFactory.getRowSignature())

View File

@ -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.Granularity;
import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.java.util.common.granularity.PeriodGranularity; 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.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.DateTimeZone; import org.joda.time.DateTimeZone;
import org.joda.time.Days; import org.joda.time.Days;
@ -55,7 +58,7 @@ import java.util.TimeZone;
/** /**
*/ */
public class QueryGranularityTest public class QueryGranularityTest extends InitializedNullHandlingTest
{ {
@Test @Test
public void testIterableNone() public void testIterableNone()
@ -1056,6 +1059,54 @@ public class QueryGranularityTest
Assert.assertEquals("timestamp_floor(__time,'PT15M')", column.getExpression()); 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) private void assertBucketStart(final Granularity granularity, final DateTime in, final DateTime expectedInProperTz)
{ {
Assert.assertEquals( Assert.assertEquals(

View File

@ -88,7 +88,7 @@ public class TimeseriesQueryRunnerBonusTest
results = runTimeseriesCount(oneRowIndex); 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 size", 1, results.size());
Assert.assertEquals("result timestamp", DateTimes.of("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); 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")); Assert.assertEquals("result count metric", 1, (long) results.get(0).getValue().getLongMetric("rows"));
@ -103,7 +103,7 @@ public class TimeseriesQueryRunnerBonusTest
results = runTimeseriesCount(oneRowIndex); 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 size", 1, results.size());
Assert.assertEquals("result timestamp", DateTimes.of("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); 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")); Assert.assertEquals("result count metric", 2, (long) results.get(0).getValue().getLongMetric("rows"));

View File

@ -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<AggregateProjectionMetadata> 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();
}
}

View File

@ -30,6 +30,7 @@ import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.Druids; import org.apache.druid.query.Druids;
import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.IterableRowsCursorHelper;
import org.apache.druid.query.Result; 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.CountAggregatorFactory;
import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryConfig;
@ -59,6 +60,7 @@ import javax.annotation.Nullable;
import java.io.Closeable; import java.io.Closeable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
import java.util.stream.Collectors;
public class CursorHolderPreaggTest extends InitializedNullHandlingTest public class CursorHolderPreaggTest extends InitializedNullHandlingTest
{ {
@ -137,6 +139,15 @@ public class CursorHolderPreaggTest extends InitializedNullHandlingTest
return true; return true;
} }
@Nullable
@Override
public List<AggregatorFactory> getAggregatorsForPreAggregated()
{
return spec.getAggregators()
.stream().map(AggregatorFactory::getCombiningFactory)
.collect(Collectors.toList());
}
@Override @Override
public void close() public void close()
{ {

View File

@ -273,7 +273,7 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling
ColumnDescriptor serde = mapper.readValue( ColumnDescriptor serde = mapper.readValue(
IndexIO.SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class IndexIO.SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class
); );
return serde.read(byteBuffer, columnConfig, smooshedFiles); return serde.read(byteBuffer, columnConfig, smooshedFiles, null);
} }
} }
} }

View File

@ -410,7 +410,8 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest
null, null,
Granularities.NONE, Granularities.NONE,
Boolean.TRUE, Boolean.TRUE,
Cursors.ascendingTimeOrder() Cursors.ascendingTimeOrder(),
null
), ),
index.getMetadata() index.getMetadata()
); );

Some files were not shown because too many files have changed in this diff Show More