diff --git a/README.md b/README.md index b4c9a5e80a..d886de962c 100644 --- a/README.md +++ b/README.md @@ -23,7 +23,7 @@ The Record Layer is a Java API providing a record-oriented store on top of Found across one or more record types, and a query planner capable of automatic selection of indexes. * **Many record stores, shared schema** - The Record Layer provides the - the ability to support many discrete record store instances, all with + ability to support many discrete record store instances, all with a shared (and evolving) schema. For example, rather than modeling a single database in which to store all users' data, each user can be given their own record store, perhaps sharded across different FDB diff --git a/docs/ReleaseNotes.md b/docs/ReleaseNotes.md index c1f66f649a..5a6e05d370 100644 --- a/docs/ReleaseNotes.md +++ b/docs/ReleaseNotes.md @@ -30,6 +30,7 @@ The Guava dependency version has been updated to 31.1. Projects may need to chec * **Feature** Feature 3 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Feature** Feature 4 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Feature** Feature 5 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) +* **Feature** Support planning aggregate indexes in Cascades. [(Issue #1885)](https://github.com/FoundationDB/fdb-record-layer/issues/1885) * **Breaking change** Change 1 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Breaking change** Change 2 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) * **Breaking change** Change 3 [(Issue #NNN)](https://github.com/FoundationDB/fdb-record-layer/issues/NNN) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordMetaData.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordMetaData.java index 5515b38d4d..ca9e948516 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordMetaData.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/RecordMetaData.java @@ -38,7 +38,9 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -690,9 +692,12 @@ public static Map getFieldDescriptorMapFrom @Nonnull private static Map getFieldDescriptorMap(@Nonnull final Stream recordTypeStream) { + // todo: should be removed https://github.com/FoundationDB/fdb-record-layer/issues/1884 return recordTypeStream + .sorted(Comparator.comparing(RecordType::getName)) .flatMap(recordType -> recordType.getDescriptor().getFields().stream()) .collect(Collectors.groupingBy(Descriptors.FieldDescriptor::getName, + LinkedHashMap::new, Collectors.reducing(null, (fieldDescriptor, fieldDescriptor2) -> { Verify.verify(fieldDescriptor != null || fieldDescriptor2 != null); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/metadata/Index.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/metadata/Index.java index 395a588634..2489e96315 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/metadata/Index.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/metadata/Index.java @@ -259,6 +259,12 @@ public String getName() { return name; } + /** + * Returns the type of the index. + * + * @return the type of the index. + * @see IndexTypes + */ @Nonnull public String getType() { return type; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/AggregateIndexExpansionVisitor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/AggregateIndexExpansionVisitor.java new file mode 100644 index 0000000000..503e2c2bbf --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/AggregateIndexExpansionVisitor.java @@ -0,0 +1,260 @@ +/* + * AggregateIndexExpansionVisitor.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.query.plan.cascades; + +import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.record.metadata.IndexTypes; +import com.apple.foundationdb.record.metadata.RecordType; +import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; +import com.apple.foundationdb.record.metadata.expressions.KeyExpression; +import com.apple.foundationdb.record.query.plan.cascades.expressions.GroupByExpression; +import com.apple.foundationdb.record.query.plan.cascades.expressions.MatchableSortExpression; +import com.apple.foundationdb.record.query.plan.cascades.expressions.SelectExpression; +import com.apple.foundationdb.record.query.plan.cascades.predicates.ValueComparisonRangePredicate; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; +import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; +import com.apple.foundationdb.record.query.plan.cascades.values.AggregateValue; +import com.apple.foundationdb.record.query.plan.cascades.values.CountValue; +import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; +import com.apple.foundationdb.record.query.plan.cascades.values.IndexOnlyAggregateValue; +import com.apple.foundationdb.record.query.plan.cascades.values.NumericAggregationValue; +import com.apple.foundationdb.record.query.plan.cascades.values.QuantifiedObjectValue; +import com.apple.foundationdb.record.query.plan.cascades.values.RecordConstructorValue; +import com.apple.foundationdb.record.query.plan.cascades.values.Value; +import com.apple.foundationdb.record.query.plan.cascades.values.Values; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.base.Verify; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import org.apache.commons.lang3.tuple.Pair; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Expands an aggregate index into a {@link MatchCandidate}. The generation will expand a {@link KeyExpression} into a + * group by triplet QGM comprising a select-where, group-by-expression, and select-having, the triplet is followed + * by an optional {@link MatchableSortExpression} that defines the sort order of the match candidate stream of records. + */ +public class AggregateIndexExpansionVisitor extends KeyExpressionExpansionVisitor + implements ExpansionVisitor { + + @Nonnull + private static final Supplier> allowedIndexTypes = Suppliers.memoize(AggregateIndexExpansionVisitor::computeAllowedIndexesMap); + + @Nonnull + private static final Supplier>> aggregateMap = Suppliers.memoize(AggregateIndexExpansionVisitor::computeAggregateMap); + + @Nonnull + private final Index index; + + @Nonnull + private final Collection recordTypes; + + @Nonnull + private final GroupingKeyExpression groupingKeyExpression; + + /** + * Constructs a new instance of {@link AggregateIndexExpansionVisitor}. + * + * @param index The target index. + * @param recordTypes The indexed record types. + */ + public AggregateIndexExpansionVisitor(@Nonnull final Index index, @Nonnull final Collection recordTypes) { + Preconditions.checkArgument(allowedIndexTypes.get().contains(index.getType())); + Preconditions.checkArgument(index.getRootExpression() instanceof GroupingKeyExpression); + this.index = index; + this.groupingKeyExpression = ((GroupingKeyExpression)index.getRootExpression()); + this.recordTypes = recordTypes; + } + + /** + * Creates a new match candidate representing the aggregate index. + * + * @param baseQuantifierSupplier a quantifier supplier to create base data access + * @param ignored the primary key of the data object the caller wants to access, this parameter is ignored since + * an aggregate index does not possess primary key information, must be {@code null}. + * @param isReverse an indicator whether the result set is expected to be returned in reverse order. + * @return A match candidate representing the aggregate index. + */ + @Nonnull + @Override + public MatchCandidate expand(@Nonnull final java.util.function.Supplier baseQuantifierSupplier, + @Nullable final KeyExpression ignored, + final boolean isReverse) { + Verify.verify(ignored == null); + final var baseQuantifier = baseQuantifierSupplier.get(); + final var groupingAndGroupedCols = Value.fromKeyExpressions(groupingKeyExpression.normalizeKeyForPositions(), baseQuantifier.getAlias(), baseQuantifier.getFlowedObjectType()); + final var groupingValues = groupingAndGroupedCols.subList(0, groupingKeyExpression.getGroupingCount()); + final var groupedValues = groupingAndGroupedCols.subList(groupingKeyExpression.getGroupingCount(), groupingAndGroupedCols.size()); + + if (groupedValues.size() > 1) { + throw new UnsupportedOperationException(String.format("aggregate index is expected to contain exactly one aggregation, however it contains %d aggregations", groupedValues.size())); + } + + // 1. create a SELECT-WHERE expression. + final var selectWhereQun = constructSelectWhere(baseQuantifier, groupingValues); + + // 2. create a GROUP-BY expression on top. + final var groupByQun = constructGroupBy(baseQuantifier.getAlias(), groupedValues, selectWhereQun); + + // 3. construct SELECT-HAVING with SORT on top. + final var selectHavingAndPlaceholderAliases = constructSelectHaving(groupByQun); + final var selectHaving = selectHavingAndPlaceholderAliases.getLeft(); + final var placeHolderAliases = selectHavingAndPlaceholderAliases.getRight(); + + // 4. add sort on top, if necessary, this will be absorbed later on as an ordering property of the match candidate. + final var maybeWithSort = placeHolderAliases.isEmpty() + ? GroupExpressionRef.of(selectHaving) // single group, sort by constant + : GroupExpressionRef.of(new MatchableSortExpression(placeHolderAliases, isReverse, selectHaving)); + + final var traversal = ExpressionRefTraversal.withRoot(maybeWithSort); + return new AggregateIndexMatchCandidate(index, + traversal, + placeHolderAliases, + recordTypes, + baseQuantifier.getFlowedObjectType(), + groupByQun.getRangesOver().get().getResultValue(), + selectHaving.getResultValue()); + } + + @Nonnull + private Quantifier constructSelectWhere(@Nonnull final Quantifier.ForEach baseQuantifier, final List groupingValues) { + final var allExpansionsBuilder = ImmutableList.builder(); + allExpansionsBuilder.add(GraphExpansion.ofQuantifier(baseQuantifier)); + + // add the SELECT-WHERE part, where we expose grouping and grouped columns, allowing query fragments that governs + // only these columns to properly bind to this part, similar to how value indices work. + final var keyValues = Lists.newArrayList(); + final var valueValues = Lists.newArrayList(); + final var state = VisitorState.of(keyValues, valueValues, baseQuantifier, ImmutableList.of(), 0, 0); + final var selectWhereGraphExpansion = pop(groupingKeyExpression.getWholeKey().expand(push(state))); + + // add an RCV column representing the grouping columns as the first result set column + final var groupingValue = RecordConstructorValue.ofColumns(groupingValues + .stream() + .map(Column::unnamedOf) // REMOVE: name is important? + .collect(Collectors.toList())); + + // flow all underlying quantifiers in their own QOV columns. + final var builder = GraphExpansion.builder(); + // we need to refer to the following colum later on in GroupByExpression, but since its ordinal position is fixed, we can simply refer + // to it using an ordinal FieldAccessor (we do the same in plan generation). + builder.addResultColumn(Column.unnamedOf(groupingValue)); + Stream.concat(Stream.of(baseQuantifier), selectWhereGraphExpansion.getQuantifiers().stream()) + .forEach(qun -> { + final var quantifiedValue = QuantifiedObjectValue.of(qun.getAlias(), qun.getFlowedObjectType()); + builder.addResultColumn(Column.of(Type.Record.Field.of( quantifiedValue.getResultType(), Optional.of(qun.getAlias().getId())), quantifiedValue)); + }); + builder.addAllPlaceholders(selectWhereGraphExpansion.getPlaceholders()); + builder.addAllPredicates(selectWhereGraphExpansion.getPredicates()); + builder.addAllQuantifiers(selectWhereGraphExpansion.getQuantifiers()); + allExpansionsBuilder.add(builder.build()); + + return Quantifier.forEach(GroupExpressionRef.of(GraphExpansion.ofOthers(allExpansionsBuilder.build()).buildSelect())); + } + + @SuppressWarnings("deprecation") + @Nonnull + private Quantifier constructGroupBy(@Nonnull final CorrelationIdentifier baseQuantifierCorrelationIdentifier, + @Nonnull final List groupedValue, + @Nonnull final Quantifier selectWhereQun) { + // construct aggregation RCV + final int[] cnt = {0}; + final var aggregateValue = RecordConstructorValue.ofColumns(groupedValue.stream().map(gv -> { + final var prefixedFieldPath = Stream.concat(Stream.of(baseQuantifierCorrelationIdentifier.getId()), ((FieldValue)gv).getFieldPathNames().stream()).collect(Collectors.toList()); + final var groupedFieldReference = FieldValue.ofFieldNames(selectWhereQun.getFlowedObjectValue(), prefixedFieldPath); + return (AggregateValue)aggregateMap.get().get(index.getType()).encapsulate(TypeRepository.newBuilder(), List.of(groupedFieldReference)); + }).map(av -> Column.of(Type.Record.Field.of(av.getResultType(), Optional.of(generateAggregateFieldName(cnt[0]++))), av)).collect(Collectors.toList())); + + // construct grouping column(s) value, the grouping column is _always_ fixed at position-0 in the underlying select-where. + final var groupingColsValue = FieldValue.ofOrdinalNumber(selectWhereQun.getFlowedObjectValue(), 0); + + if (groupingColsValue.getResultType() instanceof Type.Record && ((Type.Record)groupingColsValue.getResultType()).getFields().isEmpty()) { + return Quantifier.forEach(GroupExpressionRef.of(new GroupByExpression(aggregateValue, null, selectWhereQun))); + } else { + return Quantifier.forEach(GroupExpressionRef.of(new GroupByExpression(aggregateValue, groupingColsValue, selectWhereQun))); + } + } + + @Nonnull + private String generateAggregateFieldName(int fieldIdx) { + return index.getName() + "_" + index.getType() + "_agg_" + fieldIdx; + } + + @Nonnull + private Pair> constructSelectHaving(@Nonnull final Quantifier groupByQun) { + // the grouping value in GroupByExpression comes first (if set). + @Nullable final var groupingValueReference = + (groupByQun.getRangesOver().get() instanceof GroupByExpression && ((GroupByExpression)groupByQun.getRangesOver().get()).getGroupingValue() == null) + ? null + : FieldValue.ofOrdinalNumber(groupByQun.getFlowedObjectValue(), 0); + + final var aggregateValueReference = FieldValue.ofOrdinalNumber(FieldValue.ofOrdinalNumber(groupByQun.getFlowedObjectValue(), groupingValueReference == null ? 0 : 1), 0); + + final var placeholderAliases = ImmutableList.builder(); + final var selectHavingGraphExpansionBuilder = GraphExpansion.builder().addQuantifier(groupByQun); + if (groupingValueReference != null) { + Values.deconstructRecord(groupingValueReference).forEach(v -> { + final var field = (FieldValue)v; + final var placeholder = v.asPlaceholder(CorrelationIdentifier.uniqueID(ValueComparisonRangePredicate.Placeholder.class)); + placeholderAliases.add(placeholder.getAlias()); + selectHavingGraphExpansionBuilder + .addResultColumn(Column.unnamedOf(field)) + .addPlaceholder(placeholder) + .addPredicate(placeholder); + }); + } + selectHavingGraphExpansionBuilder.addResultColumn(Column.unnamedOf(aggregateValueReference)); // TODO should we also add the aggregate reference as a placeholder? // REMOVE: name is important? + return Pair.of(selectHavingGraphExpansionBuilder.build().buildSelect(), placeholderAliases.build()); + } + + @Nonnull + private static Set computeAllowedIndexesMap() { + final ImmutableSet.Builder setBuilder = ImmutableSet.builder(); + setBuilder.add(IndexTypes.COUNT); + setBuilder.add(IndexTypes.SUM); + setBuilder.add(IndexTypes.MIN_EVER_LONG); + setBuilder.add(IndexTypes.MAX_EVER_LONG); + return setBuilder.build(); + } + + @Nonnull + private static Map> computeAggregateMap() { + final ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); + mapBuilder.put(IndexTypes.MAX_EVER_LONG, new IndexOnlyAggregateValue.MaxEverLongFn()); + mapBuilder.put(IndexTypes.MIN_EVER_LONG, new IndexOnlyAggregateValue.MinEverLongFn()); + mapBuilder.put(IndexTypes.SUM, new NumericAggregationValue.SumFn()); + mapBuilder.put(IndexTypes.COUNT, new CountValue.CountFn()); + return mapBuilder.build(); + } +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/AggregateIndexMatchCandidate.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/AggregateIndexMatchCandidate.java new file mode 100644 index 0000000000..f7eaa4bd42 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/AggregateIndexMatchCandidate.java @@ -0,0 +1,381 @@ +/* + * AggregateIndexMatchCandidate.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.query.plan.cascades; + +import com.apple.foundationdb.record.IndexScanType; +import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordMetaData; +import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.record.metadata.RecordType; +import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; +import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; +import com.apple.foundationdb.record.metadata.expressions.KeyExpression; +import com.apple.foundationdb.record.provider.foundationdb.IndexScanComparisons; +import com.apple.foundationdb.record.query.expressions.Comparisons; +import com.apple.foundationdb.record.query.plan.AvailableFields; +import com.apple.foundationdb.record.query.plan.IndexKeyValueToPartialRecord; +import com.apple.foundationdb.record.query.plan.ScanComparisons; +import com.apple.foundationdb.record.query.plan.cascades.expressions.RelationalExpression; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; +import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; +import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; +import com.apple.foundationdb.record.query.plan.cascades.values.Value; +import com.apple.foundationdb.record.query.plan.cascades.values.Values; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryAggregateIndexPlan; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryFetchFromPartialRecordPlan; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryIndexPlan; +import com.google.common.base.Preconditions; +import com.google.common.base.Verify; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.ImmutableSetMultimap; +import com.google.common.primitives.ImmutableIntArray; +import com.google.protobuf.Descriptors; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Case class that represents a grouping index with aggregate function(s). + */ +public class AggregateIndexMatchCandidate implements MatchCandidate { + + // The backing index metadata structure. + @Nonnull + private final Index index; + + // The expression representation of the match candidate. + @Nonnull + private final ExpressionRefTraversal traversal; + + // list of aliases pertaining ordering information. + @Nonnull + private final List sargableAndOrderAliases; + + // list of base indexed record types. + @Nonnull + private final List recordTypes; + + @Nonnull + private final Type baseType; + + @Nonnull + private final Value groupByResultValue; + + @Nonnull + private final Value selectHavingResultValue; + + /** + * Creates a new instance of {@link AggregateIndexMatchCandidate}. + * + * @param index The underlying index. + * @param traversal The expression representation of the match candidate. + * @param sargableAndOrderAliases A list of sargable and order aliases. + * @param recordTypes The underlying base record types. + * @param baseType The base type. + * @param groupByResultValue The group by expression result value. + * @param selectHavingResultValue The select-having expression result value. + */ + public AggregateIndexMatchCandidate(@Nonnull final Index index, + @Nonnull final ExpressionRefTraversal traversal, + @Nonnull final List sargableAndOrderAliases, + @Nonnull final Collection recordTypes, + @Nonnull final Type baseType, + @Nonnull final Value groupByResultValue, + @Nonnull final Value selectHavingResultValue) { + Preconditions.checkArgument(!recordTypes.isEmpty()); + this.index = index; + this.traversal = traversal; + this.sargableAndOrderAliases = sargableAndOrderAliases; + this.recordTypes = ImmutableList.copyOf(recordTypes); + this.baseType = baseType; + this.groupByResultValue = groupByResultValue; + this.selectHavingResultValue = selectHavingResultValue; + } + + @Nonnull + @Override + public String getName() { + return index.getName(); + } + + @Nonnull + @Override + public ExpressionRefTraversal getTraversal() { + return traversal; + } + + @Nonnull + @Override + public List getSargableAliases() { + return sargableAndOrderAliases; // only these for now, later on we should also add the aggregated column alias as well. + } + + @Nonnull + @Override + public List getOrderingAliases() { + return sargableAndOrderAliases; + } + + @Nonnull + @Override + public KeyExpression getFullKeyExpression() { + return index.getRootExpression(); + } + + @Override + public boolean createsDuplicates() { + return index.getRootExpression().createsDuplicates(); + } + + @Override + public int getColumnSize() { + return index.getColumnSize(); + } + + @Override + public boolean isUnique() { + return index.isUnique(); + } + + @Nonnull + @Override + public List computeMatchedOrderingParts(@Nonnull final MatchInfo matchInfo, @Nonnull final List sortParameterIds, final boolean isReverse) { + final var parameterBindingMap = matchInfo.getParameterBindingMap(); + final var parameterBindingPredicateMap = matchInfo.getParameterPredicateMap(); + + final var normalizedKeys = + getFullKeyExpression().normalizeKeyForPositions(); + + final var builder = ImmutableList.builder(); + final var candidateParameterIds = getOrderingAliases(); + + for (final var parameterId : sortParameterIds) { + final var ordinalInCandidate = candidateParameterIds.indexOf(parameterId); + Verify.verify(ordinalInCandidate >= 0); + final var normalizedKeyExpression = normalizedKeys.get(ordinalInCandidate); + + Objects.requireNonNull(parameterId); + Objects.requireNonNull(normalizedKeyExpression); + @Nullable final var comparisonRange = parameterBindingMap.get(parameterId); + @Nullable final var queryPredicate = parameterBindingPredicateMap.get(parameterId); + + Verify.verify(comparisonRange == null || comparisonRange.getRangeType() == ComparisonRange.Type.EMPTY || queryPredicate != null); + + if (normalizedKeyExpression.createsDuplicates()) { + if (comparisonRange != null) { + if (comparisonRange.getRangeType() == ComparisonRange.Type.EQUALITY) { + continue; + } else { + break; + } + } else { + break; + } + } + + // + // Compute a Value for this normalized key. + // + final var value = + new ScalarTranslationVisitor(normalizedKeyExpression).toResultValue(Quantifier.CURRENT, + baseType); + + builder.add( + MatchedOrderingPart.of(value, + comparisonRange == null ? ComparisonRange.Type.EMPTY : comparisonRange.getRangeType(), + queryPredicate, + isReverse)); + } + + return builder.build(); + } + + @Nonnull + @Override + public Ordering computeOrderingFromScanComparisons(@Nonnull final ScanComparisons scanComparisons, final boolean isReverse, final boolean isDistinct) { + final var equalityBoundValueMapBuilder = ImmutableSetMultimap.builder(); + final var groupingKey = ((GroupingKeyExpression)index.getRootExpression()).getGroupingSubKey(); + + if (groupingKey instanceof EmptyKeyExpression) { + // TODO this should be something like anything-order. + return Ordering.emptyOrder(); + } + + // TODO include the aggregate Value itself in the ordering. + final var normalizedKeyExpressions = groupingKey.normalizeKeyForPositions(); + final var equalityComparisons = scanComparisons.getEqualityComparisons(); + + for (var i = 0; i < equalityComparisons.size(); i++) { + final var normalizedKeyExpression = normalizedKeyExpressions.get(i); + final var comparison = equalityComparisons.get(i); + + if (normalizedKeyExpression.createsDuplicates()) { + continue; + } + + final var normalizedValue = + new ScalarTranslationVisitor(normalizedKeyExpression).toResultValue(Quantifier.CURRENT, + baseType); + equalityBoundValueMapBuilder.put(normalizedValue, comparison); + } + + final var result = ImmutableList.builder(); + for (var i = scanComparisons.getEqualitySize(); i < normalizedKeyExpressions.size(); i++) { + final var normalizedKeyExpression = normalizedKeyExpressions.get(i); + + if (normalizedKeyExpression.createsDuplicates()) { + break; + } + + // + // Note that it is not really important here if the keyExpression can be normalized in a lossless way + // or not. A key expression containing repeated fields is sort-compatible with its normalized key + // expression. We used to refuse to compute the sort order in the presence of repeats, however, + // I think that restriction can be relaxed. + // + final var normalizedValue = + new ScalarTranslationVisitor(normalizedKeyExpression).toResultValue(Quantifier.CURRENT, + baseType); + + result.add(OrderingPart.of(normalizedValue, isReverse)); + } + + return new Ordering(equalityBoundValueMapBuilder.build(), result.build(), isDistinct); + } + + @Nonnull + @Override + public RelationalExpression toEquivalentExpression(@Nonnull final PartialMatch partialMatch, @Nonnull final PlanContext planContext, @Nonnull final List comparisonRanges) { + final var reverseScanOrder = + partialMatch.getMatchInfo() + .deriveReverseScanOrder() + .orElseThrow(() -> new RecordCoreException("match info should unambiguously indicate reversed-ness of scan")); + + final var baseRecordType = Type.Record.fromFieldDescriptorsMap(RecordMetaData.getFieldDescriptorMapFromTypes(recordTypes)); + + // reset indexes of all fields, such that we can normalize them + final var type = reset(groupByResultValue.getResultType()); + final var messageBuilder = TypeRepository.newBuilder().addTypeIfNeeded(type).build().newMessageBuilder(type); + final var messageDescriptor = Objects.requireNonNull(messageBuilder).getDescriptorForType(); + + final var indexEntryConverter = createIndexEntryConverter(messageDescriptor); + final var aggregateIndexScan = new RecordQueryIndexPlan(index.getName(), + null, + new IndexScanComparisons(IndexScanType.BY_GROUP, toScanComparisons(comparisonRanges)), + planContext.getPlannerConfiguration().getIndexFetchMethod(), + RecordQueryFetchFromPartialRecordPlan.FetchIndexRecords.PRIMARY_KEY, + reverseScanOrder, + false, + partialMatch.getMatchCandidate(), + baseRecordType); + + return new RecordQueryAggregateIndexPlan(aggregateIndexScan, + recordTypes.get(0).getName(), + indexEntryConverter, + messageDescriptor, + groupByResultValue); + } + + @Nonnull + @Override + public List getQueriedRecordTypes() { + return recordTypes; + } + + @SuppressWarnings("UnstableApiUsage") + @Nonnull + private IndexKeyValueToPartialRecord createIndexEntryConverter(final Descriptors.Descriptor messageDescriptor) { + final var selectHavingFields = Values.deconstructRecord(selectHavingResultValue); + final var groupingCount = ((GroupingKeyExpression)index.getRootExpression()).getGroupingCount(); + Verify.verify(selectHavingFields.size() >= groupingCount); + + // key structure : KEY(groupingCol1, groupingCol2, ... groupingColn), VALUE(agg(coln+1)) + // groupingCount : n+1 + // select-having value structure: (groupingCol1, groupingCol2, ... groupingColn, agg(coln+1)) + + final IndexKeyValueToPartialRecord.Builder builder = IndexKeyValueToPartialRecord.newBuilder(messageDescriptor); + for (int i = 0; i < groupingCount; i++) { + final Value keyValue = selectHavingFields.get(i); + if (keyValue instanceof FieldValue) { + final AvailableFields.FieldData fieldData = AvailableFields.FieldData.ofUnconditional(IndexKeyValueToPartialRecord.TupleSource.KEY, ImmutableIntArray.of(i)); + addCoveringField(builder, (FieldValue)keyValue, fieldData); + } + } + for (int i = groupingCount; i < selectHavingFields.size(); i++) { + final Value keyValue = selectHavingFields.get(i); + if (keyValue instanceof FieldValue) { + final AvailableFields.FieldData fieldData = AvailableFields.FieldData.ofUnconditional(IndexKeyValueToPartialRecord.TupleSource.VALUE, ImmutableIntArray.of(i - groupingCount)); + addCoveringField(builder, (FieldValue)keyValue, fieldData); + } + } + + if (!builder.isValid()) { + throw new RecordCoreException(String.format("could not generate a covering index scan operator for '%s'; Invalid mapping between index entries to partial record", index.getName())); + } + return builder.build(); + } + + @Nonnull + private static Type reset(@Nonnull final Type type) { + if (type instanceof Type.Record) { + return Type.Record.fromFields(((Type.Record)type).getFields().stream().map(f -> Type.Record.Field.of( + reset(f.getFieldType()), + f.getFieldNameOptional(), + Optional.empty())).collect(Collectors.toList())); + } + return type; + } + + private static void addCoveringField(@Nonnull IndexKeyValueToPartialRecord.Builder builder, + @Nonnull FieldValue fieldValue, + @Nonnull AvailableFields.FieldData fieldData) { + // TODO field names are for debugging purposes only, we should probably use field ordinals here instead. + final var simplifiedFieldValue = (FieldValue)fieldValue.simplify(AliasMap.emptyMap(), ImmutableSet.of()); + for (final var maybeFieldName : simplifiedFieldValue.getFieldPrefix().getFieldNamesMaybe()) { + Verify.verify(maybeFieldName.isPresent()); + builder = builder.getFieldBuilder(maybeFieldName.get()); + } + + // TODO not sure what to do with the null standing requirement + + final var maybeFieldName = simplifiedFieldValue.getLastFieldName(); + Verify.verify(maybeFieldName.isPresent()); + final String fieldName = maybeFieldName.get(); + if (!builder.hasField(fieldName)) { + builder.addField(fieldName, fieldData.getSource(), fieldData.getCopyIfPredicate(), fieldData.getOrdinalPath()); + } + } + + @Nonnull + private static ScanComparisons toScanComparisons(@Nonnull final List comparisonRanges) { + final ScanComparisons.Builder builder = new ScanComparisons.Builder(); + for (ComparisonRange comparisonRange : comparisonRanges) { + builder.addComparisonRange(comparisonRange); + } + return builder.build(); + } +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/CascadesPlanner.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/CascadesPlanner.java index 3cbe31a1c4..2d77018bb4 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/CascadesPlanner.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/CascadesPlanner.java @@ -973,7 +973,7 @@ protected Object getBindable() { } /** - * Adjust Match Task. Attempts to improve an existing partial partial match on a (group, expression) pair + * Adjust Match Task. Attempts to improve an existing partial match on a (group, expression) pair * to a better one by enqueuing rules defined on {@link PartialMatch}. * * Simplified enqueue/execute overview: diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/KeyExpressionExpansionVisitor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/KeyExpressionExpansionVisitor.java index 2f8fc5647f..3b3a05fcdc 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/KeyExpressionExpansionVisitor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/KeyExpressionExpansionVisitor.java @@ -32,10 +32,10 @@ import com.apple.foundationdb.record.metadata.expressions.ThenKeyExpression; import com.apple.foundationdb.record.query.plan.cascades.KeyExpressionExpansionVisitor.VisitorState; import com.apple.foundationdb.record.query.plan.cascades.expressions.SelectExpression; +import com.apple.foundationdb.record.query.plan.cascades.predicates.ValueComparisonRangePredicate.Placeholder; import com.apple.foundationdb.record.query.plan.cascades.values.EmptyValue; import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; -import com.apple.foundationdb.record.query.plan.cascades.predicates.ValueComparisonRangePredicate.Placeholder; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchCandidate.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchCandidate.java index 3d4910a821..1106282da5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchCandidate.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchCandidate.java @@ -25,6 +25,8 @@ import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.metadata.RecordType; +import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; +import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; import com.apple.foundationdb.record.metadata.expressions.KeyExpression; import com.apple.foundationdb.record.query.plan.ScanComparisons; import com.apple.foundationdb.record.query.plan.cascades.expressions.FullUnorderedScanExpression; @@ -160,7 +162,7 @@ default Map computeBoundParameterPrefixM * Compute a list of {@link MatchedOrderingPart}s which forms a bridge to relate {@link KeyExpression}s and * {@link QueryPredicate}s. * @param matchInfo a pre-existing match info structure - * @param sortParameterIds the query should be ordered by + * @param sortParameterIds the parameter IDs which the query should be ordered by * @param isReverse reversed-ness of the order * @return a list of bound key parts that express the order of the outgoing data stream and their respective mappings * between query and match candidate @@ -207,7 +209,7 @@ default RelationalExpression toEquivalentExpression(@Nonnull final PartialMatch /** * Creates a logical expression that represents a scan over the materialized candidate data. This method is expected * to be implemented by specific implementations of {@link MatchCandidate}. - * @param partialMatch the {@link PartialMatch} that matched th query and the candidate + * @param partialMatch the {@link PartialMatch} that matched the query and the candidate * @param planContext the plan context for the query * @param comparisonRanges a {@link List} of {@link ComparisonRange}s to be applied * @return a new {@link RelationalExpression} @@ -237,6 +239,20 @@ default SetMultimap, RelationalExp return refToExpressionMap; } + @Nonnull + List getQueriedRecordTypes(); + + int getColumnSize(); + + boolean isUnique(); + + @Nonnull + default Set getQueriedRecordTypeNames() { + return getQueriedRecordTypes().stream() + .map(RecordType::getName) + .collect(ImmutableSet.toImmutableSet()); + } + @Nonnull static Iterable fromIndexDefinition(@Nonnull final RecordMetaData metaData, @Nonnull final Index index, @@ -255,22 +271,11 @@ static Iterable fromIndexDefinition(@Nonnull final RecordMetaDat final var availableRecordTypeNames = recordTypeMap.keySet(); final var availableRecordTypes = recordTypeMap.values(); - final var type = index.getType(); - - if (IndexTypes.VALUE.equals(type)) { - expandIndexMatchCandidate(index, - availableRecordTypeNames, - availableRecordTypes, - queriedRecordTypeNames, - queriedRecordTypes, - isReverse, - commonPrimaryKeyForIndex, - new ValueIndexExpansionVisitor(index, queriedRecordTypes)).ifPresent(resultBuilder::add); - } + final var indexType = index.getType(); - if (IndexTypes.RANK.equals(type)) { - // For rank() we need to create at least two candidates. One for BY_RANK scans and one for BY_VALUE scans. - expandIndexMatchCandidate(index, + switch (indexType) { + case IndexTypes.VALUE: + expandIndexMatchCandidate(index, availableRecordTypeNames, availableRecordTypes, queriedRecordTypeNames, @@ -278,18 +283,49 @@ static Iterable fromIndexDefinition(@Nonnull final RecordMetaDat isReverse, commonPrimaryKeyForIndex, new ValueIndexExpansionVisitor(index, queriedRecordTypes)).ifPresent(resultBuilder::add); - - expandIndexMatchCandidate(index, - availableRecordTypeNames, - availableRecordTypes, - queriedRecordTypeNames, - queriedRecordTypes, - isReverse, - commonPrimaryKeyForIndex, - new WindowedIndexExpansionVisitor(index, queriedRecordTypes)) - .ifPresent(resultBuilder::add); + break; + case IndexTypes.RANK: + // For rank() we need to create at least two candidates. One for BY_RANK scans and one for BY_VALUE scans. + expandIndexMatchCandidate(index, + availableRecordTypeNames, + availableRecordTypes, + queriedRecordTypeNames, + queriedRecordTypes, + isReverse, + commonPrimaryKeyForIndex, + new ValueIndexExpansionVisitor(index, queriedRecordTypes)).ifPresent(resultBuilder::add); + + expandIndexMatchCandidate(index, + availableRecordTypeNames, + availableRecordTypes, + queriedRecordTypeNames, + queriedRecordTypes, + isReverse, + commonPrimaryKeyForIndex, + new WindowedIndexExpansionVisitor(index, queriedRecordTypes)) + .ifPresent(resultBuilder::add); + break; + case IndexTypes.MAX_EVER_LONG: // fallthrough + case IndexTypes.MIN_EVER_LONG: // fallthrough + case IndexTypes.SUM: // fallthrough + case IndexTypes.COUNT: + final var rootExpression = index.getRootExpression(); + if (!(rootExpression instanceof GroupingKeyExpression) || ((GroupingKeyExpression)rootExpression).getWholeKey() instanceof EmptyKeyExpression) { + break; + } + expandIndexMatchCandidate(index, + availableRecordTypeNames, + availableRecordTypes, + queriedRecordTypeNames, + queriedRecordTypes, + isReverse, + null, + new AggregateIndexExpansionVisitor(index, queriedRecordTypes)) + .ifPresent(resultBuilder::add); + break; + default: + break; } - return resultBuilder.build(); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchInfo.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchInfo.java index ff2d81a6f5..904335ec34 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchInfo.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/MatchInfo.java @@ -217,6 +217,7 @@ public Optional deriveReverseScanOrder() { } } + @Nonnull public MatchInfo withOrderingInfo(@Nonnull final List matchedOrderingParts) { return new MatchInfo(parameterBindingMap, quantifierToPartialMatchMap, diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PartialMatch.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PartialMatch.java index 5875bbbfaf..bb99c64505 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PartialMatch.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PartialMatch.java @@ -261,7 +261,7 @@ public boolean compensationCanBeDeferred() { } @Nonnull - public static Collection matchesFromMap(@Nonnull IdentityBiMap partialMatchMap) { + public static Collection matchesFromMap(@Nonnull final IdentityBiMap partialMatchMap) { return partialMatchMap.values() .stream() .map(IdentityBiMap::unwrap) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PrimaryScanMatchCandidate.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PrimaryScanMatchCandidate.java index b2665aa87a..8606bdcf05 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PrimaryScanMatchCandidate.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/PrimaryScanMatchCandidate.java @@ -157,6 +157,16 @@ public boolean createsDuplicates() { return false; } + @Override + public int getColumnSize() { + return primaryKey.getColumnSize(); + } + + @Override + public boolean isUnique() { + return true; + } + @Nonnull @Override public RelationalExpression toEquivalentExpression(@Nonnull PartialMatch partialMatch, diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScalarTranslationVisitor.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScalarTranslationVisitor.java index cf1ef9b99b..0aa3962611 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScalarTranslationVisitor.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScalarTranslationVisitor.java @@ -44,7 +44,9 @@ import java.util.List; /** - * Visitor that translates a {@link KeyExpression} into a {@link Value}. + * Visitor that translates a {@link KeyExpression} into a {@link Value}, keeping a state of the currently processed + * input type while it is processing the {@link KeyExpression}. The main caveat of this visitor is that it is meant to only + * expand key expressions that are provably scalar. */ @SuppressWarnings("java:S5993") public class ScalarTranslationVisitor implements KeyExpressionVisitor { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScanWithFetchMatchCandidate.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScanWithFetchMatchCandidate.java index c147f65d30..364d0062e9 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScanWithFetchMatchCandidate.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ScanWithFetchMatchCandidate.java @@ -20,7 +20,6 @@ package com.apple.foundationdb.record.query.plan.cascades; -import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.record.query.plan.cascades.values.Value; import javax.annotation.Nonnull; @@ -31,9 +30,6 @@ */ public interface ScanWithFetchMatchCandidate extends WithPrimaryKeyMatchCandidate { - @Nonnull - Index getIndex(); - @Nonnull Optional pushValueThroughFetch(@Nonnull Value value, @Nonnull CorrelationIdentifier sourceAlias, diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ValueIndexScanMatchCandidate.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ValueIndexScanMatchCandidate.java index d48332c12e..d2ba42b2e4 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ValueIndexScanMatchCandidate.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/ValueIndexScanMatchCandidate.java @@ -136,10 +136,14 @@ public ValueIndexScanMatchCandidate(@Nonnull Index index, this.primaryKeyValuesSupplier = Suppliers.memoize(() -> MatchCandidate.computePrimaryKeyValuesMaybe(primaryKey, baseType)); } - @Nonnull @Override - public Index getIndex() { - return index; + public int getColumnSize() { + return index.getColumnSize(); + } + + @Override + public boolean isUnique() { + return index.isUnique(); } @Nonnull @@ -328,20 +332,20 @@ private static ScanComparisons toScanComparisons(@Nonnull final List MatchCandidate.computePrimaryKeyValuesMaybe(primaryKey, baseType)); } - @Nonnull @Override - public Index getIndex() { - return index; + public int getColumnSize() { + return index.getColumnSize(); + } + + @Override + public boolean isUnique() { + return index.isUnique(); } @Nonnull @@ -452,20 +456,21 @@ private static ScanComparisons toScanComparisons(@Nonnull final List true, fieldData.getOrdinalPath()); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/WithPrimaryKeyMatchCandidate.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/WithPrimaryKeyMatchCandidate.java index 8dd3372dbc..d2419376d3 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/WithPrimaryKeyMatchCandidate.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/WithPrimaryKeyMatchCandidate.java @@ -20,14 +20,11 @@ package com.apple.foundationdb.record.query.plan.cascades; -import com.apple.foundationdb.record.metadata.RecordType; import com.apple.foundationdb.record.query.plan.cascades.values.Value; -import com.google.common.collect.ImmutableSet; import javax.annotation.Nonnull; import java.util.List; import java.util.Optional; -import java.util.Set; /** * Interface to represent a candidate that uses a primary key to identify a record. @@ -36,16 +33,6 @@ public interface WithPrimaryKeyMatchCandidate extends MatchCandidate { @Nonnull Optional> getPrimaryKeyValuesMaybe(); - @Nonnull - List getQueriedRecordTypes(); - - @Nonnull - default Set getQueriedRecordTypeNames() { - return getQueriedRecordTypes().stream() - .map(RecordType::getName) - .collect(ImmutableSet.toImmutableSet()); - } - @Nonnull static Optional> commonPrimaryKeyValuesMaybe(@Nonnull Iterable matchCandidates) { List common = null; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/GroupByExpression.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/GroupByExpression.java index f71a8586ea..6db79ecebb 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/GroupByExpression.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/GroupByExpression.java @@ -23,8 +23,14 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.Column; +import com.apple.foundationdb.record.query.plan.cascades.ComparisonRange; +import com.apple.foundationdb.record.query.plan.cascades.Compensation; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; +import com.apple.foundationdb.record.query.plan.cascades.IdentityBiMap; import com.apple.foundationdb.record.query.plan.cascades.OrderingPart; +import com.apple.foundationdb.record.query.plan.cascades.MatchInfo; +import com.apple.foundationdb.record.query.plan.cascades.PartialMatch; +import com.apple.foundationdb.record.query.plan.cascades.PredicateMap; import com.apple.foundationdb.record.query.plan.cascades.Quantifier; import com.apple.foundationdb.record.query.plan.cascades.RequestedOrdering; import com.apple.foundationdb.record.query.plan.cascades.TranslationMap; @@ -46,6 +52,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -249,6 +256,42 @@ public Value getRuntimeValue() { return computeRuntimeResultSupplier.get(); } + @Override + public Compensation compensate(@Nonnull final PartialMatch partialMatch, + @Nonnull final Map boundParameterPrefixMap) { + // subsumedBy() is based on equality, thus we return empty here as + // if there is a match, it's exact + return Compensation.noCompensation(); + } + + @Nonnull + @Override + public Iterable subsumedBy(@Nonnull final RelationalExpression candidateExpression, + @Nonnull final AliasMap aliasMap, + @Nonnull final IdentityBiMap partialMatchMap) { + + // the candidate must be a GROUP-BY expression. + if (candidateExpression.getClass() != this.getClass()) { + return ImmutableList.of(); + } + + final var otherGroupByExpression = (GroupByExpression)candidateExpression; + + // the grouping values are encoded directly in the underlying SELECT-WHERE, reaching this point means that the + // grouping values had exact match so we don't need to check them. + + + // check that aggregate value is the same. + final var otherAggregateValue = otherGroupByExpression.getAggregateValue(); + if (aggregateValue.subsumedBy(otherAggregateValue, aliasMap)) { + // placeholder for information needed for later compensation. + return MatchInfo.tryMerge(partialMatchMap, ImmutableMap.of(), PredicateMap.empty(), Optional.empty()) + .map(ImmutableList::of) + .orElse(ImmutableList.of()); + } + return ImmutableList.of(); + } + @Nonnull private Value computeResultValue() { final var aggregateColumn = Column.of(Type.Record.Field.of(getAggregateValue().getResultType(), Optional.of(getAggregateValueAlias().getId())), getAggregateValue()); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/SelectExpression.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/SelectExpression.java index a0d17a2e57..589e3e1c8a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/SelectExpression.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/expressions/SelectExpression.java @@ -217,7 +217,7 @@ public PartiallyOrderedSet getCorrelationOrder() { private PartiallyOrderedSet computeCorrelationOrder() { return RelationalExpressionWithChildren.ChildrenAsSet.super.getCorrelationOrder(); } - + @Nonnull @Override @SuppressWarnings("PMD.CompareObjectsWithEquals") @@ -360,16 +360,16 @@ public Iterable subsumedBy(@Nonnull final RelationalExpression candid // FROM R r, S s // WHERE r.a < 5 AND s.b = 10 AND r.x = s.y // - // The predicate 'r.x = r.y' can be used as a predicate for matching an index on R(x, a) or for + // The predicate 'r.x = s.y' can be used as a predicate for matching an index on R(x, a) or for // matching an index on S(b, y). In fact the predicate needs to be shared in some way such that the planner // can later on make the right decision based on cost, etc. // // The way this is implemented is to create two matches one where the predicate is repossessed to the match // at hand. When we match R(x, a) we repossess r.x = r.y to be subsumed by r.x = ? on - // the candidate side. Vica versa, when we match S(b, y) we repossess s.y = r.x to be subsumed by s.y = ? on the + // the candidate side. Vice versa, when we match S(b, y) we repossess s.y = r.x to be subsumed by s.y = ? on the // candidate side. // - // Using this approach we create a problem that these two matches cannot coexist in a way that they cannot + // Using this approach we create a problem that these two matches can coexist in a way that they cannot // be realized, that is planned together at all as both matches provide the other's placeholder value. In fact, // we have forced the match to (if it were to be planned) become the inner of a join. It would be beneficial, // however, to also create a version of the match that does not consider the join predicate at all. diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/RelationalExpressionMatchers.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/RelationalExpressionMatchers.java index d8ae8e4948..2b59a37455 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/RelationalExpressionMatchers.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/RelationalExpressionMatchers.java @@ -36,6 +36,7 @@ import com.apple.foundationdb.record.query.plan.cascades.expressions.RelationalExpressionWithPredicates; import com.apple.foundationdb.record.query.plan.cascades.expressions.SelectExpression; import com.apple.foundationdb.record.query.plan.cascades.predicates.QueryPredicate; +import com.apple.foundationdb.record.query.plan.cascades.values.RecordConstructorValue; import com.apple.foundationdb.record.query.plan.plans.RecordQueryPlan; import com.google.common.collect.ImmutableList; @@ -234,4 +235,19 @@ public static BindingMatcher explodeExpression() { public static BindingMatcher groupByExpression(@Nonnull final CollectionMatcher downstream) { return ofTypeOwning(GroupByExpression.class, downstream); } + + @Nonnull + public static BindingMatcher groupByExpression(@Nonnull final BindingMatcher downstreamAggregation, + @Nonnull final CollectionMatcher downstreamQuantifiers) { + return typedWithDownstream(GroupByExpression.class, + Extractor.identity(), + AllOfMatcher.matchingAllOf(GroupByExpression.class, + ImmutableList.of( + typedWithDownstream(GroupByExpression.class, + Extractor.of(GroupByExpression::getAggregateValue, name -> "aggregation(" + name + ")"), + downstreamAggregation), + typedWithDownstream(GroupByExpression.class, + Extractor.of(RelationalExpression::getQuantifiers, name -> "quantifiers(" + name + ")"), + downstreamQuantifiers)))); + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/ValueMatchers.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/ValueMatchers.java index 329d41b556..15ba79fa57 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/ValueMatchers.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/matching/structure/ValueMatchers.java @@ -28,12 +28,14 @@ import com.apple.foundationdb.record.query.plan.cascades.values.NumericAggregationValue; import com.apple.foundationdb.record.query.plan.cascades.values.QuantifiedObjectValue; import com.apple.foundationdb.record.query.plan.cascades.values.RecordConstructorValue; +import com.apple.foundationdb.record.query.plan.cascades.values.StreamableAggregateValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; import com.google.common.collect.ImmutableList; import javax.annotation.Nonnull; import java.util.Arrays; +import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.AllOfMatcher.matchingAllOf; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ListMatcher.exactly; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.TypedMatcher.typed; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.TypedMatcherWithExtractAndDownstream.typedWithDownstream; @@ -81,45 +83,41 @@ public static BindingMatcher fieldValueWithFieldNa return typedWithDownstream(FieldValue.class, Extractor.identity(), - AllOfMatcher.matchingAllOf(FieldValue.class, ImmutableList.of(downstreamValueMatcher, downstreamFieldPathMatcher))); + matchingAllOf(FieldValue.class, ImmutableList.of(downstreamValueMatcher, downstreamFieldPathMatcher))); } + @SuppressWarnings("UnstableApiUsage") @Nonnull public static BindingMatcher fieldValueWithFieldPath(@Nonnull final BindingMatcher downstreamValue, - @Nonnull final CollectionMatcher downstreamFieldPath) { + @Nonnull final CollectionMatcher downstreamFieldPathOrdinals, + @Nonnull final CollectionMatcher downstreamFieldPathTypes) { final TypedMatcherWithExtractAndDownstream downstreamValueMatcher = typedWithDownstream(FieldValue.class, Extractor.of(FieldValue::getChild, name -> "child(" + name + ")"), downstreamValue); - final TypedMatcherWithExtractAndDownstream downstreamFieldPathMatcher = + final TypedMatcherWithExtractAndDownstream downstreamFieldPathOrdinalsMatcher = typedWithDownstream(FieldValue.class, - Extractor.of(FieldValue::getFields, name -> "fieldPath(" + name + ")"), - downstreamFieldPath); + Extractor.of(f -> f.getFieldPathOrdinals().asList(), name -> "fieldPathOrdinals(" + name + ")"), + downstreamFieldPathOrdinals); + final TypedMatcherWithExtractAndDownstream downstreamFieldPathTypesMatcher = + typedWithDownstream(FieldValue.class, + Extractor.of(FieldValue::getFieldPathTypes, name -> "fieldPathTypes(" + name + ")"), + downstreamFieldPathTypes); return typedWithDownstream(FieldValue.class, Extractor.identity(), - AllOfMatcher.matchingAllOf(FieldValue.class, ImmutableList.of(downstreamValueMatcher, downstreamFieldPathMatcher))); + matchingAllOf(FieldValue.class, ImmutableList.of(downstreamValueMatcher, downstreamFieldPathOrdinalsMatcher, downstreamFieldPathTypesMatcher))); } - @Nonnull - public static BindingMatcher numericAggregationValue(@Nonnull final String operatorName) { - return numericAggregationValue(anyValue(), operatorName); + public static BindingMatcher sumAggregationValue() { + return sumAggregationValue(anyValue()); } @Nonnull - public static BindingMatcher numericAggregationValue(@Nonnull final BindingMatcher downstreamValue, - @Nonnull final String operatorName) { - final TypedMatcherWithExtractAndDownstream downstreamValueMatcher = - typedWithDownstream(NumericAggregationValue.class, - Extractor.of(NumericAggregationValue::getChild, name -> "child(" + name + ")"), - downstreamValue); - final TypedMatcherWithExtractAndDownstream downstreamOperatorMatcher = - typedWithDownstream(NumericAggregationValue.class, - Extractor.of(NumericAggregationValue::getOperatorName, name -> "operator(" + name + ")"), - PrimitiveMatchers.equalsObject(operatorName)); - return typedWithDownstream(NumericAggregationValue.class, - Extractor.identity(), - AllOfMatcher.matchingAllOf(NumericAggregationValue.class, ImmutableList.of(downstreamValueMatcher, downstreamOperatorMatcher))); + public static BindingMatcher sumAggregationValue(@Nonnull final BindingMatcher downstream) { + return typedWithDownstream(NumericAggregationValue.Sum.class, + Extractor.of(NumericAggregationValue.Sum::getChild, name -> "child(" + name + ")"), + downstream); } @Nonnull @@ -149,4 +147,16 @@ public static BindingMatcher currentObjectValue() { Extractor.of(QuantifiedObjectValue::getAlias, name -> "alias(" + name + ")"), PrimitiveMatchers.equalsObject(Quantifier.CURRENT)); } + + @Nonnull + public static BindingMatcher streamableAggregateValue() { + return streamableAggregateValue(exactly(ImmutableList.of(anyValue()))); + } + + @Nonnull + public static BindingMatcher streamableAggregateValue(@Nonnull final CollectionMatcher downstreamValues) { + return typedWithDownstream(StreamableAggregateValue.class, + Extractor.of(StreamableAggregateValue::getChildren, name -> "children(" + name + ")"), + downstreamValues); + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/CardinalitiesProperty.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/CardinalitiesProperty.java index ee30d7d85e..f60cdc52d6 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/CardinalitiesProperty.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/CardinalitiesProperty.java @@ -43,6 +43,7 @@ import com.apple.foundationdb.record.query.plan.plans.InParameterSource; import com.apple.foundationdb.record.query.plan.plans.InValuesSource; import com.apple.foundationdb.record.query.plan.plans.QueryPlan; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryAggregateIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryComparatorPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryCoveringIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryExplodePlan; @@ -121,6 +122,31 @@ public Cardinalities visitRecordQueryInValuesJoinPlan(@Nonnull final RecordQuery Cardinality.ofCardinality(valuesSize).times(childCardinalities.getMaxCardinality())); } + @Nonnull + @Override + public Cardinalities visitRecordQueryAggregateIndexPlan(@Nonnull final RecordQueryAggregateIndexPlan aggregateIndexPlan) { + final var groupingValueMaybe = aggregateIndexPlan.getGroupingValueMaybe(); + if (groupingValueMaybe.isEmpty()) { + return new Cardinalities(Cardinality.ofCardinality(1L), Cardinality.ofCardinality(1L)); + } + final var groupingValue = groupingValueMaybe.get(); + final var indexScanPlan = aggregateIndexPlan.getIndexPlan(); + final var primaryMatchCandidate = indexScanPlan.getMatchCandidateMaybe(); + if (primaryMatchCandidate.isEmpty()) { + return Cardinalities.unknownCardinalities(); + } + final var ordering = primaryMatchCandidate.get() + .computeOrderingFromScanComparisons( + indexScanPlan.getComparisons(), + indexScanPlan.isReverse(), + false); + if (ordering.getEqualityBoundValues().contains(groupingValue)) { + return new Cardinalities(Cardinality.ofCardinality(0L), Cardinality.ofCardinality(1L)); + } else { + return Cardinalities.unknownCardinalities(); + } + } + @Nonnull @Override public Cardinalities visitRecordQueryCoveringIndexPlan(@Nonnull final RecordQueryCoveringIndexPlan element) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/DistinctRecordsProperty.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/DistinctRecordsProperty.java index 8e48d312e1..cde0bcfe5a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/DistinctRecordsProperty.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/DistinctRecordsProperty.java @@ -28,6 +28,7 @@ import com.apple.foundationdb.record.query.plan.cascades.Quantifier; import com.apple.foundationdb.record.query.plan.cascades.expressions.RelationalExpression; import com.apple.foundationdb.record.query.plan.cascades.values.QuantifiedObjectValue; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryAggregateIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryComparatorPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryCoveringIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryExplodePlan; @@ -115,6 +116,12 @@ public Boolean visitInComparandJoinPlan(@Nonnull final RecordQueryInComparandJoi return visitInJoinPlan(inComparandJoinPlan); } + @Nonnull + @Override + public Boolean visitAggregateIndexPlan(@Nonnull final RecordQueryAggregateIndexPlan aggregateIndexPlan) { + return true; + } + @Nonnull @Override public Boolean visitCoveringIndexPlan(@Nonnull final RecordQueryCoveringIndexPlan coveringIndexPlan) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/OrderingProperty.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/OrderingProperty.java index cccf912b08..eb71b8dc57 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/OrderingProperty.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/OrderingProperty.java @@ -36,6 +36,7 @@ import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; import com.apple.foundationdb.record.query.plan.cascades.values.ObjectValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryAggregateIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryComparatorPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryCoveringIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryExplodePlan; @@ -81,6 +82,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.List; +import java.util.Optional; import java.util.function.BinaryOperator; import java.util.stream.Stream; @@ -131,9 +133,9 @@ public Ordering visitPredicatesFilterPlan(@Nonnull final RecordQueryPredicatesFi } final var fieldValue = (FieldValue)valuePredicate.getValue(); - if (fieldValue.getFields() + if (fieldValue.getFieldPathNamesMaybe() .stream() - .anyMatch(field -> field.getFieldNameOptional().isEmpty())) { + .anyMatch(Optional::isEmpty)) { return Stream.of(); } @@ -176,6 +178,12 @@ public Ordering visitInComparandJoinPlan(@Nonnull final RecordQueryInComparandJo return visitInJoinPlan(inComparandJoinPlan); } + @Nonnull + @Override + public Ordering visitAggregateIndexPlan(@Nonnull final RecordQueryAggregateIndexPlan aggregateIndexPlan) { + return visit(aggregateIndexPlan.getIndexPlan()); + } + @Nonnull @Override public Ordering visitCoveringIndexPlan(@Nonnull final RecordQueryCoveringIndexPlan coveringIndexPlan) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/PrimaryKeyProperty.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/PrimaryKeyProperty.java index 8549383658..9f19167baf 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/PrimaryKeyProperty.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/PrimaryKeyProperty.java @@ -28,6 +28,7 @@ import com.apple.foundationdb.record.query.plan.cascades.ScalarTranslationVisitor; import com.apple.foundationdb.record.query.plan.cascades.expressions.RelationalExpression; import com.apple.foundationdb.record.query.plan.cascades.values.Value; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryAggregateIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryComparatorPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryCoveringIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryExplodePlan; @@ -121,6 +122,12 @@ public Optional> visitInComparandJoinPlan(@Nonnull final RecordQuery return visitInJoinPlan(inComparandJoinPlan); } + @Nonnull + @Override + public Optional> visitAggregateIndexPlan(@Nonnull final RecordQueryAggregateIndexPlan aggregateIndexPlan) { + return Optional.empty(); + } + @Nonnull @Override public Optional> visitCoveringIndexPlan(@Nonnull final RecordQueryCoveringIndexPlan coveringIndexPlan) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/RecordTypesProperty.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/RecordTypesProperty.java index 5a321156ae..769dd882e9 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/RecordTypesProperty.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/RecordTypesProperty.java @@ -25,9 +25,9 @@ import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; import com.apple.foundationdb.record.query.plan.cascades.ExpressionProperty; import com.apple.foundationdb.record.query.plan.cascades.ExpressionRef; +import com.apple.foundationdb.record.query.plan.cascades.MatchCandidate; import com.apple.foundationdb.record.query.plan.cascades.Quantifier; import com.apple.foundationdb.record.query.plan.cascades.Quantifiers.AliasResolver; -import com.apple.foundationdb.record.query.plan.cascades.WithPrimaryKeyMatchCandidate; import com.apple.foundationdb.record.query.plan.cascades.expressions.FullUnorderedScanExpression; import com.apple.foundationdb.record.query.plan.cascades.expressions.LogicalUnionExpression; import com.apple.foundationdb.record.query.plan.cascades.expressions.PrimaryScanExpression; @@ -77,7 +77,7 @@ public Set evaluateAtExpression(@Nonnull RelationalExpression expression return ((FullUnorderedScanExpression)expression).getRecordTypes(); } else if (expression instanceof RecordQueryIndexPlan) { return ((RecordQueryIndexPlan)expression).getMatchCandidateMaybe() - .map(WithPrimaryKeyMatchCandidate::getQueriedRecordTypeNames) + .map(MatchCandidate::getQueriedRecordTypeNames) .orElse(ImmutableSet.of()); } else if (expression instanceof TypeFilterExpression) { return Sets.filter(childResults.get(0), ((TypeFilterExpression)expression).getRecordTypes()::contains); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/StoredRecordProperty.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/StoredRecordProperty.java index d6bd83c3af..b8adf8e8d3 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/StoredRecordProperty.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/properties/StoredRecordProperty.java @@ -26,6 +26,7 @@ import com.apple.foundationdb.record.query.plan.cascades.PlanProperty; import com.apple.foundationdb.record.query.plan.cascades.Quantifier; import com.apple.foundationdb.record.query.plan.cascades.expressions.RelationalExpression; +import com.apple.foundationdb.record.query.plan.plans.RecordQueryAggregateIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryComparatorPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryCoveringIndexPlan; import com.apple.foundationdb.record.query.plan.plans.RecordQueryExplodePlan; @@ -115,6 +116,12 @@ public Boolean visitInComparandJoinPlan(@Nonnull final RecordQueryInComparandJoi return visitInJoinPlan(inComparandJoinPlan); } + @Nonnull + @Override + public Boolean visitAggregateIndexPlan(@Nonnull final RecordQueryAggregateIndexPlan element) { + return false; + } + @Nonnull @Override public Boolean visitCoveringIndexPlan(@Nonnull final RecordQueryCoveringIndexPlan element) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AbstractDataAccessRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AbstractDataAccessRule.java index bff36e23ab..44cc98963a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AbstractDataAccessRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AbstractDataAccessRule.java @@ -256,7 +256,6 @@ private static List maximumCoverageMatches(@Nonnul final var partialMatchesWithCompensation = matches .stream() - .filter(partialMatch -> partialMatch.getMatchCandidate() instanceof WithPrimaryKeyMatchCandidate) .filter(partialMatch -> !satisfiedOrderings(partialMatch, interestedOrderings).isEmpty()) .map(partialMatch -> new PartialMatchWithCompensation(partialMatch, partialMatch.compensate())) .filter(partialMatchWithCompensation -> !partialMatchWithCompensation.getCompensation().isImpossible()) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AdjustMatchRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AdjustMatchRule.java index 9530f48b65..d48dfc12a8 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AdjustMatchRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/AdjustMatchRule.java @@ -43,11 +43,16 @@ /** * A rule that attempts to improve an existing {@link PartialMatch} by absorbing an expression on the * candidate side. - * + * This rule delegates the actual adjustment legwork to the {@link RelationalExpression} on the candidate-side + * that is: + *
    + *
  • referencing the {@link MatchCandidate}'s traversal w.r.t. the (partially) matched query.
  • + *
  • does not have a corresponding match on the query side.
  • + *
+ * For more information, see {@link RelationalExpression#adjustMatch(PartialMatch)}. * Currently the only such expression that can be absorbed is * {@link com.apple.foundationdb.record.query.plan.cascades.expressions.MatchableSortExpression}. - * TODO Maybe that expression should just be a generic property-defining expression or properties should be kept - * on quantifiers. + * TODO Maybe that expression should just be a generic property-defining expression or properties should be kept on quantifiers. * It is special in a way that there is no corresponding expression on the query side that is subsumed by that * expression. Absorbing such a candidate-side-only expression into the match allows us to fine-tune interesting * provided properties guaranteed by the candidate side. diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/ImplementStreamingAggregationRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/ImplementStreamingAggregationRule.java index b8cada725a..7d38008012 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/ImplementStreamingAggregationRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/ImplementStreamingAggregationRule.java @@ -20,6 +20,7 @@ package com.apple.foundationdb.record.query.plan.cascades.rules; +import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.CascadesRule; import com.apple.foundationdb.record.query.plan.cascades.CascadesRuleCall; @@ -42,12 +43,17 @@ import javax.annotation.Nonnull; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.AnyMatcher.any; +import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.MultiMatcher.all; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.QuantifierMatchers.forEachQuantifierOverRef; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.RelationalExpressionMatchers.groupByExpression; +import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers.recordConstructorValue; +import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers.streamableAggregateValue; /** * Rule for implementing logical {@code GROUP BY} into a physical streaming aggregate operator {@link RecordQueryStreamingAggregationPlan}. */ +@API(API.Status.EXPERIMENTAL) +@SuppressWarnings("PMD.TooManyStaticImports") public class ImplementStreamingAggregationRule extends CascadesRule { @Nonnull @@ -56,7 +62,7 @@ public class ImplementStreamingAggregationRule extends CascadesRule innerQuantifierMatcher = forEachQuantifierOverRef(lowerRefMatcher); @Nonnull private static final BindingMatcher root = - groupByExpression(any(innerQuantifierMatcher)); + groupByExpression(recordConstructorValue(all(streamableAggregateValue())), any(innerQuantifierMatcher)); public ImplementStreamingAggregationRule() { super(root, ImmutableSet.of(RequestedOrderingConstraint.REQUESTED_ORDERING)); @@ -71,14 +77,16 @@ public void onMatch(@Nonnull final CascadesRuleCall call) { final var innerQuantifier = Iterables.getOnlyElement(groupByExpression.getQuantifiers()); final var groupingValue = groupByExpression.getGroupingValue(); + final var currentGroupingValue = groupingValue == null ? null : groupingValue.rebase(AliasMap.of(innerQuantifier.getAlias(), Quantifier.CURRENT)); - final var requiredOrderingKeyValues = - currentGroupingValue == null || currentGroupingValue.isConstant() - ? null - : Values.primitiveAccessorsForType(currentGroupingValue.getResultType(), () -> currentGroupingValue, correlatedTo) - .stream() - .collect(ImmutableSet.toImmutableSet()); + // TODO: isConstant is not implemented correctly. + // for the following FV(col1, QOV( --> RCV(FV(col1(Literal(42))...) ) it is returning false while it should return true. + final var requiredOrderingKeyValues = currentGroupingValue == null + ? null + : Values.primitiveAccessorsForType(currentGroupingValue.getResultType(), () -> currentGroupingValue, correlatedTo) + .stream() + .collect(ImmutableSet.toImmutableSet()); final var innerReference = innerQuantifier.getRangesOver(); final var planPartitions = PlanPartition.rollUpTo(innerReference.getPlanPartitions(), OrderingProperty.ORDERING); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/RemoveSortRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/RemoveSortRule.java index a0a6ddaf72..16006aaab1 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/RemoveSortRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/rules/RemoveSortRule.java @@ -151,8 +151,7 @@ public static boolean strictlyOrderedIfUnique(@Nonnull RecordQueryPlan orderedPl final var matchCandidateOptional = indexPlan.getMatchCandidateMaybe(); if (matchCandidateOptional.isPresent()) { final var matchCandidate = matchCandidateOptional.get(); - final var index = matchCandidate.getIndex(); - return index.isUnique() && nkeys >= index.getColumnSize(); + return matchCandidate.isUnique() && nkeys >= matchCandidate.getColumnSize(); } } return false; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/typing/Type.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/typing/Type.java index 09095ee987..5a9dbf0aa4 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/typing/Type.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/typing/Type.java @@ -48,6 +48,7 @@ import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * Provides type information about the output of an expression such as {@link Value} in a QGM. @@ -322,7 +323,7 @@ private static Array fromProtoTypeToArray(@Nullable Descriptors.GenericDescripto return new Array(isNullable, enumType); } else { if (isNullable) { - Descriptors.Descriptor wrappedDescriptor = ((Descriptors.Descriptor)descriptor).findFieldByName(NullableArrayTypeUtils.getRepeatedFieldName()).getMessageType(); + Descriptors.Descriptor wrappedDescriptor = ((Descriptors.Descriptor)Objects.requireNonNull(descriptor)).findFieldByName(NullableArrayTypeUtils.getRepeatedFieldName()).getMessageType(); Objects.requireNonNull(wrappedDescriptor); return new Array(isNullable, fromProtoType(wrappedDescriptor, Descriptors.FieldDescriptor.Type.MESSAGE, FieldDescriptorProto.Label.LABEL_OPTIONAL, true)); } else { @@ -791,6 +792,12 @@ class Record implements Type { @Nonnull private final Supplier> fieldNameFieldMapSupplier; + @Nonnull + private final Supplier> fieldNameToOrdinalSupplier; + + @Nonnull + private final Supplier> fieldIndexToOrdinalSupplier; + /** * function that returns a list of {@link Field} types. */ @@ -827,6 +834,8 @@ protected Record(@Nullable final String name, final boolean isNullable, @Nullabl this.isNullable = isNullable; this.fields = fields == null ? null : normalizeFields(fields); this.fieldNameFieldMapSupplier = Suppliers.memoize(this::computeFieldNameFieldMap); + this.fieldNameToOrdinalSupplier = Suppliers.memoize(this::computeFieldNameToOrdinalMap); + this.fieldIndexToOrdinalSupplier = Suppliers.memoize(this::computeFieldIndexToOrdinal); this.elementTypesSupplier = Suppliers.memoize(this::computeElementTypes); } @@ -860,6 +869,16 @@ public List getFields() { return Objects.requireNonNull(fields); } + @Nonnull + public Map getFieldNameToOrdinalMap() { + return fieldNameToOrdinalSupplier.get(); + } + + @Nonnull + public Map getFieldIndexToOrdinalMap() { + return fieldIndexToOrdinalSupplier.get(); + } + /** * Returns the list of {@link Field} {@link Type}s. * @return the list of {@link Field} {@link Type}s. @@ -900,6 +919,32 @@ private Map computeFieldNameFieldMap() { .collect(ImmutableMap.toImmutableMap(field -> field.getFieldNameOptional().get(), Function.identity())); } + /** + * Compute a mapping from {@link Field} to their ordinal positions in their {@link Type}. + * @return a mapping from {@link Field} to their ordinal positions in their {@link Type}. + */ + @Nonnull + @SuppressWarnings("OptionalGetWithoutIsPresent") + private Map computeFieldNameToOrdinalMap() { + return IntStream + .range(0, Objects.requireNonNull(fields).size()) + .boxed() + .collect(ImmutableMap.toImmutableMap(id -> fields.get(id).getFieldNameOptional().get(), Function.identity())); + } + + /** + * Compute a mapping from {@link Field} to their ordinal positions in their {@link Type}. + * @return a mapping from {@link Field} to their ordinal positions in their {@link Type}. + */ + @Nonnull + @SuppressWarnings("OptionalGetWithoutIsPresent") + private Map computeFieldIndexToOrdinal() { + return IntStream + .range(0, Objects.requireNonNull(fields).size()) + .boxed() + .collect(ImmutableMap.toImmutableMap(id -> fields.get(id).getFieldIndexOptional().get(), Function.identity())); + } + /** * Checks whether the {@link Record} type instance is erased or not. * @return true if the {@link Record} type is erased, other false. @@ -1263,6 +1308,7 @@ public static Field of(@Nonnull final Type fieldType, @Nonnull final Optional newChildren) { Iterables.get(newChildren, 1)); } - /** * The {@code and} function. */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java index 1f1580e41e..54d00ee2e8 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/CountValue.java @@ -25,6 +25,7 @@ import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.ObjectPlanHash; import com.apple.foundationdb.record.PlanHashable; +import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.BuiltInFunction; @@ -50,18 +51,42 @@ * A counting aggregate value. */ @API(API.Status.EXPERIMENTAL) -public class CountValue implements Value, AggregateValue { +public class CountValue implements Value, AggregateValue, StreamableAggregateValue, IndexableAggregateValue { private static final ObjectPlanHash BASE_HASH = new ObjectPlanHash("Count-Value"); @Nonnull - private final PhysicalOperator operator; + protected final PhysicalOperator operator; @Nullable private final Value child; + @Nonnull + private final String indexName; + + public CountValue(@Nonnull final Value child) { + this(isCountStar(child), child); + } + + public CountValue(boolean isCountStar, @Nonnull final Value child) { + this(isCountStar ? PhysicalOperator.COUNT_STAR : PhysicalOperator.COUNT, child); + } + + public CountValue(@Nonnull PhysicalOperator operator, @Nullable Value child) { + this(operator, child, operator == PhysicalOperator.COUNT ? IndexTypes.COUNT_NOT_NULL : IndexTypes.COUNT); + } + public CountValue(@Nonnull PhysicalOperator operator, - @Nullable Value child) { + @Nullable Value child, + @Nonnull String indexName) { this.operator = operator; this.child = child; + this.indexName = indexName; + } + + private static boolean isCountStar(@Nonnull Typed valueType) { + // todo: we should dispatch on the right function depending on whether the child + // value type is nullable or not. The '*' in count(*) must be guaranteed to be not-null + // during plan generation. + return !valueType.getResultType().isPrimitive(); } @Nullable @@ -146,6 +171,12 @@ public boolean equals(final Object other) { return semanticEquals(other, AliasMap.identitiesFor(getCorrelatedTo())); } + @Nonnull + @Override + public String getIndexName() { + return indexName; + } + /** * The {@code count(x)} function. */ @@ -162,26 +193,7 @@ private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignore @Nonnull BuiltInFunction builtInFunction, @Nonnull final List arguments) { final Typed arg0 = arguments.get(0); - return new CountValue(PhysicalOperator.COUNT, (Value)arg0); - } - } - - /** - * The {@code count(*)} function. - */ - @AutoService(BuiltInFunction.class) - @SuppressWarnings("PMD.UnusedFormalParameter") - public static class CountStarFn extends BuiltInFunction { - public CountStarFn() { - super("count", - ImmutableList.of(), CountStarFn::encapsulate); - } - - @Nonnull - private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, - @Nonnull BuiltInFunction builtInFunction, - @Nonnull final List arguments) { - return new CountValue(PhysicalOperator.COUNT_STAR, null); + return new CountValue((Value)arg0); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/FieldValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/FieldValue.java index e0a3dbd5ec..867b63063e 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/FieldValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/FieldValue.java @@ -25,6 +25,7 @@ import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.ObjectPlanHash; import com.apple.foundationdb.record.PlanHashable; +import com.apple.foundationdb.record.RecordCoreException; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.Formatter; @@ -32,9 +33,14 @@ import com.apple.foundationdb.record.query.plan.cascades.SemanticException; import com.apple.foundationdb.record.query.plan.cascades.typing.Type; import com.apple.foundationdb.record.query.plan.cascades.typing.Type.Record.Field; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.google.common.base.Verify; import com.google.common.collect.Comparators; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; +import com.google.common.primitives.ImmutableIntArray; import com.google.protobuf.Message; import javax.annotation.Nonnull; @@ -48,6 +54,7 @@ /** * A value representing the contents of a (non-repeated, arbitrarily-nested) field of a quantifier. */ +@SuppressWarnings("UnstableApiUsage") // caused by usage of Guava's ImmutableIntArray. @API(API.Status.EXPERIMENTAL) public class FieldValue implements ValueWithChild { private static final ObjectPlanHash BASE_HASH = new ObjectPlanHash("Field-Value"); @@ -57,9 +64,17 @@ public class FieldValue implements ValueWithChild { @Nonnull private final FieldPath fieldPath; + @Nonnull + private final Supplier> fieldNamesSupplier; + private FieldValue(@Nonnull Value childValue, @Nonnull FieldPath fieldPath) { this.childValue = childValue; this.fieldPath = fieldPath; + fieldNamesSupplier = Suppliers.memoize(() -> + fieldPath.getFieldNamesMaybe() + .stream() + .map(maybe -> maybe.orElseThrow(() -> new RecordCoreException("field name should have been set"))) + .collect(Collectors.toList())); } @Nonnull @@ -68,31 +83,39 @@ public FieldPath getFieldPath() { } @Nonnull - public List getFields() { - return fieldPath.getFields(); + public List getFieldPathNames() { + return fieldNamesSupplier.get(); } @Nonnull - public List getFieldPathNames() { - return getFields().stream() - .map(Field::getFieldName) - .collect(ImmutableList.toImmutableList()); + public List getFieldPathTypes() { + return fieldPath.getFieldTypes(); } @Nonnull - public List getFieldPrefix() { + public ImmutableIntArray getFieldPathOrdinals() { + return fieldPath.fieldOrdinals; + } + + @Nonnull + public List> getFieldPathNamesMaybe() { + return fieldPath.getFieldNamesMaybe(); + } + + @Nonnull + public FieldPath getFieldPrefix() { return fieldPath.getFieldPrefix(); } @Nonnull - public Field getLastField() { - return fieldPath.getLastField(); + public Optional getLastFieldName() { + return fieldPath.getLastFieldName(); } @Nonnull @Override public Type getResultType() { - return getLastField().getFieldType(); + return fieldPath.getLastFieldType(); } @Nonnull @@ -113,7 +136,7 @@ public Object eval(@Nonnull final FDBRecordStoreBase stor if (!(childResult instanceof Message)) { return null; } - final var fieldValue = MessageValue.getFieldValueForFields((Message)childResult, getFields()); + final var fieldValue = MessageValue.getFieldValueForFieldOrdinals((Message)childResult, fieldPath.getFieldOrdinals()); // // If the last step in the field path is an array that is also nullable, then we need to unwrap the value // wrapper. @@ -128,8 +151,22 @@ public boolean equalsWithoutChildren(@Nonnull final Value other, @Nonnull final } final var that = (FieldValue)other; - return childValue.semanticEquals(that.childValue, equivalenceMap) && - fieldPath.equals(that.fieldPath); + return fieldPath.equals(that.fieldPath) && + childValue.semanticEquals(that.childValue, equivalenceMap); + } + + @Override + public boolean subsumedBy(@Nullable final Value other, @Nonnull final AliasMap equivalenceMap) { + if (other == null) { + return false; + } + + if (!(other instanceof FieldValue)) { + return false; + } + final var otherFieldValue = (FieldValue)other; + return fieldPath.getFieldOrdinals().equals(otherFieldValue.getFieldPath().getFieldOrdinals()) && + childValue.subsumedBy(otherFieldValue.childValue, equivalenceMap); } @Override @@ -139,7 +176,7 @@ public int hashCodeWithoutChildren() { @Override public int planHash(@Nonnull final PlanHashKind hashKind) { - return PlanHashable.objectsPlanHash(hashKind, BASE_HASH, getFields().stream().map(Field::getFieldName).collect(ImmutableList.toImmutableList())); + return PlanHashable.objectsPlanHash(hashKind, BASE_HASH, getFieldPathNames()); } @Override @@ -173,6 +210,7 @@ public boolean equals(final Object other) { @Nonnull private static FieldPath resolveFieldPath(@Nonnull final Type inputType, @Nonnull final List accessors) { final var accessorPathBuilder = ImmutableList.builder(); + final var fieldOrdinals = ImmutableIntArray.builder(); var currentType = inputType; for (final var accessor : accessors) { final var fieldName = accessor.getFieldName(); @@ -184,62 +222,68 @@ private static FieldPath resolveFieldPath(@Nonnull final Type inputType, @Nonnul if (fieldName != null) { SemanticException.check(fieldNameFieldMap.containsKey(fieldName), SemanticException.ErrorCode.RECORD_DOES_NOT_CONTAIN_FIELD); field = fieldNameFieldMap.get(fieldName); + final var fieldOrdinalsMap = Objects.requireNonNull(recordType.getFieldNameToOrdinalMap()); + SemanticException.check(fieldOrdinalsMap.containsKey(fieldName), SemanticException.ErrorCode.RECORD_DOES_NOT_CONTAIN_FIELD); + fieldOrdinals.add(fieldOrdinalsMap.get(fieldName)); } else { // field is not accessed by field but by ordinal number Verify.verify(accessor.getOrdinalFieldNumber() >= 0); field = recordType.getFields().get(accessor.getOrdinalFieldNumber()); + fieldOrdinals.add(accessor.getOrdinalFieldNumber()); } accessorPathBuilder.add(field); currentType = field.getFieldType(); } - return new FieldPath(accessorPathBuilder.build()); + return new FieldPath(accessorPathBuilder.build(), fieldOrdinals.build()); } @Nonnull public static FieldValue ofFieldName(@Nonnull Value childValue, @Nonnull final String fieldName) { - return new FieldValue(childValue, resolveFieldPath(childValue.getResultType(), ImmutableList.of(new Accessor(fieldName, -1)))); + final var resolved = resolveFieldPath(childValue.getResultType(), ImmutableList.of(new Accessor(fieldName, -1))); + return new FieldValue(childValue, resolved); } public static FieldValue ofFieldNames(@Nonnull Value childValue, @Nonnull final List fieldNames) { - return new FieldValue(childValue, resolveFieldPath(childValue.getResultType(), fieldNames.stream().map(fieldName -> new Accessor(fieldName, -1)).collect(ImmutableList.toImmutableList()))); - } - - public static FieldValue ofAccessors(@Nonnull Value childValue, @Nonnull final List accessors) { - return new FieldValue(childValue, resolveFieldPath(childValue.getResultType(), accessors)); + final var resolved = resolveFieldPath(childValue.getResultType(), fieldNames.stream().map(fieldName -> new Accessor(fieldName, -1)).collect(ImmutableList.toImmutableList())); + return new FieldValue(childValue, resolved); } - public static FieldValue ofFields(@Nonnull Value childValue, @Nonnull final List fields) { - return new FieldValue(childValue, new FieldPath(fields)); + public static FieldValue ofFields(@Nonnull Value childValue, @Nonnull final FieldPath fieldPath) { + return new FieldValue(childValue, fieldPath); } - public static FieldValue ofFieldsAndFuseIfPossible(@Nonnull Value childValue, @Nonnull final List fields) { + public static FieldValue ofFieldsAndFuseIfPossible(@Nonnull Value childValue, @Nonnull final FieldPath fields) { if (childValue instanceof FieldValue) { final var childFieldValue = (FieldValue)childValue; - return FieldValue.ofFields(childFieldValue.getChild(), - ImmutableList.builder().addAll(childFieldValue.getFields()).addAll(fields).build()); + return FieldValue.ofFields(childFieldValue.getChild(), childFieldValue.fieldPath.withSuffix(fields)); } return FieldValue.ofFields(childValue, fields); } @Nonnull public static FieldValue ofOrdinalNumber(@Nonnull Value childValue, final int ordinalNumber) { - return new FieldValue(childValue, resolveFieldPath(childValue.getResultType(), ImmutableList.of(new Accessor(null, ordinalNumber)))); + final var resolved = resolveFieldPath(childValue.getResultType(), ImmutableList.of(new Accessor(null, ordinalNumber))); + return new FieldValue(childValue, resolved); } + @SuppressWarnings("PMD.CompareObjectsWithEquals") @Nonnull - public static Optional> stripFieldPrefixMaybe(@Nonnull List fieldPath, - @Nonnull List potentialPrefixPath) { + public static Optional stripFieldPrefixMaybe(@Nonnull FieldPath fieldPath, + @Nonnull FieldPath potentialPrefixPath) { if (fieldPath.size() < potentialPrefixPath.size()) { return Optional.empty(); } + final var fieldPathOrdinals = fieldPath.getFieldOrdinals(); + final var fieldPathTypes = fieldPath.getFieldTypes(); + final var potentialPrefixFieldOrdinals = potentialPrefixPath.getFieldOrdinals(); + final var potentialPrefixFieldTypes = potentialPrefixPath.getFieldTypes(); for (int i = 0; i < potentialPrefixPath.size(); i++) { - if (!potentialPrefixPath.get(i).equals(fieldPath.get(i))) { + if (fieldPathOrdinals.get(i) != potentialPrefixFieldOrdinals.get(i) || !fieldPathTypes.get(i).equals(potentialPrefixFieldTypes.get(i))) { return Optional.empty(); } } - - return Optional.of(ImmutableList.copyOf(fieldPath.subList(potentialPrefixPath.size(), fieldPath.size()))); + return Optional.of(fieldPath.subList(potentialPrefixPath.size(), fieldPath.size())); } /** @@ -271,21 +315,42 @@ public int getOrdinalFieldNumber() { */ @SuppressWarnings("UnstableApiUsage") public static class FieldPath { - private static final FieldPath EMPTY = new FieldPath(ImmutableList.of()); + private static final FieldPath EMPTY = new FieldPath(ImmutableList.of(), ImmutableIntArray.of()); private static final Comparator COMPARATOR = - Comparator.comparing(FieldPath::getFields, Comparators.lexicographical(Comparator.naturalOrder())); + Comparator.comparing(f -> f.fieldOrdinals.asList(), Comparators.lexicographical(Comparator.naturalOrder())); @Nonnull - private final List fields; + private final List> fieldNames; - public FieldPath(@Nonnull final List fields) { - this.fields = ImmutableList.copyOf(fields); + @Nonnull + private final List fieldTypes; + + /** + * This encapsulates the ordinals of the field path encoded by this {@link FieldValue}. It serves two purposes: + *
    + *
  • checking whether a {@link FieldValue} subsumes another {@link FieldValue}.
  • + *
  • evaluating a {@link Message} to get the corresponding field value object.
  • + *
+ */ + @Nonnull + private final ImmutableIntArray fieldOrdinals; + + FieldPath(@Nonnull final List fields, @Nonnull final ImmutableIntArray fieldOrdinals) { + Preconditions.checkArgument(fieldOrdinals.length() == fields.size()); + Preconditions.checkArgument(fieldOrdinals.stream().allMatch(f -> f >= 0)); + this.fieldNames = fields.stream().map(Field::getFieldNameOptional).collect(Collectors.toList()); + this.fieldTypes = fields.stream().map(Field::getFieldType).collect(Collectors.toList()); + this.fieldOrdinals = fieldOrdinals; } - @Nonnull - public List getFields() { - return fields; + public FieldPath(@Nonnull final List> fieldNames, @Nonnull final List fieldTypes, @Nonnull final ImmutableIntArray fieldOrdinals) { + Preconditions.checkArgument(fieldNames.size() == fieldTypes.size()); + Preconditions.checkArgument(fieldTypes.size() == fieldOrdinals.length()); + Preconditions.checkArgument(fieldOrdinals.stream().allMatch(f -> f >= 0)); + this.fieldNames = ImmutableList.copyOf(fieldNames); + this.fieldTypes = ImmutableList.copyOf(fieldTypes); + this.fieldOrdinals = fieldOrdinals; // already immutable. } @Override @@ -296,56 +361,129 @@ public boolean equals(final Object o) { if (!(o instanceof FieldPath)) { return false; } - final FieldPath fieldPath = (FieldPath)o; - return getFields().equals(fieldPath.getFields()); + final FieldPath otherFieldPath = (FieldPath)o; + return fieldNames.equals(otherFieldPath.fieldNames) + && fieldTypes.equals(otherFieldPath.fieldTypes) + && fieldOrdinals.equals(otherFieldPath.getFieldOrdinals()); } @Override public int hashCode() { - return Objects.hash(getFields()); + return Objects.hash(fieldNames, fieldOrdinals, fieldTypes); } @Override @Nonnull public String toString() { - return fields.stream() - .map(field -> { - if (field.getFieldNameOptional().isPresent()) { - return "." + field.getFieldName(); - } else if (field.getFieldIndexOptional().isPresent()) { - return "#" + field.getFieldIndex(); - } - return "(null)"; - }) + return Streams.zip(fieldNames.stream(), fieldOrdinals.stream().boxed(), + (maybeFieldName, fieldOrdinal) -> maybeFieldName.map(s -> "." + s).orElseGet(() -> "#" + fieldOrdinal)) .collect(Collectors.joining()); } @Nonnull - public List getFieldPrefix() { - return fields.subList(0, getFields().size() - 1); + public List> getFieldNamesMaybe() { + return fieldNames; + } + + @Nonnull + public FieldPath getFieldPrefix() { + Preconditions.checkArgument(!isEmpty()); + return subList(0, size() - 1); } @Nonnull - public Field getLastField() { - return fields.get(getFields().size() - 1); + public Optional getLastFieldName() { + Preconditions.checkArgument(!isEmpty()); + return fieldNames.get(size() - 1); + } + + public int getLastFieldOrdinal() { + Preconditions.checkArgument(!isEmpty()); + return fieldOrdinals.get(size() - 1); + } + + @Nonnull + public Type getLastFieldType() { + Preconditions.checkArgument(!isEmpty()); + return fieldTypes.get(size() - 1); + } + + @Nonnull + public ImmutableIntArray getFieldOrdinals() { + return fieldOrdinals; + } + + public int size() { + return fieldTypes.size(); + } + + public boolean isEmpty() { + return fieldTypes.isEmpty(); + } + + @Nonnull + public FieldPath subList(int fromIndex, int toIndex) { + return new FieldPath(fieldNames.subList(fromIndex, toIndex), + fieldTypes.subList(fromIndex, toIndex), + fieldOrdinals.subArray(fromIndex, toIndex)); + } + + @Nonnull + public FieldPath skip(int count) { + Preconditions.checkArgument(count >= 0); + Preconditions.checkArgument(count <= size()); + + if (count == 0) { + return this; + } else if (count == size()) { + return empty(); + } + + return subList(count, size()); + } + + @Nonnull + public List getFieldTypes() { + return fieldTypes; } public boolean isPrefixOf(@Nonnull final FieldPath otherFieldPath) { - final var otherFields = otherFieldPath.getFields(); - for (int i = 0; i < fields.size(); i++) { - final Field otherField = otherFields.get(i); - if (!fields.get(i).equals(otherField)) { + if (otherFieldPath.size() < size()) { + return false; + } + for (int i = 0; i < size(); i++) { + if (otherFieldPath.fieldOrdinals.get(i) != fieldOrdinals.get(i) || !otherFieldPath.fieldTypes.get(i).equals(fieldTypes.get(i))) { return false; } } return true; } + @Nonnull + public FieldPath withSuffix(@Nonnull final FieldPath suffix) { + if (suffix.isEmpty() && this.isEmpty()) { + return empty(); + } else if (suffix.isEmpty()) { + return this; + } else if (this.isEmpty()) { + return suffix; + } + return new FieldPath(ImmutableList.>builder().addAll(fieldNames).addAll(suffix.fieldNames).build(), + ImmutableList.builder().addAll(fieldTypes).addAll(suffix.fieldTypes).build(), + ImmutableIntArray.builder().addAll(fieldOrdinals).addAll(suffix.fieldOrdinals).build()); + } + @Nonnull public static FieldPath empty() { return EMPTY; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") + @Nonnull + public static FieldPath flat(@Nonnull final Optional fieldName, @Nonnull final Type fieldType, @Nonnull final Integer fieldOrdinal) { + return new FieldPath(ImmutableList.of(fieldName), ImmutableList.of(fieldType), ImmutableIntArray.of(fieldOrdinal)); + } + @Nonnull public static Comparator comparator() { return COMPARATOR; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/IndexOnlyAggregateValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/IndexOnlyAggregateValue.java new file mode 100644 index 0000000000..4bb506f838 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/IndexOnlyAggregateValue.java @@ -0,0 +1,230 @@ +/* + * IndexOnlyAggregateValue.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.query.plan.cascades.values; + +import com.apple.foundationdb.annotation.API; +import com.apple.foundationdb.annotation.SpotBugsSuppressWarnings; +import com.apple.foundationdb.record.EvaluationContext; +import com.apple.foundationdb.record.ObjectPlanHash; +import com.apple.foundationdb.record.PlanHashable; +import com.apple.foundationdb.record.metadata.IndexTypes; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; +import com.apple.foundationdb.record.query.plan.cascades.AliasMap; +import com.apple.foundationdb.record.query.plan.cascades.BuiltInFunction; +import com.apple.foundationdb.record.query.plan.cascades.SemanticException; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; +import com.apple.foundationdb.record.query.plan.cascades.typing.TypeRepository; +import com.apple.foundationdb.record.query.plan.cascades.typing.Typed; +import com.google.auto.service.AutoService; +import com.google.common.base.Verify; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Message; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.List; +import java.util.Locale; + +/** + * Represents a compile-time aggregation value that must be backed by an aggregation index, and can not be evaluated + * at runtime by a streaming aggregation operator. + * This value will be absorbed by a matching aggregation index at optimisation phase. + */ +@API(API.Status.EXPERIMENTAL) +public abstract class IndexOnlyAggregateValue implements Value, AggregateValue, Value.CompileTimeValue, ValueWithChild, IndexableAggregateValue { + + private static final ObjectPlanHash BASE_HASH = new ObjectPlanHash("Index-Only-Aggregate-Value"); + + enum PhysicalOperator { + MAX_EVER_LONG, + MIN_EVER_LONG + } + + @Nonnull + protected final PhysicalOperator operator; + + @Nonnull + private final Value child; + + /** + * Creates a new instance of {@link IndexOnlyAggregateValue}. + * @param operator the aggregation function. + * @param child the child {@link Value}. + */ + protected IndexOnlyAggregateValue(@Nonnull final PhysicalOperator operator, + @Nonnull final Value child) { + this.operator = operator; + this.child = child; + } + + @Nonnull + @Override + public Value getChild() { + return child; + } + + @Nonnull + @Override + public Type getResultType() { + return child.getResultType(); + } + + @Nonnull + @Override + public Accumulator createAccumulator(@Nonnull final TypeRepository typeRepository) { + throw new IllegalStateException("unable to create accumulator in a compile-time aggregation function"); + } + + @Nullable + @Override + public Object evalToPartial(@Nonnull final FDBRecordStoreBase store, @Nonnull final EvaluationContext context) { + throw new IllegalStateException("unable to evalToPartial in a compile-time aggregation function"); + } + + @Override + public int hashCodeWithoutChildren() { + return PlanHashable.objectsPlanHash(PlanHashKind.FOR_CONTINUATION, BASE_HASH, operator); + } + + @Override + public String toString() { + return operator.name().toLowerCase(Locale.getDefault()) + "(" + child + ")"; + } + + @Override + public int planHash(@Nonnull final PlanHashKind hashKind) { + return PlanHashable.objectsPlanHash(hashKind, BASE_HASH, operator, child); + } + + @Override + public int hashCode() { + return semanticHashCode(); + } + + @SuppressWarnings("PMD.CompareObjectsWithEquals") + @Override + public boolean equalsWithoutChildren(@Nonnull final Value other, @Nonnull final AliasMap equivalenceMap) { + if (this == other) { + return true; + } + + return other.getClass() == getClass() && ((IndexOnlyAggregateValue)other).operator.equals(operator); + } + + @SuppressWarnings("EqualsWhichDoesntCheckParameterClass") + @SpotBugsSuppressWarnings("EQ_UNUSUAL") + @Override + public boolean equals(final Object other) { + return semanticEquals(other, AliasMap.identitiesFor(getCorrelatedTo())); + } + + static class MinEverLong extends IndexOnlyAggregateValue { + + /** + * Creates a new instance of {@link MinEverLong}. + * + * @param operator the aggregation function. + * @param child the child {@link Value}. + */ + MinEverLong(@Nonnull final PhysicalOperator operator, @Nonnull final Value child) { + super(operator, child); + } + + @Nonnull + @Override + public String getIndexName() { + return IndexTypes.MIN_EVER_LONG; + } + + @Nonnull + private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, + @Nonnull BuiltInFunction builtInFunction, + @Nonnull final List arguments) { + Verify.verify(arguments.size() == 1); + final Typed arg0 = arguments.get(0); + final Type type0 = arg0.getResultType(); + SemanticException.check(type0.isNumeric(), SemanticException.ErrorCode.UNKNOWN, String.format("only numeric types allowed in %s aggregation operation", IndexTypes.MIN_EVER_LONG)); + return new MinEverLong(PhysicalOperator.valueOf(builtInFunction.getFunctionName()), (Value)arg0); + } + + @Nonnull + @Override + public ValueWithChild withNewChild(@Nonnull final Value rebasedChild) { + return new MinEverLong(operator, rebasedChild); + } + } + + static class MaxEverLong extends IndexOnlyAggregateValue { + + /** + * Creates a new instance of {@link MaxEverLong}. + * + * @param operator the aggregation function. + * @param child the child {@link Value}. + */ + MaxEverLong(@Nonnull final PhysicalOperator operator, @Nonnull final Value child) { + super(operator, child); + } + + @Nonnull + @Override + public String getIndexName() { + return IndexTypes.MAX_EVER_LONG; + } + + @Nonnull + private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, + @Nonnull BuiltInFunction builtInFunction, + @Nonnull final List arguments) { + Verify.verify(arguments.size() == 1); + final Typed arg0 = arguments.get(0); + final Type type0 = arg0.getResultType(); + SemanticException.check(type0.isNumeric(), SemanticException.ErrorCode.UNKNOWN, String.format("only numeric types allowed in %s aggregation operation", IndexTypes.MAX_EVER_LONG)); + return new MaxEverLong(PhysicalOperator.valueOf(builtInFunction.getFunctionName()), (Value)arg0); + } + + @Nonnull + @Override + public ValueWithChild withNewChild(@Nonnull final Value rebasedChild) { + return new MaxEverLong(operator, rebasedChild); + } + } + + /** + * The {@code min_ever} function. + */ + @AutoService(BuiltInFunction.class) + public static class MinEverLongFn extends BuiltInFunction { + public MinEverLongFn() { + super(PhysicalOperator.MIN_EVER_LONG.name(), ImmutableList.of(new Type.Any()), MinEverLong::encapsulate); + } + } + + /** + * The {@code max_ever} function. + */ + @AutoService(BuiltInFunction.class) + public static class MaxEverLongFn extends BuiltInFunction { + public MaxEverLongFn() { + super(PhysicalOperator.MAX_EVER_LONG.name(), ImmutableList.of(new Type.Any()), MaxEverLong::encapsulate); + } + } +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/IndexableAggregateValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/IndexableAggregateValue.java new file mode 100644 index 0000000000..4ba9948f03 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/IndexableAggregateValue.java @@ -0,0 +1,36 @@ +/* + * IndexableAggregationValue.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.query.plan.cascades.values; + +import com.apple.foundationdb.annotation.API; + +import javax.annotation.Nonnull; + +/** + * Tag interface for {@link AggregateValue} that are backed by an aggregate index. + */ +@API(API.Status.EXPERIMENTAL) +public interface IndexableAggregateValue extends Value { + + @Nonnull + String getIndexName(); + +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/MessageValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/MessageValue.java index 0d0ed99495..7356f40d12 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/MessageValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/MessageValue.java @@ -25,6 +25,7 @@ import com.apple.foundationdb.record.metadata.expressions.TupleFieldsHelper; import com.apple.foundationdb.record.query.expressions.Query; import com.apple.foundationdb.record.query.plan.cascades.typing.Type; +import com.google.common.primitives.ImmutableIntArray; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; import com.google.protobuf.MessageOrBuilder; @@ -91,6 +92,24 @@ public static Object getFieldValueForFields(@Nonnull MessageOrBuilder message, @ return getFieldOnMessage(current, fields.get(fields.size() - 1).getFieldIndex()); } + @SuppressWarnings("UnstableApiUsage") // caused by usage of Guava's ImmutableIntArray. + @Nullable + public static Object getFieldValueForFieldOrdinals(@Nonnull MessageOrBuilder message, @Nonnull ImmutableIntArray fieldOrdinals) { + if (fieldOrdinals.isEmpty()) { + throw new RecordCoreException("empty list of fields"); + } + MessageOrBuilder current = message; + int fieldOrdinal; + // Notice that up to fieldOrdinals.length() - 2 are calling getFieldMessageOnMessageByOrdinal, and fieldOrdinals.length() - 1 is calling getFieldOnMessageByOrdinal + for (fieldOrdinal = 0; fieldOrdinal < fieldOrdinals.length() - 1; fieldOrdinal++) { + current = getFieldMessageOnMessageByOrdinal(current, fieldOrdinals.get(fieldOrdinal)); + if (current == null) { + return null; + } + } + return getFieldOnMessageByOrdinal(current, fieldOrdinals.get(fieldOrdinals.length() - 1)); + } + /** * Get the value of the field with the given field name on the given message. @@ -107,7 +126,7 @@ public static Object getFieldOnMessage(@Nonnull MessageOrBuilder message, @Nonnu } /** - * Get the value of the field with the given field name on the given message. + * Get the value of the field with the given field number on the given message. * If the field is repeated, the repeated values are combined into a list. If the field has a message type, * the value is returned as a {@link Message} of that type. Otherwise, the field is returned as a primitive. * @param message a message or builder to extract the field from @@ -142,6 +161,12 @@ private static Object getFieldOnMessage(@Nonnull MessageOrBuilder message, @Nonn } } + @Nullable + public static Object getFieldOnMessageByOrdinal(@Nonnull MessageOrBuilder message, int fieldOrdinal) { + final Descriptors.FieldDescriptor field = findFieldDescriptorOnMessageByOrdinal(message, fieldOrdinal); + return getFieldOnMessage(message, field); + } + @Nonnull public static Descriptors.FieldDescriptor findFieldDescriptorOnMessage(@Nonnull MessageOrBuilder message, @Nonnull String fieldName) { final Descriptors.FieldDescriptor field = message.getDescriptorForType().findFieldByName(fieldName); @@ -160,6 +185,14 @@ public static Descriptors.FieldDescriptor findFieldDescriptorOnMessage(@Nonnull return field; } + @Nonnull + public static Descriptors.FieldDescriptor findFieldDescriptorOnMessageByOrdinal(@Nonnull MessageOrBuilder message, int fieldOrdinal) { + if (fieldOrdinal < 0 || fieldOrdinal >= message.getDescriptorForType().getFields().size()) { + throw new Query.InvalidExpressionException("Missing field (#ord=" + fieldOrdinal + ")"); + } + return message.getDescriptorForType().getFields().get(fieldOrdinal); + } + @Nullable private static Message getFieldMessageOnMessage(@Nonnull MessageOrBuilder message, @Nonnull String fieldName) { final Descriptors.FieldDescriptor field = findFieldDescriptorOnMessage(message, fieldName); @@ -182,6 +215,12 @@ private static Message getFieldMessageOnMessage(@Nonnull MessageOrBuilder messag return null; } + @Nullable + private static Message getFieldMessageOnMessageByOrdinal(@Nonnull MessageOrBuilder message, int fieldOrdinal) { + final Descriptors.FieldDescriptor field = findFieldDescriptorOnMessageByOrdinal(message, fieldOrdinal); + return getFieldMessageOnMessage(message, field); + } + private MessageValue() { } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java index 398d156c7f..83d36def57 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/NumericAggregationValue.java @@ -25,6 +25,7 @@ import com.apple.foundationdb.record.EvaluationContext; import com.apple.foundationdb.record.ObjectPlanHash; import com.apple.foundationdb.record.PlanHashable; +import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.BuiltInFunction; @@ -50,6 +51,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.function.BiFunction; import java.util.function.BinaryOperator; import java.util.function.Supplier; import java.util.function.UnaryOperator; @@ -60,7 +62,7 @@ * Aggregation over numeric values. */ @API(API.Status.EXPERIMENTAL) -public class NumericAggregationValue implements ValueWithChild, AggregateValue { +public abstract class NumericAggregationValue implements ValueWithChild, AggregateValue { @Nonnull private static final ObjectPlanHash BASE_HASH = new ObjectPlanHash("Sum-Value"); @Nonnull @@ -68,12 +70,12 @@ public class NumericAggregationValue implements ValueWithChild, AggregateValue { Suppliers.memoize(NumericAggregationValue::computeOperatorMap); @Nonnull - private final PhysicalOperator operator; + protected final PhysicalOperator operator; @Nonnull private final Value child; - public NumericAggregationValue(@Nonnull PhysicalOperator operator, - @Nonnull Value child) { + protected NumericAggregationValue(@Nonnull final PhysicalOperator operator, + @Nonnull final Value child) { this.operator = operator; this.child = child; } @@ -84,6 +86,108 @@ public Object eval(@Nonnull final FDBRecordStoreBase stor throw new IllegalStateException("unable to eval an aggregation function with eval()"); } + /** + * Sum aggregation {@code Value}. + */ + public static class Sum extends NumericAggregationValue implements StreamableAggregateValue, IndexableAggregateValue { + + public Sum(@Nonnull final PhysicalOperator operator, @Nonnull final Value child) { + super(operator, child); + } + + @Nonnull + @Override + public String getIndexName() { + return IndexTypes.SUM; + } + + @Nonnull + @SuppressWarnings("PMD.UnusedFormalParameter") + private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, + @Nonnull BuiltInFunction builtInFunction, + @Nonnull final List arguments) { + return NumericAggregationValue.encapsulate(builtInFunction.getFunctionName(), arguments, Sum::new); + } + + @Nonnull + @Override + public ValueWithChild withNewChild(@Nonnull final Value newChild) { + return new Sum(operator, newChild); + } + } + + /** + * Average aggregation {@code Value}. + */ + public static class Avg extends NumericAggregationValue implements StreamableAggregateValue { + + public Avg(@Nonnull final PhysicalOperator operator, @Nonnull final Value child) { + super(operator, child); + } + + @Nonnull + @SuppressWarnings("PMD.UnusedFormalParameter") + private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, + @Nonnull BuiltInFunction builtInFunction, + @Nonnull final List arguments) { + return NumericAggregationValue.encapsulate(builtInFunction.getFunctionName(), arguments, Avg::new); + } + + @Nonnull + @Override + public ValueWithChild withNewChild(@Nonnull final Value newChild) { + return new Avg(operator, newChild); + } + } + + /** + * Min aggregation {@code Value}. + */ + public static class Min extends NumericAggregationValue implements StreamableAggregateValue { + + public Min(@Nonnull final PhysicalOperator operator, @Nonnull final Value child) { + super(operator, child); + } + + @Nonnull + @SuppressWarnings("PMD.UnusedFormalParameter") + private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, + @Nonnull BuiltInFunction builtInFunction, + @Nonnull final List arguments) { + return NumericAggregationValue.encapsulate(builtInFunction.getFunctionName(), arguments, Min::new); + } + + @Nonnull + @Override + public ValueWithChild withNewChild(@Nonnull final Value newChild) { + return new Min(operator, newChild); + } + } + + /** + * Max aggregation {@code Value}. + */ + public static class Max extends NumericAggregationValue implements StreamableAggregateValue { + + public Max(@Nonnull final PhysicalOperator operator, @Nonnull final Value child) { + super(operator, child); + } + + @Nonnull + @SuppressWarnings("PMD.UnusedFormalParameter") + private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, + @Nonnull BuiltInFunction builtInFunction, + @Nonnull final List arguments) { + return NumericAggregationValue.encapsulate(builtInFunction.getFunctionName(), arguments, Max::new); + } + + @Nonnull + @Override + public ValueWithChild withNewChild(@Nonnull final Value newChild) { + return new Max(operator, newChild); + } + } + @Nullable @Override public Object evalToPartial(@Nonnull final FDBRecordStoreBase store, @Nonnull final EvaluationContext context) { @@ -108,23 +212,12 @@ public Type getResultType() { return Type.primitiveType(operator.getResultTypeCode()); } - @Nonnull - public String getOperatorName() { - return operator.logicalOperator.name(); - } - @Nonnull @Override public Value getChild() { return child; } - @Nonnull - @Override - public ValueWithChild withNewChild(@Nonnull final Value newChild) { - return new NumericAggregationValue(this.operator, newChild); - } - @Override public int hashCodeWithoutChildren() { return PlanHashable.objectsPlanHash(PlanHashKind.FOR_CONTINUATION, BASE_HASH, operator); @@ -158,15 +251,9 @@ private static Map, PhysicalOperator> getOperato } @Nonnull - @SuppressWarnings("PMD.UnusedFormalParameter") - private static AggregateValue encapsulate(@Nonnull TypeRepository.Builder ignored, - @Nonnull BuiltInFunction builtInFunction, - @Nonnull final List arguments) { - return encapsulate(builtInFunction.getFunctionName(), arguments); - } - - @Nonnull - private static AggregateValue encapsulate(@Nonnull final String functionName, @Nonnull final List arguments) { + private static AggregateValue encapsulate(@Nonnull final String functionName, + @Nonnull final List arguments, + @Nonnull final BiFunction valueSupplier) { Verify.verify(arguments.size() == 1); final Typed arg0 = arguments.get(0); final Type type0 = arg0.getResultType(); @@ -181,7 +268,7 @@ private static AggregateValue encapsulate(@Nonnull final String functionName, @N Verify.verifyNotNull(physicalOperator, "unable to encapsulate aggregate operation due to type mismatch(es)"); - return new NumericAggregationValue(physicalOperator, (Value)arg0); + return valueSupplier.apply(physicalOperator, (Value)arg0); } private static Map, PhysicalOperator> computeOperatorMap() { @@ -199,7 +286,7 @@ private static Map, PhysicalOperator> computeOpe public static class SumFn extends BuiltInFunction { public SumFn() { super("sum", - ImmutableList.of(new Type.Any()), NumericAggregationValue::encapsulate); + ImmutableList.of(new Type.Any()), Sum::encapsulate); } } @@ -210,7 +297,7 @@ public SumFn() { public static class AvgFn extends BuiltInFunction { public AvgFn() { super("avg", - ImmutableList.of(new Type.Any()), NumericAggregationValue::encapsulate); + ImmutableList.of(new Type.Any()), Avg::encapsulate); } } @@ -221,7 +308,7 @@ public AvgFn() { public static class MinFn extends BuiltInFunction { public MinFn() { super("min", - ImmutableList.of(new Type.Any()), NumericAggregationValue::encapsulate); + ImmutableList.of(new Type.Any()), Min::encapsulate); } } @@ -232,7 +319,7 @@ public MinFn() { public static class MaxFn extends BuiltInFunction { public MaxFn() { super("max", - ImmutableList.of(new Type.Any()), NumericAggregationValue::encapsulate); + ImmutableList.of(new Type.Any()), Max::encapsulate); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/StreamableAggregateValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/StreamableAggregateValue.java new file mode 100644 index 0000000000..23855c37cd --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/StreamableAggregateValue.java @@ -0,0 +1,32 @@ +/* + * StreamableAggregateValue.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2022 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.query.plan.cascades.values; + +import com.apple.foundationdb.annotation.API; + +/** + * Tag interface used to mark {@link AggregateValue} implementations that can be calculated by a + * {@link com.apple.foundationdb.record.query.plan.plans.RecordQueryStreamingAggregationPlan} operator. + * This is mainly used for matching. + */ +@API(API.Status.EXPERIMENTAL) +public interface StreamableAggregateValue extends Value { +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/TransformValue.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/TransformValue.java index f8ace57c0c..a8b0aa4f68 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/TransformValue.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/TransformValue.java @@ -40,16 +40,19 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; +import com.google.common.collect.Maps; import com.google.common.collect.PeekingIterator; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; import com.google.protobuf.Message; +import org.apache.commons.lang3.tuple.Pair; import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -172,22 +175,23 @@ public Object evalSubtree(@Nonnull final FDBRecordStoreBase< return value.eval(store, context); } else { Verify.verifyNotNull(descriptor); - final var fieldIndexToFieldMap = Verify.verifyNotNull(trieNode.getFieldIndexToFieldMap()); + final var fieldOrdinalToFieldInfoMap = Verify.verifyNotNull(trieNode.getFieldOrdinalToFieldMap()); final var childrenMap = Verify.verifyNotNull(trieNode.getChildrenMap()); final var subRecord = (M)current; - final var fieldDescriptors = descriptor.getFields(); + final var fieldDescriptors = Objects.requireNonNull(descriptor).getFields(); final var resultMessageBuilder = DynamicMessage.newBuilder(descriptor); - for (final var fieldDescriptor : fieldDescriptors) { - final var field = fieldIndexToFieldMap.get(fieldDescriptor.getNumber()); - if (field != null) { - final var fieldTrieNode = Verify.verifyNotNull(childrenMap.get(field)); + for (int i = 0; i < Objects.requireNonNull(descriptor).getFields().size(); ++i) { + final var fieldDescriptor = fieldDescriptors.get(i); + final var fieldOrdinalAndType = fieldOrdinalToFieldInfoMap.get(i); + if (fieldOrdinalAndType != null) { + final var fieldTrieNode = Verify.verifyNotNull(childrenMap.get(fieldOrdinalAndType)); var fieldResult = evalSubtree(store, context, fieldTrieNode, fieldDescriptor.getJavaType() == Descriptors.FieldDescriptor.JavaType.MESSAGE ? fieldDescriptor.getMessageType() : null, subRecord == null ? null : subRecord.getField(fieldDescriptor)); - final var fieldType = field.getFieldType(); + final var fieldType = fieldOrdinalAndType.getValue(); if (fieldType.getTypeCode() == Type.TypeCode.ARRAY && fieldType.isNullable()) { final var wrappedDescriptor = fieldDescriptor.getMessageType(); final var wrapperBuilder = DynamicMessage.newBuilder(wrappedDescriptor); @@ -261,10 +265,10 @@ private static Map checkAndPrepareTransformMap(@Nonnull final // TODO check this using the type, not just the type code! For that to work we need isAssignableTo() checking // in the type system, so we can account for e.g. differences in nullabilities between the types. SemanticException.check(entry.getKey() - .getLastField() - .getFieldType().getTypeCode().equals(entry.getValue().getResultType().getTypeCode()), SemanticException.ErrorCode.ASSIGNMENT_WRONG_TYPE); + .getLastFieldType() + .getTypeCode().equals(entry.getValue().getResultType().getTypeCode()), SemanticException.ErrorCode.ASSIGNMENT_WRONG_TYPE); } - return ImmutableMap.copyOf(transformMap); + return Maps.newLinkedHashMap(transformMap); } @Nonnull @@ -292,10 +296,11 @@ private static List checkAndPrepareOrderedFieldPaths(@Nonnull final M * @return a {@link TrieNode} */ @Nonnull - private static TrieNode computeTrieForFieldPaths(@Nonnull final Collection orderedFieldPaths, @Nonnull final Map transformMap) { - return computeTrieForFieldPaths(new FieldPath(ImmutableList.of()), transformMap, Iterators.peekingIterator(orderedFieldPaths.iterator())); + public static TrieNode computeTrieForFieldPaths(@Nonnull final Collection orderedFieldPaths, @Nonnull final Map transformMap) { + return computeTrieForFieldPaths(FieldPath.empty(), transformMap, Iterators.peekingIterator(orderedFieldPaths.iterator())); } + @SuppressWarnings("UnstableApiUsage") @Nonnull private static TrieNode computeTrieForFieldPaths(@Nonnull final FieldPath prefix, @Nonnull final Map transformMap, @@ -304,25 +309,22 @@ private static TrieNode computeTrieForFieldPaths(@Nonnull final FieldPath prefix orderedFieldPathIterator.next(); return new TrieNode(Verify.verifyNotNull(transformMap.get(prefix)), null); } - final var childrenMapBuilder = ImmutableMap.builder(); + final var childrenMapBuilder = Maps., TrieNode>newLinkedHashMap(); while (orderedFieldPathIterator.hasNext()) { final var fieldPath = orderedFieldPathIterator.peek(); if (!prefix.isPrefixOf(fieldPath)) { break; } - final var prefixFields = prefix.getFields(); - final var currentField = fieldPath.getFields().get(prefixFields.size()); - final var nestedPrefix = new FieldPath(ImmutableList.builder() - .addAll(prefixFields) - .add(currentField) - .build()); + final var prefixLength = prefix.size(); + final var currentField = FieldPath.flat(fieldPath.getFieldNamesMaybe().get(prefixLength), fieldPath.getFieldTypes().get(prefixLength), fieldPath.getFieldOrdinals().get(prefixLength)); + final var nestedPrefix = prefix.withSuffix(currentField); final var currentTrie = computeTrieForFieldPaths(nestedPrefix, transformMap, orderedFieldPathIterator); - childrenMapBuilder.put(currentField, currentTrie); + childrenMapBuilder.put(Pair.of(currentField.getLastFieldOrdinal(), currentField.getLastFieldType()), currentTrie); } - return new TrieNode(null, childrenMapBuilder.build()); + return new TrieNode(null, childrenMapBuilder); } @Nonnull @@ -330,7 +332,7 @@ private static TrieNode computeTrieForFieldPaths(@Nonnull final FieldPath prefix public TransformValue withChildren(final Iterable newChildren) { Verify.verify(getChildren().size() == Iterables.size(newChildren)); - final var newTransformMapBuilder = ImmutableMap.builder(); + final var newTransformMapBuilder = Maps.newLinkedHashMap(); final var newChildrenIterator = newChildren.iterator(); @@ -346,7 +348,7 @@ public TransformValue withChildren(final Iterable newChildren) } Verify.verify(i == orderedFieldPaths.size()); - return new TransformValue(newInValue, newTransformMapBuilder.build()); + return new TransformValue(newInValue, newTransformMapBuilder); } /** @@ -356,14 +358,14 @@ public static class TrieNode { @Nullable private final Value value; @Nullable - private final Map childrenMap; - @Nullable - private final Map fieldIndexToFieldMap; + private final Map, TrieNode> childrenMap; + @Nonnull + private final Supplier>> fieldOrdinalToFieldMapSupplier; - public TrieNode(@Nullable final Value value, @Nullable final Map childrenMap) { + public TrieNode(@Nullable final Value value, @Nullable final Map, TrieNode> childrenMap) { this.value = value; - this.childrenMap = childrenMap == null ? null : ImmutableMap.copyOf(childrenMap); - this.fieldIndexToFieldMap = childrenMap == null ? null : computeFieldIndexToFieldMap(this.childrenMap); + this.childrenMap = childrenMap == null ? null : Maps.newLinkedHashMap(childrenMap); + this.fieldOrdinalToFieldMapSupplier = Suppliers.memoize(() -> computeFieldOrdinalToFieldMap(childrenMap)); } @Nullable @@ -372,20 +374,23 @@ public Value getValue() { } @Nullable - public Map getChildrenMap() { + public Map, TrieNode> getChildrenMap() { return childrenMap; } @Nullable - public Map getFieldIndexToFieldMap() { - return fieldIndexToFieldMap; + public Map> getFieldOrdinalToFieldMap() { + return fieldOrdinalToFieldMapSupplier.get(); } - @Nonnull - private static Map computeFieldIndexToFieldMap(@Nonnull final Map childrenMap) { - final var resultBuilder = ImmutableMap.builder(); + @Nullable + private static Map> computeFieldOrdinalToFieldMap(@Nullable final Map, TrieNode> childrenMap) { + if (childrenMap == null) { + return null; + } + final var resultBuilder = ImmutableMap.>builder(); for (final var entry : childrenMap.entrySet()) { - resultBuilder.put(entry.getKey().getFieldIndex(), entry.getKey()); + resultBuilder.put(entry.getKey().getLeft(), entry.getKey()); } return resultBuilder.build(); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Value.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Value.java index 65c985d728..7c4647c34d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Value.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Value.java @@ -552,4 +552,44 @@ default List simplifyOrderingValue(@Nonnull final AliasMap aliasMap, @Non return ImmutableList.of(Simplification.simplify(simplifiedOrderingValue, aliasMap, constantAliases, OrderingValueSimplificationPerPartRuleSet.ofOrderingSimplificationPerPartRules())); } } + + @SuppressWarnings("PMD.CompareObjectsWithEquals") + default boolean subsumedBy(@Nullable final Value other, @Nonnull final AliasMap aliasMap) { + if (other == null) { + return false; + } + + if (this == other) { + return true; + } + + if (!subsumedByWithoutChildren(other, aliasMap)) { + return false; + } + + final Iterator children = getChildren().iterator(); + final Iterator otherChildren = other.getChildren().iterator(); + + while (children.hasNext()) { + if (!otherChildren.hasNext()) { + return false; + } + + if (!children.next().subsumedBy(otherChildren.next(), aliasMap)) { + return false; + } + } + + return !otherChildren.hasNext(); + } + + @SuppressWarnings({"unused", "PMD.CompareObjectsWithEquals"}) + default boolean subsumedByWithoutChildren(@Nonnull final Value other, + @Nonnull final AliasMap equivalenceMap) { + if (this == other) { + return true; + } + + return other.getClass() == getClass(); + } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Values.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Values.java index 678aea33f8..981ee11e86 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Values.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/Values.java @@ -91,8 +91,10 @@ public static Set primitiveAccessorsForType(@Nonnull final Type type, final var recordType = (Type.Record)type; final var fields = recordType.getFields(); - for (final var field : fields) { - primitiveAccessorsForType(field.getFieldType(), () -> FieldValue.ofFieldsAndFuseIfPossible(baseValueSupplier.get(), ImmutableList.of(field)), constantAliases).stream() + for (int i = 0; i < fields.size(); i++) { + final var field = fields.get(i); + final int finalI1 = i; + primitiveAccessorsForType(field.getFieldType(), () -> FieldValue.ofFieldsAndFuseIfPossible(baseValueSupplier.get(), FieldValue.FieldPath.flat(field.getFieldNameOptional(), field.getFieldType(), finalI1)), constantAliases).stream() .map(orderingValue -> orderingValue.simplify(DefaultValueSimplificationRuleSet.ofSimplificationRules(), AliasMap.emptyMap(), constantAliases)) .forEach(orderingValuesBuilder::add); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/CompensateRecordConstructorRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/CompensateRecordConstructorRule.java index 2bcd567fb6..c39b22a2ca 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/CompensateRecordConstructorRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/CompensateRecordConstructorRule.java @@ -1,5 +1,5 @@ /* - * ComposeFieldValueOverRecordConstructorRule.java + * CompensateRecordConstructorRule.java * * This source file is part of the FoundationDB open source project * @@ -23,17 +23,14 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.query.plan.cascades.LinkedIdentityMap; import com.apple.foundationdb.record.query.plan.cascades.matching.structure.BindingMatcher; +import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; import com.apple.foundationdb.record.query.plan.cascades.values.RecordConstructorValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; import com.apple.foundationdb.record.query.plan.cascades.values.simplification.MatchOrCompensateFieldValueRule.FieldValueCompensation; -import com.google.common.collect.ImmutableList; -import org.apache.commons.lang3.tuple.Pair; import javax.annotation.Nonnull; import java.util.Map; import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.MultiMatcher.all; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers.anyValue; @@ -62,37 +59,27 @@ public CompensateRecordConstructorRule() { public void onMatch(@Nonnull final ValueComputationRuleCall, Map>> call) { final var bindings = call.getBindings(); final var recordConstructorValue = bindings.get(rootMatcher); + final var mergedMatchedValuesMap = new LinkedIdentityMap>(); - final var mergedMatchedValuesMap = - recordConstructorValue.getColumns() - .stream() - .flatMap(column -> { - final var childResult = call.getResult(column.getValue()); - return childResult != null ? Stream.of(Pair.of(column, childResult.getResult())) : Stream.empty(); - }) - .map(columnWithResult -> { - final var column = columnWithResult.getLeft(); - final var matchedValuesMap = columnWithResult.getRight(); + for (int i = 0; i < recordConstructorValue.getColumns().size(); ++i) { + final var column = recordConstructorValue.getColumns().get(i); + final var childResult = call.getResult(column.getValue()); + if (childResult == null) { + continue; + } - // - // No we have a column and the result we computed for all columns that do have results associated with them, - // i.e. the columns flowing results of values we care about. - // - final var newMatchedValuesMap = new LinkedIdentityMap>(); - - for (final var childValueEntry : matchedValuesMap.entrySet()) { - final var argumentValue = childValueEntry.getKey(); - final var argumentValueCompensation = childValueEntry.getValue(); - newMatchedValuesMap.put(argumentValue, - new FieldValueCompensation(ImmutableList.of(column.getField()), argumentValueCompensation)); - } - return newMatchedValuesMap; - }) - .flatMap(map -> map.entrySet().stream()) - .collect(Collectors.toMap(Map.Entry::getKey, - Map.Entry::getValue, - (l, r) -> l, - LinkedIdentityMap::new)); + // + // No we have a column and the result we computed for all columns that do have results associated with them, + // i.e. the columns flowing results of values we care about. + // + for (final var childValueEntry : childResult.getResult().entrySet()) { + final var argumentValue = childValueEntry.getKey(); + final var argumentValueCompensation = childValueEntry.getValue(); + final var field = column.getField(); + mergedMatchedValuesMap.putIfAbsent(argumentValue, + new FieldValueCompensation(FieldValue.FieldPath.flat(field.getFieldNameOptional(), field.getFieldType(), i), argumentValueCompensation)); + } + } call.yield(recordConstructorValue, mergedMatchedValuesMap); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverFieldValueRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverFieldValueRule.java index effc2d8eaa..44f4ff2463 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverFieldValueRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverFieldValueRule.java @@ -24,11 +24,10 @@ import com.apple.foundationdb.record.query.plan.cascades.matching.structure.BindingMatcher; import com.apple.foundationdb.record.query.plan.cascades.matching.structure.CollectionMatcher; import com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers; -import com.apple.foundationdb.record.query.plan.cascades.typing.Type.Record.Field; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; import com.google.common.base.Verify; -import com.google.common.collect.ImmutableList; import javax.annotation.Nonnull; @@ -48,18 +47,24 @@ public class ComposeFieldValueOverFieldValueRule extends ValueSimplificationRule @Nonnull private static final BindingMatcher innerChildMatcher = anyValue(); @Nonnull - private static final CollectionMatcher innerFieldPathMatcher = all(anyObject()); + private static final CollectionMatcher innerFieldPathOrdinalsMatcher = all(anyObject()); + + @Nonnull + private static final CollectionMatcher innerFieldPathTypesMatcher = all(anyObject()); @Nonnull private static final BindingMatcher innerFieldValueMatcher = - ValueMatchers.fieldValueWithFieldPath(innerChildMatcher, innerFieldPathMatcher); + ValueMatchers.fieldValueWithFieldPath(innerChildMatcher, innerFieldPathOrdinalsMatcher, innerFieldPathTypesMatcher); @Nonnull - private static final CollectionMatcher outerFieldPathMatcher = all(anyObject()); + private static final CollectionMatcher outerFieldPathOrdinalsMatcher = all(anyObject()); + + @Nonnull + private static final CollectionMatcher outerFieldPathTypesMatcher = all(anyObject()); @Nonnull private static final BindingMatcher rootMatcher = - ValueMatchers.fieldValueWithFieldPath(innerFieldValueMatcher, outerFieldPathMatcher); + ValueMatchers.fieldValueWithFieldPath(innerFieldValueMatcher, outerFieldPathOrdinalsMatcher, outerFieldPathTypesMatcher); public ComposeFieldValueOverFieldValueRule() { super(rootMatcher); @@ -69,12 +74,18 @@ public ComposeFieldValueOverFieldValueRule() { public void onMatch(@Nonnull final ValueSimplificationRuleCall call) { final var bindings = call.getBindings(); - final var innerChild = bindings.get(innerChildMatcher); - final var innerFieldPath = bindings.get(innerFieldPathMatcher); - Verify.verify(!innerFieldPath.isEmpty()); - final var outerFieldPath = bindings.get(outerFieldPathMatcher); - Verify.verify(!outerFieldPath.isEmpty()); - - call.yield(FieldValue.ofFields(innerChild, ImmutableList.builder().addAll(innerFieldPath).addAll(outerFieldPath).build())); + final var grandChild = bindings.get(innerChildMatcher); + final var innerFieldPathOrdinals = bindings.get(innerFieldPathOrdinalsMatcher); + final var innerFieldPathTypes = bindings.get(innerFieldPathTypesMatcher); + Verify.verify(!innerFieldPathOrdinals.isEmpty()); + Verify.verify(!innerFieldPathTypes.isEmpty()); + final var outer = bindings.get(rootMatcher); + final var child = outer.getChild(); + final var outerFieldPathOrdinals = bindings.get(outerFieldPathOrdinalsMatcher); + final var outerFieldPathTypes = bindings.get(outerFieldPathTypesMatcher); + Verify.verify(child instanceof FieldValue); + Verify.verify(!outerFieldPathOrdinals.isEmpty()); + Verify.verify(!outerFieldPathTypes.isEmpty()); + call.yield(FieldValue.ofFields(grandChild, ((FieldValue)(child)).getFieldPath().withSuffix(outer.getFieldPath()))); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverRecordConstructorRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverRecordConstructorRule.java index fe7ba53566..7900f66e70 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverRecordConstructorRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/ComposeFieldValueOverRecordConstructorRule.java @@ -22,17 +22,15 @@ import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.annotation.SpotBugsSuppressWarnings; -import com.apple.foundationdb.record.RecordCoreException; import com.apple.foundationdb.record.query.plan.cascades.Column; import com.apple.foundationdb.record.query.plan.cascades.matching.structure.BindingMatcher; import com.apple.foundationdb.record.query.plan.cascades.matching.structure.CollectionMatcher; import com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers; -import com.apple.foundationdb.record.query.plan.cascades.typing.Type.Record.Field; +import com.apple.foundationdb.record.query.plan.cascades.typing.Type; import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; import com.apple.foundationdb.record.query.plan.cascades.values.RecordConstructorValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; import com.google.common.base.Verify; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import javax.annotation.Nonnull; @@ -54,11 +52,14 @@ public class ComposeFieldValueOverRecordConstructorRule extends ValueSimplificat recordConstructorValue(all(anyValue())); @Nonnull - private static final CollectionMatcher fieldPathMatcher = all(anyObject()); + private static final CollectionMatcher fieldPathOrdinalsMatcher = all(anyObject()); + + @Nonnull + private static final CollectionMatcher fieldPathTypesMatcher = all(anyObject()); @Nonnull private static final BindingMatcher rootMatcher = - ValueMatchers.fieldValueWithFieldPath(recordConstructorMatcher, fieldPathMatcher); + ValueMatchers.fieldValueWithFieldPath(recordConstructorMatcher, fieldPathOrdinalsMatcher, fieldPathTypesMatcher); public ComposeFieldValueOverRecordConstructorRule() { super(rootMatcher); @@ -69,31 +70,29 @@ public ComposeFieldValueOverRecordConstructorRule() { public void onMatch(@Nonnull final ValueSimplificationRuleCall call) { final var bindings = call.getBindings(); - final var fieldPath = bindings.get(fieldPathMatcher); - Verify.verify(!fieldPath.isEmpty()); + final var fieldPathOrdinals = bindings.get(fieldPathOrdinalsMatcher); + Verify.verify(!fieldPathOrdinals.isEmpty()); + final var fieldPathTypes = bindings.get(fieldPathTypesMatcher); + Verify.verify(!fieldPathTypes.isEmpty()); final var recordConstructor = bindings.get(recordConstructorMatcher); - final var firstField = Objects.requireNonNull(Iterables.getFirst(fieldPath, null)); - final var column = findColumn(recordConstructor, firstField); - if (fieldPath.size() == 1) { + final var firstFieldOrdinal = Objects.requireNonNull(Iterables.getFirst(fieldPathOrdinals, null)); + final var fieldFieldType = Objects.requireNonNull(Iterables.getFirst(fieldPathTypes, null)); + final var column = findColumn(recordConstructor, firstFieldOrdinal, fieldFieldType); + + final var root = bindings.get(rootMatcher); + if (fieldPathOrdinals.size() == 1) { // just return the child call.yield(column.getValue()); } else { - call.yield(FieldValue.ofFields(column.getValue(), - fieldPath.stream() - .skip(1L) - .collect(ImmutableList.toImmutableList()))); + call.yield(FieldValue.ofFields(column.getValue(), root.getFieldPath().skip(1))); } } @Nonnull - private static Column findColumn(@Nonnull final RecordConstructorValue recordConstructorValue, @Nonnull final Field field) { - for (final var column : recordConstructorValue.getColumns()) { - if (field.getFieldIndex() == column.getField().getFieldIndex()) { - Verify.verify(field.getFieldNameOptional().equals(column.getField().getFieldNameOptional())); - return column; - } - } - throw new RecordCoreException("should have found field by field name"); + private static Column findColumn(@Nonnull final RecordConstructorValue recordConstructorValue, final int fieldOrdinal, @Nonnull Type fieldType) { + final var result = recordConstructorValue.getColumns().get(fieldOrdinal); + Verify.verify(result.getField().getFieldType().equals(fieldType)); + return result; } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/MatchOrCompensateFieldValueRule.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/MatchOrCompensateFieldValueRule.java index 1307b8e147..d5f906cfce 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/MatchOrCompensateFieldValueRule.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/cascades/values/simplification/MatchOrCompensateFieldValueRule.java @@ -28,10 +28,8 @@ import com.apple.foundationdb.record.query.plan.cascades.typing.Type; import com.apple.foundationdb.record.query.plan.cascades.values.FieldValue; import com.apple.foundationdb.record.query.plan.cascades.values.Value; -import com.google.common.collect.ImmutableList; import javax.annotation.Nonnull; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.function.Function; @@ -47,11 +45,14 @@ @SuppressWarnings("PMD.TooManyStaticImports") public class MatchOrCompensateFieldValueRule extends ValueComputationRule, Map>, FieldValue> { @Nonnull - private static final CollectionMatcher fieldPathMatcher = all(anyObject()); + private static final CollectionMatcher fieldPathOrdinalsMatcher = all(anyObject()); + + @Nonnull + private static final CollectionMatcher fieldPathTypesMatcher = all(anyObject()); @Nonnull private static final BindingMatcher rootMatcher = - ValueMatchers.fieldValueWithFieldPath(anyValue(), fieldPathMatcher); + ValueMatchers.fieldValueWithFieldPath(anyValue(), fieldPathOrdinalsMatcher, fieldPathTypesMatcher); public MatchOrCompensateFieldValueRule() { super(rootMatcher); @@ -80,7 +81,7 @@ public void onMatch(@Nonnull final ValueComputationRuleCall { if (pathSuffix.isEmpty()) { newMatchedValuesMap.put(toBePulledUpValue, Function.identity()); @@ -94,7 +95,7 @@ public void onMatch(@Nonnull final ValueComputationRuleCall newMatchedValuesMap.put(toBePulledUpValue, fieldValueCompensation.withSuffix(pathSuffix))); } } @@ -108,22 +109,23 @@ public void onMatch(@Nonnull final ValueComputationRuleCall { @Nonnull - private final List fieldPath; + private final FieldValue.FieldPath fieldPath; @Nonnull private final Function downstreamCompensation; - public FieldValueCompensation(@Nonnull final List fieldPath) { + public FieldValueCompensation(@Nonnull final FieldValue.FieldPath fieldPath) { this(fieldPath, Function.identity()); } - public FieldValueCompensation(@Nonnull final List fieldPath, @Nonnull final Function downstreamCompensation) { - this.fieldPath = ImmutableList.copyOf(fieldPath); + public FieldValueCompensation(@Nonnull final FieldValue.FieldPath fieldPath, @Nonnull final Function downstreamCompensation) { + this.fieldPath = fieldPath; this.downstreamCompensation = downstreamCompensation; } + @Nonnull - public List getFieldPath() { + public FieldValue.FieldPath getFieldPath() { return fieldPath; } @@ -133,7 +135,7 @@ public Value apply(final Value value) { return downstreamCompensation.apply(FieldValue.ofFieldsAndFuseIfPossible(value, fieldPath)); } - public Function withSuffix(@Nonnull final List suffixFieldPath) { + public Function withSuffix(@Nonnull final FieldValue.FieldPath suffixFieldPath) { if (suffixFieldPath.isEmpty()) { return downstreamCompensation; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryAggregateIndexPlan.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryAggregateIndexPlan.java new file mode 100644 index 0000000000..52044bde88 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryAggregateIndexPlan.java @@ -0,0 +1,298 @@ +/* + * RecordQueryAggregateIndexPlan.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2018 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.query.plan.plans; + +import com.apple.foundationdb.annotation.API; +import com.apple.foundationdb.record.EvaluationContext; +import com.apple.foundationdb.record.ExecuteProperties; +import com.apple.foundationdb.record.IndexScanType; +import com.apple.foundationdb.record.ObjectPlanHash; +import com.apple.foundationdb.record.PlanHashable; +import com.apple.foundationdb.record.RecordCursor; +import com.apple.foundationdb.record.RecordMetaData; +import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.record.metadata.RecordType; +import com.apple.foundationdb.record.provider.common.StoreTimer; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; +import com.apple.foundationdb.record.provider.foundationdb.FDBStoreTimer; +import com.apple.foundationdb.record.query.plan.AvailableFields; +import com.apple.foundationdb.record.query.plan.IndexKeyValueToPartialRecord; +import com.apple.foundationdb.record.query.plan.cascades.AliasMap; +import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; +import com.apple.foundationdb.record.query.plan.cascades.MatchCandidate; +import com.apple.foundationdb.record.query.plan.cascades.Quantifier; +import com.apple.foundationdb.record.query.plan.cascades.TranslationMap; +import com.apple.foundationdb.record.query.plan.cascades.explain.NodeInfo; +import com.apple.foundationdb.record.query.plan.cascades.explain.PlannerGraph; +import com.apple.foundationdb.record.query.plan.cascades.expressions.RelationalExpression; +import com.apple.foundationdb.record.query.plan.cascades.values.Value; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.StreamSupport; + +/** + * A query plan that reconstructs records from the entries in an aggregate index. + */ +@API(API.Status.INTERNAL) +public class RecordQueryAggregateIndexPlan implements RecordQueryPlanWithNoChildren, RecordQueryPlanWithMatchCandidate { + private static final ObjectPlanHash BASE_HASH = new ObjectPlanHash("Record-Query-Aggregate-Index-Plan"); + + @Nonnull + private final RecordQueryIndexPlan indexPlan; + @Nonnull + private final String recordTypeName; + @Nonnull + private final IndexKeyValueToPartialRecord toRecord; + + @Nonnull + private final Descriptors.Descriptor partialRecordDescriptor; + + @Nonnull + private final Value resultValue; + + /** + * Creates an instance of {@link RecordQueryAggregateIndexPlan}. + * + * @param indexPlan The underlying index. + * @param recordTypeName The name of the base record, used for debugging. + * @param indexEntryToPartialRecordConverter A converter from index entry to record. + * @param partialRecordDescriptor The descriptor of the resulting record. + * @param resultValue The result value. + */ + public RecordQueryAggregateIndexPlan(@Nonnull final RecordQueryIndexPlan indexPlan, + @Nonnull final String recordTypeName, + @Nonnull final IndexKeyValueToPartialRecord indexEntryToPartialRecordConverter, + @Nonnull final Descriptors.Descriptor partialRecordDescriptor, + @Nonnull final Value resultValue) { + + this.indexPlan = indexPlan; + this.recordTypeName = recordTypeName; + this.toRecord = indexEntryToPartialRecordConverter; + this.partialRecordDescriptor = partialRecordDescriptor; + this.resultValue = resultValue; + } + + @Nonnull + @Override + @SuppressWarnings("unchecked") + public RecordCursor executePlan(@Nonnull final FDBRecordStoreBase store, + @Nonnull final EvaluationContext context, + @Nullable final byte[] continuation, + @Nonnull final ExecuteProperties executeProperties) { + return indexPlan + .executeEntries(store, context, continuation, executeProperties) + .map(indexEntry -> { + final RecordMetaData metaData = store.getRecordMetaData(); + final RecordType recordType = metaData.getRecordType(recordTypeName); + final Index index = metaData.getIndex(getIndexName()); + return store.coveredIndexQueriedRecord(index, indexEntry, recordType, (M)toRecord.toRecord(partialRecordDescriptor, indexEntry), false); + }) + .map(QueryResult::fromQueriedRecord); + } + + @Nonnull + public RecordQueryIndexPlan getIndexPlan() { + return indexPlan; + } + + public Optional getGroupingValueMaybe() { + final var hasGroupingValue = StreamSupport.stream(resultValue.getChildren().spliterator(), false).count() > 1; + if (hasGroupingValue) { + return Optional.of(resultValue.getChildren().iterator().next()); + } else { + return Optional.empty(); + } + } + + @Nonnull + public String getIndexName() { + return indexPlan.getIndexName(); + } + + @Nonnull + public IndexScanType getScanType() { + return indexPlan.getScanType(); + } + + @Override + public boolean isReverse() { + return indexPlan.isReverse(); + } + + @Override + public boolean hasRecordScan() { + return false; + } + + @Override + public boolean hasFullRecordScan() { + return false; + } + + @Override + public boolean hasIndexScan(@Nonnull String indexName) { + return indexPlan.hasIndexScan(indexName); + } + + @Nonnull + @Override + public Set getUsedIndexes() { + return indexPlan.getUsedIndexes(); + } + + @Override + public int maxCardinality(@Nonnull RecordMetaData metaData) { + return indexPlan.maxCardinality(metaData); + } + + @Override + public boolean isStrictlySorted() { + return indexPlan.isStrictlySorted(); + } + + @Override + public RecordQueryAggregateIndexPlan strictlySorted() { + return new RecordQueryAggregateIndexPlan(indexPlan.strictlySorted(), recordTypeName, toRecord, partialRecordDescriptor, resultValue); + } + + @Nonnull + @Override + public Optional getMatchCandidateMaybe() { + return indexPlan.getMatchCandidateMaybe(); + } + + @Nonnull + @Override + public AvailableFields getAvailableFields() { + return AvailableFields.NO_FIELDS; + } + + @Override + public boolean hasLoadBykeys() { + return false; + } + + @Nonnull + @Override + public Value getResultValue() { + return resultValue; + } + + @Nonnull + @Override + public String toString() { + return "AggregateIndexScan(" + indexPlan + " -> " + toRecord + ")"; + } + + @Nonnull + @Override + public Set getCorrelatedTo() { + final var result = ImmutableSet.builder(); + result.addAll(indexPlan.getCorrelatedTo()); + result.addAll(resultValue.getCorrelatedTo()); + return result.build(); + } + + @Nonnull + @Override + @SuppressWarnings("PMD.CompareObjectsWithEquals") + public RecordQueryAggregateIndexPlan translateCorrelations(@Nonnull final TranslationMap translationMap, @Nonnull final List translatedQuantifiers) { + final var translatedIndexPlan = indexPlan.translateCorrelations(translationMap, translatedQuantifiers); + final var maybeNewResult = resultValue.translateCorrelations(translationMap); + if (translatedIndexPlan != indexPlan || maybeNewResult != resultValue) { + return new RecordQueryAggregateIndexPlan(translatedIndexPlan, recordTypeName, toRecord, partialRecordDescriptor, maybeNewResult); + } + return this; + } + + @Override + @SuppressWarnings("PMD.CompareObjectsWithEquals") + public boolean equalsWithoutChildren(@Nonnull RelationalExpression otherExpression, + @Nonnull final AliasMap equivalencesMap) { + if (this == otherExpression) { + return true; + } + if (getClass() != otherExpression.getClass()) { + return false; + } + final RecordQueryAggregateIndexPlan other = (RecordQueryAggregateIndexPlan) otherExpression; + return indexPlan.structuralEquals(other.indexPlan, equivalencesMap) && + recordTypeName.equals(other.recordTypeName) && + toRecord.equals(other.toRecord) && + resultValue.semanticEquals(other.resultValue, equivalencesMap); + } + + @SuppressWarnings("EqualsWhichDoesntCheckParameterClass") + @Override + public boolean equals(final Object other) { + return structuralEquals(other); + } + + @Override + public int hashCode() { + return structuralHashCode(); + } + + @Override + public int hashCodeWithoutChildren() { + return Objects.hash(indexPlan, recordTypeName, toRecord, resultValue); + } + + @Override + public void logPlanStructure(StoreTimer timer) { + timer.increment(FDBStoreTimer.Counts.PLAN_COVERING_INDEX); + } + + @Override + public int getComplexity() { + return indexPlan.getComplexity(); + } + + @Override + public int planHash(@Nonnull final PlanHashKind hashKind) { + switch (hashKind) { + case FOR_CONTINUATION: + case STRUCTURAL_WITHOUT_LITERALS: + return PlanHashable.objectsPlanHash(hashKind, BASE_HASH, indexPlan, resultValue); + default: + throw new UnsupportedOperationException("Hash kind " + hashKind.name() + " is not supported"); + } + } + + @Nonnull + @Override + public PlannerGraph rewritePlannerGraph(@Nonnull final List childGraphs) { + return indexPlan.createIndexPlannerGraph(this, + NodeInfo.INDEX_SCAN_OPERATOR, + ImmutableList.of(), + ImmutableMap.of()); + } +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryIndexPlan.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryIndexPlan.java index 781df011aa..bb094d5865 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryIndexPlan.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/query/plan/plans/RecordQueryIndexPlan.java @@ -59,8 +59,8 @@ import com.apple.foundationdb.record.query.plan.ScanComparisons; import com.apple.foundationdb.record.query.plan.cascades.AliasMap; import com.apple.foundationdb.record.query.plan.cascades.CorrelationIdentifier; +import com.apple.foundationdb.record.query.plan.cascades.MatchCandidate; import com.apple.foundationdb.record.query.plan.cascades.Quantifier; -import com.apple.foundationdb.record.query.plan.cascades.ScanWithFetchMatchCandidate; import com.apple.foundationdb.record.query.plan.cascades.TranslationMap; import com.apple.foundationdb.record.query.plan.cascades.explain.Attribute; import com.apple.foundationdb.record.query.plan.cascades.explain.NodeInfo; @@ -136,7 +136,7 @@ public class RecordQueryIndexPlan implements RecordQueryPlanWithNoChildren, Reco protected final boolean reverse; protected final boolean strictlySorted; @Nonnull - private final Optional matchCandidateOptional; + private final Optional matchCandidateOptional; @Nonnull private final Type resultType; @@ -162,7 +162,7 @@ public RecordQueryIndexPlan(@Nonnull final String indexName, @Nonnull final FetchIndexRecords fetchIndexRecords, final boolean reverse, final boolean strictlySorted, - @Nonnull final ScanWithFetchMatchCandidate matchCandidate, + @Nonnull final MatchCandidate matchCandidate, @Nonnull final Type.Record resultType) { this(indexName, commonPrimaryKey, scanParameters, indexFetchMethod, fetchIndexRecords, reverse, strictlySorted, Optional.of(matchCandidate), resultType); } @@ -174,7 +174,7 @@ public RecordQueryIndexPlan(@Nonnull final String indexName, @Nonnull final FetchIndexRecords fetchIndexRecords, final boolean reverse, final boolean strictlySorted, - @Nonnull final Optional matchCandidateOptional, + @Nonnull final Optional matchCandidateOptional, @Nonnull final Type resultType) { this.indexName = indexName; this.commonPrimaryKey = commonPrimaryKey; @@ -411,7 +411,7 @@ public boolean isStrictlySorted() { @Nonnull @Override - public Optional getMatchCandidateMaybe() { + public Optional getMatchCandidateMaybe() { return matchCandidateOptional; } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/AggregateValueTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/AggregateValueTest.java index 5aa65b9022..8503693fd9 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/AggregateValueTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/AggregateValueTest.java @@ -78,98 +78,98 @@ void setup() { @Test void testSum() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_I, ofScalar(1)), ints, 21); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_L, ofScalar(1L)), longs, 21L); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_F, ofScalar(1F)), floats, 21F); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_D, ofScalar(1D)), doubles, 21D); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_I, ofScalar(1)), ints, 21); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_L, ofScalar(1L)), longs, 21L); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_F, ofScalar(1F)), floats, 21F); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_D, ofScalar(1D)), doubles, 21D); } @Test void testSumWithNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_I, ofScalar(1)), intsWithNulls, 18); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_L, ofScalar(1L)), longsWithNulls, 18L); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_F, ofScalar(1F)), floatsWithNulls, 18F); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_D, ofScalar(1D)), doublesWithNulls, 18D); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_I, ofScalar(1)), intsWithNulls, 18); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_L, ofScalar(1L)), longsWithNulls, 18L); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_F, ofScalar(1F)), floatsWithNulls, 18F); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_D, ofScalar(1D)), doublesWithNulls, 18D); } @Test void testSumOnlyNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_I, ofScalar(1)), intsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_L, ofScalar(1L)), longsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_F, ofScalar(1F)), floatsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.SUM_D, ofScalar(1D)), doublesOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_I, ofScalar(1)), intsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_L, ofScalar(1L)), longsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_F, ofScalar(1F)), floatsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Sum(PhysicalOperator.SUM_D, ofScalar(1D)), doublesOnlyNull, (Object)null); } @Test void testMin() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_I, ofScalar(1)), ints, 1); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_L, ofScalar(1L)), longs, 1L); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_F, ofScalar(1F)), floats, 1F); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_D, ofScalar(1D)), doubles, 1D); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_I, ofScalar(1)), ints, 1); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_L, ofScalar(1L)), longs, 1L); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_F, ofScalar(1F)), floats, 1F); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_D, ofScalar(1D)), doubles, 1D); } @Test void testMinWithNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_I, ofScalar(1)), intsWithNulls, 1); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_L, ofScalar(1L)), longsWithNulls, 1L); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_F, ofScalar(1F)), floatsWithNulls, 1F); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_D, ofScalar(1D)), doublesWithNulls, 1D); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_I, ofScalar(1)), intsWithNulls, 1); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_L, ofScalar(1L)), longsWithNulls, 1L); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_F, ofScalar(1F)), floatsWithNulls, 1F); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_D, ofScalar(1D)), doublesWithNulls, 1D); } @Test void testMinOnlyNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_I, ofScalar(1)), intsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_L, ofScalar(1L)), longsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_F, ofScalar(1F)), floatsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MIN_D, ofScalar(1D)), doublesOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_I, ofScalar(1)), intsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_L, ofScalar(1L)), longsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_F, ofScalar(1F)), floatsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Min(PhysicalOperator.MIN_D, ofScalar(1D)), doublesOnlyNull, (Object)null); } @Test void testMax() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_I, ofScalar(1)), ints, 6); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_L, ofScalar(1L)), longs, 6L); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_F, ofScalar(1F)), floats, 6F); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_D, ofScalar(1D)), doubles, 6D); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_I, ofScalar(1)), ints, 6); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_L, ofScalar(1L)), longs, 6L); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_F, ofScalar(1F)), floats, 6F); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_D, ofScalar(1D)), doubles, 6D); } @Test void testMaxWithNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_I, ofScalar(1)), intsWithNulls, 6); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_L, ofScalar(1L)), longsWithNulls, 6L); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_F, ofScalar(1F)), floatsWithNulls, 6F); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_D, ofScalar(1D)), doublesWithNulls, 6D); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_I, ofScalar(1)), intsWithNulls, 6); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_L, ofScalar(1L)), longsWithNulls, 6L); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_F, ofScalar(1F)), floatsWithNulls, 6F); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_D, ofScalar(1D)), doublesWithNulls, 6D); } @Test void testMaxOnlyNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_I, ofScalar(1)), intsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_L, ofScalar(1L)), longsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_F, ofScalar(1F)), floatsOnlyNull, (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.MAX_D, ofScalar(1D)), doublesOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_I, ofScalar(1)), intsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_L, ofScalar(1L)), longsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_F, ofScalar(1F)), floatsOnlyNull, (Object)null); + accumulateAndAssert(new NumericAggregationValue.Max(PhysicalOperator.MAX_D, ofScalar(1D)), doublesOnlyNull, (Object)null); } @Test void testAvg() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_I, ofScalar(1)), pairsForAvg(ints), 3.5D); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_L, ofScalar(1L)), pairsForAvg(longs), 3.5D); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_F, ofScalar(1F)), pairsForAvg(floats), 3.5D); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_D, ofScalar(1D)), pairsForAvg(doubles), 3.5D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_I, ofScalar(1)), pairsForAvg(ints), 3.5D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_L, ofScalar(1L)), pairsForAvg(longs), 3.5D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_F, ofScalar(1F)), pairsForAvg(floats), 3.5D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_D, ofScalar(1D)), pairsForAvg(doubles), 3.5D); } @Test void testAvgWithNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_I, ofScalar(1)), pairsForAvg(intsWithNulls), 3.6D); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_L, ofScalar(1L)), pairsForAvg(longsWithNulls), 3.6D); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_F, ofScalar(1F)), pairsForAvg(floatsWithNulls), 3.6D); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_D, ofScalar(1D)), pairsForAvg(doublesWithNulls), 3.6D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_I, ofScalar(1)), pairsForAvg(intsWithNulls), 3.6D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_L, ofScalar(1L)), pairsForAvg(longsWithNulls), 3.6D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_F, ofScalar(1F)), pairsForAvg(floatsWithNulls), 3.6D); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_D, ofScalar(1D)), pairsForAvg(doublesWithNulls), 3.6D); } @Test void testAvgOnlyNulls() { - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_I, ofScalar(1)), pairsForAvg(intsOnlyNull), (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_L, ofScalar(1L)), pairsForAvg(longsOnlyNull), (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_F, ofScalar(1F)), pairsForAvg(floatsOnlyNull), (Object)null); - accumulateAndAssert(new NumericAggregationValue(PhysicalOperator.AVG_D, ofScalar(1D)), pairsForAvg(doublesOnlyNull), (Object)null); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_I, ofScalar(1)), pairsForAvg(intsOnlyNull), (Object)null); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_L, ofScalar(1L)), pairsForAvg(longsOnlyNull), (Object)null); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_F, ofScalar(1F)), pairsForAvg(floatsOnlyNull), (Object)null); + accumulateAndAssert(new NumericAggregationValue.Avg(PhysicalOperator.AVG_D, ofScalar(1D)), pairsForAvg(doublesOnlyNull), (Object)null); } private Object[] pairsForAvg(Object[] objects) { @@ -187,7 +187,7 @@ void testTupleSumCount() { final var recordConstructorValue = RecordConstructorValue.ofUnnamed(ImmutableList.of( - new NumericAggregationValue(PhysicalOperator.SUM_I, ofScalar(1)), + new NumericAggregationValue.Sum(PhysicalOperator.SUM_I, ofScalar(1)), new CountValue(CountValue.PhysicalOperator.COUNT, ofScalar(1)) )); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBInQueryTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBInQueryTest.java index ea14a3d9fc..c3d82bcc53 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBInQueryTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBInQueryTest.java @@ -152,8 +152,8 @@ void testInQueryNoIndex() throws Exception { .where(predicates(valuePredicate(ValueMatchers.fieldValueWithFieldNames("num_value_2"), new Comparisons.ListComparison(Comparisons.Type.IN, ImmutableList.of(0, 2)))))); assertEquals(997592219, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(78097044, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(290624667, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(-167317704, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(45209919, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } assertEquals(67, querySimpleRecordStore(NO_HOOK, plan, EvaluationContext::empty, record -> assertThat(record.getNumValue2(), anyOf(is(0), is(2))), @@ -1563,8 +1563,8 @@ void testInQueryEmptyList() throws Exception { predicatesFilterPlan(selfOrDescendantPlans(scanPlan().and(scanComparisons(unbounded())))) .where(predicates(valuePredicate(ValueMatchers.fieldValueWithFieldNames("num_value_2"), new Comparisons.ListComparison(Comparisons.Type.IN, ImmutableList.of()))))); assertEquals(997518602, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(77994567, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(290552012, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(-167420181, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(45137264, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } assertEquals(0, querySimpleRecordStore(NO_HOOK, plan, EvaluationContext::empty, (rec) -> { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBNestedFieldQueryTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBNestedFieldQueryTest.java index 21dee0d247..f63950c394 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBNestedFieldQueryTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBNestedFieldQueryTest.java @@ -163,8 +163,8 @@ void hierarchical() throws Exception { assertMatchesExactly(plan, scanPlan().where(scanComparisons(range("[[photos],[photos]]")))); assertEquals(1063779424, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(1949336585, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(288971890, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(117315419, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(-1543049276, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } assertEquals(Arrays.asList(12, 11), fetchResultValues(plan, TestRecords3Proto.MyHierarchicalRecord.NUM_VALUE_INDEXED_FIELD_NUMBER, this::openHierarchicalRecordStore, @@ -187,8 +187,8 @@ void hierarchical() throws Exception { assertMatchesExactly(plan, scanPlan().where(scanComparisons(range("{[photos],[photos]}")))); assertEquals(224213141, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(582105050, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(2011769627, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(-1249916116, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(179748461, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } assertEquals(Arrays.asList(12, 11, 112, 111, 1111), fetchResultValues(plan, TestRecords3Proto.MyHierarchicalRecord.NUM_VALUE_INDEXED_FIELD_NUMBER, this::openHierarchicalRecordStore, @@ -972,8 +972,8 @@ void testNestedPrimaryKeyQuery() throws Exception { scanPlan() .where(scanComparisons(range("[[a, 2],[a, 2]]")))); assertEquals(1265534819, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(-1904214744, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(1445521737, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(-17638472, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(-962869287, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } try (FDBRecordContext context = openContext()) { openRecordWithHeader(context, hook); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBRecordStoreQueryTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBRecordStoreQueryTest.java index 4981ab7fa1..9b444eacd4 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBRecordStoreQueryTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBRecordStoreQueryTest.java @@ -266,8 +266,8 @@ void queryWithContinuation() throws Exception { scanPlan().where(scanComparisons(unbounded()))) .where(recordTypes(containsAll(ImmutableSet.of("MySimpleRecord"))))); assertEquals(1623132336, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(1023430112, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(1023430112, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(1846797500, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(1846797500, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } byte[] continuation = null; List retrieved = new ArrayList<>(100); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBReturnedRecordLimitQueryTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBReturnedRecordLimitQueryTest.java index f1becfa065..878b6a4040 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBReturnedRecordLimitQueryTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBReturnedRecordLimitQueryTest.java @@ -181,8 +181,8 @@ void testComplexLimits6() throws Exception { } else { assertThat(plan, typeFilter(contains("MySimpleRecord"), scan(unbounded()))); assertEquals(1623132336, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(1023430112, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(1023430112, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(1846797500, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(1846797500, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } try (FDBRecordContext context = openContext()) { @@ -221,8 +221,8 @@ void testComplexLimits7() throws Exception { } else { assertThat(plan, scan(unbounded())); assertEquals(2, plan.planHash(PlanHashable.PlanHashKind.LEGACY)); - assertEquals(-263175911, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); - assertEquals(-263175911, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); + assertEquals(1564499645, plan.planHash(PlanHashable.PlanHashKind.FOR_CONTINUATION)); + assertEquals(1564499645, plan.planHash(PlanHashable.PlanHashKind.STRUCTURAL_WITHOUT_LITERALS)); } try (FDBRecordContext context = openContext()) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java index 2c3c4cde40..b335fa4781 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/FDBStreamAggregationTest.java @@ -115,7 +115,7 @@ void aggregateOneGroupByOne(final boolean useNestedResult) { final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) .withGroupCriterion("num_value_3_indexed") .build(useNestedResult); @@ -132,7 +132,7 @@ void aggregateOneGroupByNone(final boolean useNestedResult) { final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) .build(useNestedResult); final var result = executePlan(plan); @@ -164,7 +164,7 @@ void aggregateOneGroupByTwo(final boolean useNestedResult) { final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) .withGroupCriterion("num_value_3_indexed") .withGroupCriterion("str_value_indexed") .build(useNestedResult); @@ -182,8 +182,8 @@ void aggregateTwoGroupByTwo(final boolean useNestedResult) { final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.MIN_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Min(NumericAggregationValue.PhysicalOperator.MIN_I, value)) .withGroupCriterion("num_value_3_indexed") .withGroupCriterion("str_value_indexed") .build(useNestedResult); @@ -201,9 +201,9 @@ void aggregateThreeGroupByTwo(final boolean useNestedResult) { final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MySimpleRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.MIN_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.AVG_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Min(NumericAggregationValue.PhysicalOperator.MIN_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Avg(NumericAggregationValue.PhysicalOperator.AVG_I, value)) .withGroupCriterion("num_value_3_indexed") .withGroupCriterion("str_value_indexed") .build(useNestedResult); @@ -221,9 +221,9 @@ void aggregateNoRecords(final boolean useNestedResult) { final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MyOtherRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.MIN_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.AVG_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Min(NumericAggregationValue.PhysicalOperator.MIN_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Avg(NumericAggregationValue.PhysicalOperator.AVG_I, value)) .withGroupCriterion("num_value_3_indexed") .build(useNestedResult); @@ -239,9 +239,9 @@ void aggregateNoRecordsNoGroup(final boolean useNestedResult) { openSimpleRecordStore(context, NO_HOOK); final var plan = new AggregationPlanBuilder(recordStore.getRecordMetaData(), "MyOtherRecord") - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.MIN_I, value)) - .withAggregateValue("num_value_2", value -> new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.AVG_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Min(NumericAggregationValue.PhysicalOperator.MIN_I, value)) + .withAggregateValue("num_value_2", value -> new NumericAggregationValue.Avg(NumericAggregationValue.PhysicalOperator.AVG_I, value)) .build(useNestedResult); final var result = executePlan(plan); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/GroupByTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/GroupByTest.java index ea10dded2a..1afd1aaa00 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/GroupByTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/query/GroupByTest.java @@ -64,8 +64,8 @@ import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.RecordQueryPlanMatchers.mapPlan; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.RecordQueryPlanMatchers.scanComparisons; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.RecordQueryPlanMatchers.streamingAggregationPlan; -import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers.numericAggregationValue; import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers.recordConstructorValue; +import static com.apple.foundationdb.record.query.plan.cascades.matching.structure.ValueMatchers.sumAggregationValue; /** * test suite for {@code GROUP BY} expression planning and execution. @@ -90,7 +90,7 @@ public void testSimpleGroupBy() throws Exception { mapPlan( indexPlan() .where(scanComparisons(range("<,>"))) - )).where(aggregations(recordConstructorValue(exactly(numericAggregationValue("SUM")))) + )).where(aggregations(recordConstructorValue(exactly(sumAggregationValue()))) .and(groupings(ValueMatchers.fieldValueWithFieldNames("select_grouping_cols")))))); } @@ -149,7 +149,7 @@ private GroupExpressionRef constructGroupByPlan() { { // 2.1. construct aggregate expression. final var aggCol = Column.of(Type.Record.Field.unnamedOf(Type.primitiveType(Type.TypeCode.LONG)), - new NumericAggregationValue(NumericAggregationValue.PhysicalOperator.SUM_I, FieldValue.ofFieldNames(qun.getFlowedObjectValue(), ImmutableList.of(scanAlias.getId(), "num_value_3_indexed")))); + new NumericAggregationValue.Sum(NumericAggregationValue.PhysicalOperator.SUM_I, FieldValue.ofFieldNames(qun.getFlowedObjectValue(), ImmutableList.of(scanAlias.getId(), "num_value_3_indexed")))); final var aggregationExpr = RecordConstructorValue.ofColumns(ImmutableList.of(aggCol)); // 2.2. construct grouping columns expression. diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/TransformValueTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/TransformValueTest.java index 9c7d67df7d..0975a63c31 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/TransformValueTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/TransformValueTest.java @@ -33,6 +33,7 @@ import com.google.common.base.Verify; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.tuple.Pair; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -68,30 +69,30 @@ void testTransformTrie() { Assertions.assertNull(transformTrie.getValue()); var childrenMap = transformTrie.getChildrenMap(); Assertions.assertNotNull(childrenMap); - Assertions.assertTrue(childrenMap.containsKey(aField)); + Assertions.assertTrue(childrenMap.containsKey(Pair.of(0, aField.getFieldType()))); Assertions.assertEquals(1, childrenMap.size()); - var aTrie = childrenMap.get(aField); + var aTrie = childrenMap.get(Pair.of(0, aField.getFieldType())); Assertions.assertNull(aTrie.getValue()); childrenMap = aTrie.getChildrenMap(); Assertions.assertNotNull(childrenMap); - Assertions.assertTrue(childrenMap.containsKey(aaField)); - Assertions.assertTrue(childrenMap.containsKey(abField)); + Assertions.assertTrue(childrenMap.containsKey(Pair.of(0, aaField.getFieldType()))); + Assertions.assertTrue(childrenMap.containsKey(Pair.of(1, abField.getFieldType()))); Assertions.assertEquals(2, childrenMap.size()); - var aaTrie = childrenMap.get(aaField); + var aaTrie = childrenMap.get(Pair.of(0, aaField.getFieldType())); Assertions.assertNull(aaTrie.getValue()); childrenMap = aaTrie.getChildrenMap(); Assertions.assertNotNull(childrenMap); - Assertions.assertTrue(childrenMap.containsKey(aaaField)); - Assertions.assertTrue(childrenMap.containsKey(aabField)); + Assertions.assertTrue(childrenMap.containsKey(Pair.of(0, aaaField.getFieldType()))); + Assertions.assertTrue(childrenMap.containsKey(Pair.of(1, aabField.getFieldType()))); Assertions.assertEquals(2, childrenMap.size()); - var aaaTrie = childrenMap.get(aaaField); + var aaaTrie = childrenMap.get(Pair.of(0, aaaField.getFieldType())); Assertions.assertNull(aaaTrie.getChildrenMap()); Assertions.assertNotNull(aaaTrie.getValue()); Assertions.assertEquals(aaaTrie.getValue(), transformMap.get(a_aa_aaa.getFieldPath())); - var aabTrie = childrenMap.get(aabField); + var aabTrie = childrenMap.get(Pair.of(1, aabField.getFieldType())); Assertions.assertEquals(aabTrie.getValue(), transformMap.get(a_aa_aab.getFieldPath())); childrenMap = aTrie.getChildrenMap(); - var abTrie = childrenMap.get(abField); + var abTrie = childrenMap.get(Pair.of(1, abField.getFieldType())); Assertions.assertNull(abTrie.getChildrenMap()); Assertions.assertNotNull(abTrie.getValue()); Assertions.assertEquals(abTrie.getValue(), transformMap.get(a_ab.getFieldPath())); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/view/FieldValueMatcher.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/view/FieldValueMatcher.java index 5d4c533a7c..8ec9556317 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/view/FieldValueMatcher.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/query/plan/cascades/view/FieldValueMatcher.java @@ -47,7 +47,7 @@ public FieldValueMatcher(@Nonnull String fieldName) { @Override protected boolean matchesSafely(final Value element) { return element instanceof FieldValue && - ((FieldValue)element).getFields().equals(fieldPath); + ((FieldValue)element).getFieldPathNames().equals(fieldPath); } @Override