Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43955246
  
    --- Diff: 
flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java
 ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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.flink.optimizer.traversals;
    +
    +import org.apache.flink.api.common.distributions.CommonRangeBoundaries;
    +import org.apache.flink.api.common.operators.UnaryOperatorInformation;
    +import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase;
    +import org.apache.flink.api.common.operators.base.MapOperatorBase;
    +import org.apache.flink.api.common.operators.base.MapPartitionOperatorBase;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.api.java.functions.AssignRangeIndex;
    +import org.apache.flink.api.java.functions.PartitionIDRemoveWrapper;
    +import org.apache.flink.api.java.functions.RangeBoundaryBuilder;
    +import org.apache.flink.api.java.functions.SampleInCoordinator;
    +import org.apache.flink.api.java.functions.SampleInPartition;
    +import org.apache.flink.api.java.sampling.IntermediateSampleData;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TupleTypeInfo;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.optimizer.dag.GroupReduceNode;
    +import org.apache.flink.optimizer.dag.MapNode;
    +import org.apache.flink.optimizer.dag.MapPartitionNode;
    +import org.apache.flink.optimizer.dag.TempMode;
    +import org.apache.flink.optimizer.plan.Channel;
    +import org.apache.flink.optimizer.plan.NamedChannel;
    +import org.apache.flink.optimizer.plan.OptimizedPlan;
    +import org.apache.flink.optimizer.plan.PlanNode;
    +import org.apache.flink.optimizer.plan.SingleInputPlanNode;
    +import org.apache.flink.optimizer.util.Utils;
    +import org.apache.flink.runtime.operators.DriverStrategy;
    +import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
    +import org.apache.flink.util.Visitor;
    +
    +import java.util.ArrayList;
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +public class RangePartitionRewriter implements Visitor<PlanNode> {
    +
    +   OptimizedPlan plan;
    +
    +   public RangePartitionRewriter(OptimizedPlan plan) {
    +           this.plan = plan;
    +   }
    +
    +   @Override
    +   public boolean preVisit(PlanNode visitable) {
    +           return true;
    +   }
    +
    +   @Override
    +   public void postVisit(PlanNode visitable) {
    +           List<Channel> outgoingChannels = 
visitable.getOutgoingChannels();
    +           List<Channel> appendOutgoingChannels = new LinkedList<>();
    +           List<Channel> removeOutgoingChannels = new LinkedList<>();
    +           for (Channel channel : outgoingChannels) {
    +                   ShipStrategyType shipStrategy = 
channel.getShipStrategy();
    +                   if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +                           if (channel.getDataDistribution() == null) {
    +                                   removeOutgoingChannels.add(channel);
    +                                   
appendOutgoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +                           }
    +                   }
    +           }
    +           outgoingChannels.addAll(appendOutgoingChannels);
    +           for (Channel channel : removeOutgoingChannels) {
    +                   outgoingChannels.remove(channel);
    +           }
    +   }
    +
    +   private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +           List<Channel> appendOutgoingChannels = new LinkedList<>();
    +           PlanNode sourceNode = channel.getSource();
    +           PlanNode targetNode = channel.getTarget();
    +           int sourceParallelism = sourceNode.getParallelism();
    +           int targetParallelism = targetNode.getParallelism();
    +           TypeComparatorFactory<?> comparator = 
Utils.getShipComparator(channel, 
this.plan.getOriginalPlan().getExecutionConfig());
    +           // 1. Fixed size sample in each partitions.
    +           long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +           int sampleSize = 20 * targetParallelism;
    +           SampleInPartition sampleInPartition = new 
SampleInPartition(false, sampleSize, seed);
    +           TypeInformation<?> sourceOutputType = 
sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +           TypeInformation<IntermediateSampleData> isdTypeInformation = 
TypeExtractor.getForClass(IntermediateSampleData.class);
    +           UnaryOperatorInformation sipOperatorInformation = new 
UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +           MapPartitionOperatorBase sipOperatorBase = new 
MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in 
partitions");
    +           MapPartitionNode sipNode = new 
MapPartitionNode(sipOperatorBase);
    +           Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +           sipChannel.setShipStrategy(ShipStrategyType.FORWARD, 
channel.getDataExchangeMode());
    +           SingleInputPlanNode sipPlanNode = new 
SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, 
DriverStrategy.MAP_PARTITION);
    +           sipPlanNode.setParallelism(sourceParallelism);
    +           sipChannel.setTarget(sipPlanNode);
    +           appendOutgoingChannels.add(sipChannel);
    +           this.plan.getAllNodes().add(sipPlanNode);
    +
    +           // 2. Fixed size sample in a single coordinator.
    +           SampleInCoordinator sampleInCoordinator = new 
SampleInCoordinator(false, sampleSize, seed);
    +           UnaryOperatorInformation sicOperatorInformation = new 
UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +           GroupReduceOperatorBase sicOperatorBase = new 
GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in 
coordinator");
    +           GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +           Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +           sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, 
channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, 
channel.getDataExchangeMode());
    --- End diff --
    
    The ship strategy can be set to Forward because the DOP of the node is 1. 
That means that all data is sent to the same node (Forward does only imply 
local communication if sender DOP = receiver DOP).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to