korlov42 commented on code in PR #3187: URL: https://github.com/apache/ignite-3/pull/3187#discussion_r1493768157
########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPrunerImpl.java: ########## @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.engine.prepare.pruning; + +import it.unimi.dsi.fastutil.longs.Long2ObjectArrayMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap.Entry; +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.internal.sql.engine.exec.mapping.ColocationGroup; +import org.apache.ignite.internal.sql.engine.exec.mapping.MappedFragment; +import org.apache.ignite.internal.sql.engine.prepare.Fragment; +import org.apache.ignite.internal.sql.engine.prepare.IgniteRelShuttle; +import org.apache.ignite.internal.sql.engine.rel.IgniteReceiver; +import org.apache.ignite.internal.sql.engine.rel.IgniteRel; +import org.apache.ignite.internal.sql.engine.rel.IgniteSender; +import org.apache.ignite.internal.sql.engine.schema.IgniteTable; +import org.jetbrains.annotations.Nullable; + +/** Applies partition pruning. */ +public class PartitionPrunerImpl implements PartitionPruner { + + /** Constructor. */ + public PartitionPrunerImpl() { + + } + + /** {@inheritDoc} */ + @Override + public List<MappedFragment> apply( + List<MappedFragment> mappedFragments, + Object[] dynamicParameters + ) { + List<MappedFragment> updatedFragments = new ArrayList<>(mappedFragments.size()); + Long2ObjectMap<List<String>> newNodesByExchangeId = new Long2ObjectArrayMap<>(); + + // Partition pruning (PP). For each fragment: + // + // 1. Extract PP metadata from each fragment's root in the form of [colo_col1=<val>, ..] (see PartitionPruningMetadataExtractor) + // + // 2. If PP metadata exists then update fragment's colocation group + // to retain partition that are necessary to perform an operator (e.g. for a scan operator such + // partitions only include that ones that can contain data). + // + // Iterate over fragments again to update fragments that receive data from fragments updated at step 2. + // This is accomplished by updating `sourcesByExchangeId`. + // + + PartitionPruningMetadataExtractor extractor = new PartitionPruningMetadataExtractor(); + + for (MappedFragment mappedFragment : mappedFragments) { + // Fragment that contains colocated operators has exactly one colocation group. + // Do not attempt to apply PP to other fragments. Review Comment: is there fundamental limitation to support PP for fragments containing more than a single group? If no, let's file ticket to add support for such fragments later ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningColumns.java: ########## @@ -56,7 +53,7 @@ public List<Int2ObjectMap<RexNode>> columns() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(this); Review Comment: I think, it's better to keep `S.toString` to preserve uniformity. If for some reason you are not satisfied with its output, just use `ppc.columns().toString()` instead ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/NodeWithConsistencyToken.java: ########## @@ -85,6 +84,6 @@ public int hashCode() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(NodeWithConsistencyToken.class, this); + return name + ":" + enlistmentConsistencyToken; Review Comment: see comments above ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningPredicate.java: ########## @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.engine.prepare.pruning; + +import static org.apache.ignite.internal.util.IgniteUtils.newHashMap; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntArraySet; +import it.unimi.dsi.fastutil.ints.IntSet; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import org.apache.calcite.rex.RexDynamicParam; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.calcite.util.TimestampString; +import org.apache.ignite.internal.sql.engine.exec.NodeWithConsistencyToken; +import org.apache.ignite.internal.sql.engine.exec.PartitionWithConsistencyToken; +import org.apache.ignite.internal.sql.engine.exec.mapping.ColocationGroup; +import org.apache.ignite.internal.sql.engine.schema.IgniteTable; +import org.apache.ignite.internal.sql.engine.schema.PartitionCalculator; +import org.apache.ignite.internal.type.NativeType; +import org.apache.ignite.internal.type.NativeTypeSpec; +import org.jetbrains.annotations.Nullable; + +/** + * Partition predicate encapsulates partition pruning logic. + */ +public final class PartitionPruningPredicate { + + private static final ZoneId ZONE_ID_UTC = ZoneId.of("UTC"); + + private final int tablePartitions; + + // TODO: https://issues.apache.org/jira/browse/IGNITE-21543 Remove after is resolved, + // remaining partitions should always be not null. + @Nullable + private final IntSet remainingPartitions; + + /** + * Constructor. + * + * @param table Table. + * @param pruningColumns Columns. + * @param dynamicParameters Values of dynamic parameters. + */ + public PartitionPruningPredicate(IgniteTable table, PartitionPruningColumns pruningColumns, Object[] dynamicParameters) { + this.tablePartitions = table.partitions(); + this.remainingPartitions = computeRemainingPartitions(table, pruningColumns, dynamicParameters); + } + + /** + * Applies partition pruning to the given colocation group. This group should have the same number of assignments as the source table. + * + * @param colocationGroup Colocation group. + * + * @return New colocation group. + */ + public ColocationGroup prunePartitions(ColocationGroup colocationGroup) { + assert tablePartitions == colocationGroup.assignments().size() : "Number of partitions does not match"; + + if (remainingPartitions == null) { + return colocationGroup; + } + + Map<String, List<PartitionWithConsistencyToken>> partitionsPerNode = newHashMap(colocationGroup.nodeNames().size()); + Set<String> newNodes = new HashSet<>(); + + for (String nodeName : colocationGroup.nodeNames()) { + List<PartitionWithConsistencyToken> partsWithConsistencyTokens = new ArrayList<>(); + + for (int p = 0; p < colocationGroup.assignments().size(); p++) { + NodeWithConsistencyToken nodeWithConsistencyToken = colocationGroup.assignments().get(p); + if (!remainingPartitions.contains(p)) { + continue; + } + + if (Objects.equals(nodeName, nodeWithConsistencyToken.name())) { + long t = nodeWithConsistencyToken.enlistmentConsistencyToken(); + + partsWithConsistencyTokens.add(new PartitionWithConsistencyToken(p, t)); + newNodes.add(nodeName); + } + } + + if (!partsWithConsistencyTokens.isEmpty()) { + partitionsPerNode.put(nodeName, partsWithConsistencyTokens); + } + } + + // Keep assignments intact, because they are used by DestinationFactory. + return new ColocationGroup( + colocationGroup.sourceIds(), + List.copyOf(newNodes), + colocationGroup.assignments(), + partitionsPerNode + ); + } + + @Nullable + private static IntSet computeRemainingPartitions( Review Comment: according to our [coding guidelines](https://google.github.io/styleguide/javaguide.html#s4.8.5-annotations) `Type-use annotations appear immediately before the annotated type` ```suggestion private static @Nullable IntSet computeRemainingPartitions( ``` please fix it everywhere in the patch ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPrunerImpl.java: ########## @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.engine.prepare.pruning; + +import it.unimi.dsi.fastutil.longs.Long2ObjectArrayMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap; +import it.unimi.dsi.fastutil.longs.Long2ObjectMap.Entry; +import java.util.ArrayList; +import java.util.List; +import org.apache.ignite.internal.sql.engine.exec.mapping.ColocationGroup; +import org.apache.ignite.internal.sql.engine.exec.mapping.MappedFragment; +import org.apache.ignite.internal.sql.engine.prepare.Fragment; +import org.apache.ignite.internal.sql.engine.prepare.IgniteRelShuttle; +import org.apache.ignite.internal.sql.engine.rel.IgniteReceiver; +import org.apache.ignite.internal.sql.engine.rel.IgniteRel; +import org.apache.ignite.internal.sql.engine.rel.IgniteSender; +import org.apache.ignite.internal.sql.engine.schema.IgniteTable; +import org.jetbrains.annotations.Nullable; + +/** Applies partition pruning. */ +public class PartitionPrunerImpl implements PartitionPruner { + + /** Constructor. */ + public PartitionPrunerImpl() { Review Comment: why do we need to specify default constructor explicitly? ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningMetadata.java: ########## @@ -53,6 +55,6 @@ public Long2ObjectMap<PartitionPruningColumns> data() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(this); + return data.toString(); Review Comment: the same, let's keep `S.toString(this)` ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/PartitionWithConsistencyToken.java: ########## @@ -55,4 +57,29 @@ public int partId() { public long enlistmentConsistencyToken() { return enlistmentConsistencyToken; } + + /** {@inheritDoc} */ + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionWithConsistencyToken that = (PartitionWithConsistencyToken) o; + return partId == that.partId && enlistmentConsistencyToken == that.enlistmentConsistencyToken; + } + + /** {@inheritDoc} */ + @Override + public int hashCode() { + return Objects.hash(partId, enlistmentConsistencyToken); + } + + /** {@inheritDoc} */ + @Override + public String toString() { + return partId + ":" + enlistmentConsistencyToken; Review Comment: see comments above ########## modules/sql-engine/src/test/resources/mapping/test_partition_pruning.test: ########## @@ -0,0 +1,53 @@ +# Pruning from a simple scan +N1 +SELECT * FROM t1_n1n2n3 WHERE id = 0 Review Comment: let's add one more test case to check pruning for collocated join ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningPredicate.java: ########## @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.engine.prepare.pruning; + +import static org.apache.ignite.internal.util.IgniteUtils.newHashMap; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.IntArraySet; +import it.unimi.dsi.fastutil.ints.IntSet; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import org.apache.calcite.rex.RexDynamicParam; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.calcite.util.TimestampString; +import org.apache.ignite.internal.sql.engine.exec.NodeWithConsistencyToken; +import org.apache.ignite.internal.sql.engine.exec.PartitionWithConsistencyToken; +import org.apache.ignite.internal.sql.engine.exec.mapping.ColocationGroup; +import org.apache.ignite.internal.sql.engine.schema.IgniteTable; +import org.apache.ignite.internal.sql.engine.schema.PartitionCalculator; +import org.apache.ignite.internal.type.NativeType; +import org.apache.ignite.internal.type.NativeTypeSpec; +import org.jetbrains.annotations.Nullable; + +/** + * Partition predicate encapsulates partition pruning logic. + */ +public final class PartitionPruningPredicate { + + private static final ZoneId ZONE_ID_UTC = ZoneId.of("UTC"); + + private final int tablePartitions; + + // TODO: https://issues.apache.org/jira/browse/IGNITE-21543 Remove after is resolved, + // remaining partitions should always be not null. + @Nullable + private final IntSet remainingPartitions; + + /** + * Constructor. + * + * @param table Table. + * @param pruningColumns Columns. + * @param dynamicParameters Values of dynamic parameters. + */ + public PartitionPruningPredicate(IgniteTable table, PartitionPruningColumns pruningColumns, Object[] dynamicParameters) { + this.tablePartitions = table.partitions(); + this.remainingPartitions = computeRemainingPartitions(table, pruningColumns, dynamicParameters); + } + + /** + * Applies partition pruning to the given colocation group. This group should have the same number of assignments as the source table. + * + * @param colocationGroup Colocation group. + * + * @return New colocation group. + */ + public ColocationGroup prunePartitions(ColocationGroup colocationGroup) { + assert tablePartitions == colocationGroup.assignments().size() : "Number of partitions does not match"; + + if (remainingPartitions == null) { + return colocationGroup; + } + + Map<String, List<PartitionWithConsistencyToken>> partitionsPerNode = newHashMap(colocationGroup.nodeNames().size()); + Set<String> newNodes = new HashSet<>(); + + for (String nodeName : colocationGroup.nodeNames()) { + List<PartitionWithConsistencyToken> partsWithConsistencyTokens = new ArrayList<>(); + + for (int p = 0; p < colocationGroup.assignments().size(); p++) { + NodeWithConsistencyToken nodeWithConsistencyToken = colocationGroup.assignments().get(p); + if (!remainingPartitions.contains(p)) { + continue; + } + + if (Objects.equals(nodeName, nodeWithConsistencyToken.name())) { + long t = nodeWithConsistencyToken.enlistmentConsistencyToken(); + + partsWithConsistencyTokens.add(new PartitionWithConsistencyToken(p, t)); + newNodes.add(nodeName); + } + } + + if (!partsWithConsistencyTokens.isEmpty()) { + partitionsPerNode.put(nodeName, partsWithConsistencyTokens); + } + } + + // Keep assignments intact, because they are used by DestinationFactory. + return new ColocationGroup( + colocationGroup.sourceIds(), + List.copyOf(newNodes), + colocationGroup.assignments(), + partitionsPerNode + ); + } + + @Nullable + private static IntSet computeRemainingPartitions( + IgniteTable table, + PartitionPruningColumns pruningColumns, + Object[] dynamicParameters + ) { + ImmutableIntList keys = table.distribution().getKeys(); + IntSet remainingPartitions = new IntArraySet(pruningColumns.columns().size()); + PartitionCalculator partitionCalculator = table.partitionCalculator().get(); + + for (Int2ObjectMap<RexNode> columns : pruningColumns.columns()) { + for (int key : keys) { + RexNode node = columns.get(key); + NativeType physicalType = table.descriptor().columnDescriptor(key).physicalType(); + + // TODO: https://issues.apache.org/jira/browse/IGNITE-21543 + // Remove after this issue makes it possible to have CAST('uuid_str' AS UUID) as value. + if (physicalType.spec() == NativeTypeSpec.UUID) { Review Comment: uuid still can be passed as dynamic param, right? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
