zstan commented on code in PR #3187: URL: https://github.com/apache/ignite-3/pull/3187#discussion_r1492408593
########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPrunerImpl.java: ########## @@ -0,0 +1,190 @@ +/* + * 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 eac 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`. + // + + for (MappedFragment mappedFragment : mappedFragments) { + // Fragment that contains colocated operators has exactly one colocation group. + // Do not attempt to apply PP to other fragments. + if (mappedFragment.groups().size() != 1) { + updatedFragments.add(mappedFragment); + continue; + } + + Fragment fragment = mappedFragment.fragment(); + if (fragment.tables().isEmpty()) { + updatedFragments.add(mappedFragment); + continue; + } + + PartitionPruningMetadataExtractor extractor = new PartitionPruningMetadataExtractor(); Review Comment: seems you can move this initialization above **mappedFragments** loop and change a bit : org.apache.ignite.internal.sql.engine.prepare.pruning.PartitionPruningMetadataExtractor#go(IgniteRel rel) { result.clear(); <-- a bit help to GC ? ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/mapping/MappingServiceImpl.java: ########## @@ -147,7 +152,7 @@ private CompletableFuture<List<MappedFragment>> map0(MultiStepPlan multiStepPlan return val; }); - return cacheValue.mappedFragments; + return cacheValue.mappedFragments.thenApply(mappedFragments -> applyPartitionPruning(mappedFragments, parameters)); Review Comment: seems dynamic parameters need to be part of cache key ? ########## .idea/codeStyles/Project.xml: ########## @@ -42,7 +42,7 @@ </value> </option> </AndroidXmlCodeStyleSettings> - <JSCodeStyleSettings> + <JSCodeStyleSettings version="0"> Review Comment: seems you need to revert all this stuff ? ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningPredicate.java: ########## @@ -0,0 +1,200 @@ +/* + * 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.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.HashMap; +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 = new HashMap<>(colocationGroup.nodeNames().size()); Review Comment: org.apache.ignite.internal.util.IgniteUtils#newHashMap ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPrunerImpl.java: ########## @@ -0,0 +1,190 @@ +/* + * 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 eac in the form of [colo_col1=<val>, ..] (see PartitionPruningMetadataExtractor) Review Comment: what does "eac" mean? ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningColumns.java: ########## @@ -56,7 +55,7 @@ public List<Int2ObjectMap<RexNode>> columns() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(this); + return columns.toString(); Review Comment: if you need such a change seems annotation @IgniteToStringInclude need to be removed too ? Also i see that @TestOnly method can be moved directly to tests ? but it`s up to you .. ########## modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPrunerImpl.java: ########## @@ -0,0 +1,190 @@ +/* + * 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( Review Comment: as i can see this class is not holding any values and all methods can be static ? for test cases it can be easily changed too, isn`t it ? -- 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]
