You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by ChengXiangLi <gi...@git.apache.org> on 2015/10/12 09:56:41 UTC

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

GitHub user ChengXiangLi opened a pull request:

    https://github.com/apache/flink/pull/1255

    [FLINK-7] [Runtime] Enable Range Partitioner.

    This PR enable range partitioner for Flink follow the path of existing other partitioners. It depends on the sample operator to random sample data from `DataSet` and build range boundaries based on sampled data. 2 other hints about PR:
    1. Why execute the sample data job in `JobGraphGenerator` instead of `PartitionOperator`?
         i. launch another job in compile time would lead to infinite job submission, because the `DataSink`s has not been cleared during compile time.
         ii. we need the target stage parallelism to decide sample data size, and `TypeSerializer`/`TypeComparator` to serialize/sort sampled data.
    2. Expand the `DataDistribution` API, previous `DataDistribution` take `Key[]` as range boundaries, there is not simple generic way to extract Key from nested object, and `TypeComparator::compareAgainstReference()` is not supported by current comparators. Use `DataSet` elements as the range boundaries make everything much easier, we could use 'TypeComparator::compare()' directly for sort during build `DataDistribution` and selecting channel.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ChengXiangLi/flink rangepartitioner

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1255.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1255
    
----
commit 8a41b18c6c40115d545271039e51ebad44300191
Author: chengxiang li <ch...@intel.com>
Date:   2015-10-12T07:13:38Z

    [FLINK-7] [Runtime] Enable Range Partitioner.

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46556636
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/distributions/SampledDataDistribution.java ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.api.common.distributions;
    +
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * While range partition automatically, Flink sample the source data, and assign range index for each record,
    + * so its data distribution is very simple, just return the bucket index as its boundary.
    + */
    +public class SampledDataDistribution implements DataDistribution {
    +	@Override
    +	public Integer[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
    +		return new Integer[] { bucketNum };
    +	}
    +
    +	@Override
    +	public int getNumberOfFields() {
    +		return -1;
    +	}
    +
    +	@Override
    +	public void write(DataOutputView out) throws IOException {
    --- End diff --
    
    Is there a need to implement `write()` and `read()`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43951220
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/RangeBoundaryBuilder.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.CommonRangeBoundaries;
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +
    +/**
    + * Build RangeBoundaries with input records. First, sort the input records, and then select
    + * the boundaries with same interval.
    + *
    + * @param <T>
    + */
    +public class RangeBoundaryBuilder<T> extends RichMapPartitionFunction<T, RangeBoundaries<T>> {
    +
    +	private int parallelism;
    +	private final TypeComparatorFactory<T> comparatorFactory;
    +
    +	public RangeBoundaryBuilder(TypeComparatorFactory<T> comparator, int parallelism) {
    +		this.comparatorFactory = comparator;
    +		this.parallelism = parallelism;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<T> values, Collector<RangeBoundaries<T>> out) throws Exception {
    +		final TypeComparator<T> comparator = this.comparatorFactory.createComparator();
    +		List<T> sampledData = new ArrayList<>();
    +		for (T value : values) {
    --- End diff --
    
    We can use `TypeComparator.extractKeys()` to extract only the key fields as `Object[]` from the records. `TypeComparator[] TypeComparator.getFlatComparators()` returns comparators for all fields returned by `extractKeys()`. That way we can reduce the size of the distribution, because only keys are stored.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403888
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java ---
    @@ -1141,9 +1141,6 @@ private DistributionPattern connectJobVertices(Channel channel, int inputNumber,
     			final DataDistribution dataDistribution = channel.getDataDistribution();
     			if (dataDistribution != null) {
     				sourceConfig.setOutputDataDistribution(dataDistribution, outputIndex);
    -			} else {
    --- End diff --
    
    Can we set a special marker data distribution that indicates a sampled distribution? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43220047
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/DataSet.java ---
    @@ -1223,6 +1230,51 @@ public long count() throws Exception {
     		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, getType());
     		return new PartitionOperator<T>(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys<T, K>(clean(keyExtractor), this.getType(), keyType), Utils.getCallLocationName());
     	}
    +
    +	/**
    +	 * Range-partitions a DataSet using the specified KeySelector.
    +	 * <p>
    +	 * <b>Important:</b>This operation shuffles the whole DataSet over the network and can take significant amount of time.
    +	 *
    +	 * @param keySelector The KeySelector with which the DataSet is range-partitioned.
    +	 * @return The partitioned DataSet.
    +	 *
    +	 * @see KeySelector
    +	 */
    +	public <K extends Comparable<K>> DataSet<T> partitionByRange(KeySelector<T, K> keySelector) {
    +		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, getType());
    +		String callLocation = Utils.getCallLocationName();
    +
    +		// Extract key from input element by keySelector.
    +		KeyExtractorMapper<T, K> keyExtractorMapper = new KeyExtractorMapper<T, K>(keySelector);
    --- End diff --
    
    In `connectJobVertices()`, the source and target `JobVertex` has already been created, so just rewrite the `OptimizedPlan` does not work here, unless we generate related `JobVetex` manually which is what i try to avoid, we need to rewrite the `OptimizedPlan` before the creating of related `JobVertex`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-165652344
  
    Hi, @fhueske , For the partition part, i think it's normal that `RangePartition` is slower than `HashParition`, as you've mentioned, `RangePartition` introduce more overhead. The most difference between `HashParition` and `RangePartition` is that, `HashParition` is key-wise partition(elements with same key would shuffled to same target), and `RangePartition` is key-wise and partition-wise partition(the partition is in order as well), so for global order, we can sort in parallel after `RangePartition`, that's what we can benefit from `RangePartition`.
    On the other side, it's still make sense to improve `RangePartition` performance, although i don't think increasing the sample size would help here. Based on my previous calculation and test, `parallelism * 20` is enough to generate well-proportioned partitions. Do you find there is data skew in any partition after `RangePartition`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43954837
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		appendOutgoingChannels.add(ariChannel);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +
    +		channel.setSource(ariPlanNode);
    +		ariPlanNode.addOutgoingChannel(channel);
    +
    +		NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		PartitionIDRemoveWrapper partitionIDRemoveWrapper = new PartitionIDRemoveWrapper();
    --- End diff --
    
    The `PartitionIDRemoveWrapper` must be injected before the operator that should receive the partitioned input because that operator does not expect the `Tuple2`-wrapped input and will fail otherwise. 
    So we need to append the `PartitionIDRemoveWrapper` to the `PartitionIDAssigner` with a Range-partition channel. The idAssigner has the same DOP as the source and the idRemove has the same DOP as the target node. Then the target node is appended to the idRemove node with a forward channel. The range partitioning will happen between the idAssigner and idRemover but will be preserved due to the forward channel and the equal DOP of idRemover and target node.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43952078
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.List;
    +
    +/**
    + * This mapPartition function require a DataSet with RangeBoundaries as broadcast input, it
    + * generate Tuple2 which includes range index and record itself as output.
    + *
    + * @param <IN> The original data type.
    + */
    +public class AssignRangeIndex<IN> extends RichMapPartitionFunction<IN, Tuple2<Integer, IN>> {
    +
    +	private TypeComparatorFactory<IN> typeComparator;
    +
    +	public AssignRangeIndex(TypeComparatorFactory<IN> typeComparator) {
    +		this.typeComparator = typeComparator;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<IN> values, Collector<Tuple2<Integer, IN>> out) throws Exception {
    +
    +		List<Object> broadcastVariable = getRuntimeContext().getBroadcastVariable("RangeBoundaries");
    +		if (broadcastVariable == null || broadcastVariable.size() != 1) {
    +			throw new RuntimeException("AssignRangePartition require a single RangeBoundaries as broadcast input.");
    +		}
    +		RangeBoundaries<IN> rangeBoundaries = (RangeBoundaries<IN>) broadcastVariable.get(0);
    +
    +		for (IN record : values) {
    +			out.collect(new Tuple2(rangeBoundaries.getRangeIndex(record, typeComparator.createComparator()), record));
    --- End diff --
    
    It is more efficient to create only one instance of `Tuple2<Integer, IN>` and reuse it. Once `Collector.collect(t)` is called, the content of `t` has been serialized and `t` can be reused.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-163533029
  
    Thanks, @fhueske  . I've updated the PR to use `PARTITION_CUSTOM`. The fail test should be unrelated.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-161725286
  
    Hi @ChengXiangLi, sorry for long time until until I pick the PR up again.
    It looks really good, just added a few minor comments and the one about removing the target node. 
    This is only a problem, if the PartitionOperator actually has LocalStrategy, but then it would fail IMO.
    
    I'll try to run the PR on a cluster in the next days and see how it works in a distributed setting.
    
    Thanks, Fabian


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-149203986
  
    I think the right way to implement the range partitioner feature is to inject the sampler during the JobGraph generation phase after the OptimizedPlan has been created. This would allow to transparently handle range partitioning during optimization and make it a completely runtime-related feature.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43985314
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		appendOutgoingChannels.add(ariChannel);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +
    +		channel.setSource(ariPlanNode);
    +		ariPlanNode.addOutgoingChannel(channel);
    +
    +		NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		PartitionIDRemoveWrapper partitionIDRemoveWrapper = new PartitionIDRemoveWrapper();
    --- End diff --
    
    I'm afraid not, here is the currently DAG(in `PlanNode` level) rewrite logic:
    ![image](https://cloud.githubusercontent.com/assets/4425616/10962882/95cb56a2-83d6-11e5-823f-6d7ee7e23a15.png)



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43008448
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java ---
    @@ -1138,12 +1138,13 @@ private DistributionPattern connectJobVertices(Channel channel, int inputNumber,
     		
     		if (channel.getShipStrategy() == ShipStrategyType.PARTITION_RANGE) {
     			
    -			final DataDistribution dataDistribution = channel.getDataDistribution();
    +			DataDistribution dataDistribution = channel.getDataDistribution();
     			if (dataDistribution != null) {
     				sourceConfig.setOutputDataDistribution(dataDistribution, outputIndex);
     			} else {
    -				throw new RuntimeException("Range partitioning requires data distribution");
    -				// TODO: inject code and configuration for automatic histogram generation
    --- End diff --
    
    As the TODO comment says, here should go the sampling and distribution building code :-)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403426
  
    --- Diff: docs/apis/dataset_transformations.md ---
    @@ -1981,6 +1981,41 @@ Not supported.
     </div>
     </div>
     
    +### Range-Partition
    +
    +Range-partitions a DataSet on a given key.
    +Keys can be specified as key expressions or field position keys (see [Reduce examples](#reduce-on-grouped-dataset) for how to specify keys).
    --- End diff --
    
    should be "position keys, expression keys, and key selector functions".


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43956068
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    --- End diff --
    
    We need to be very careful at this point! We cannot add two channels to the same node (`sourceNode`) that pipeline the data, where one of the successors has to wait for the other to complete. In our case this is because of the broadcast set which implies that the boundaries are first computed and broadcasted before the idAssigner can start to process the data. Since we need to see all data from `sourceNode` in order to build the boundaries but have to wait for the boundaries to be able process the data from `sourceNode`, this will lead to a deadlock in the data flow.
    
    I have to admit, I am not sure how we can break the pipeline. Previously that was achieved by the `TempMode` but recently the `DataExchangeMode` was introduced and I am not quite sure write how both modes are different from each other and how they interact. I will ask another committer to comment on this.
     
    For now, I only want to raise a flag and get your attention on this issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-165404528
  
    Thanks @uce! I extended the `RangePartitionRewriter` to cover that case and will include the changes when I merge the PR.
    
    I also found that no combiners are inserted if the input of a reduce or combinable groupreduce is explicitly partitioned (FLINK-3179). This is not directly related to this PR (because it also affects `partitionByHash` and `partitionCustom`), but should be fixed soon. 
    
    I am trying the PR right now on a cluster setup. Thanks @ChengXiangLi for your patience with this one! 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44263105
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    --- End diff --
    
    OK, so `DataExchangeMode` is a new concept and should replace `TempMode`. However, the transition is not complete yet. The difference is that `TempMode` temps data within the operator and `DataExchangeMode` temps in the network stack. So, the correct channel configuration is to set the `TempMode` for both source-node outgoing channels to `TempMode.NONE` and the `DataExchangeMode` for the sampling out-channel to `DataExchangeMode.PIPELINED` and for the assignment out-channel to `DataExchangeMode.BATCH`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43088529
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/DataSet.java ---
    @@ -1223,6 +1230,51 @@ public long count() throws Exception {
     		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, getType());
     		return new PartitionOperator<T>(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys<T, K>(clean(keyExtractor), this.getType(), keyType), Utils.getCallLocationName());
     	}
    +
    +	/**
    +	 * Range-partitions a DataSet using the specified KeySelector.
    +	 * <p>
    +	 * <b>Important:</b>This operation shuffles the whole DataSet over the network and can take significant amount of time.
    +	 *
    +	 * @param keySelector The KeySelector with which the DataSet is range-partitioned.
    +	 * @return The partitioned DataSet.
    +	 *
    +	 * @see KeySelector
    +	 */
    +	public <K extends Comparable<K>> DataSet<T> partitionByRange(KeySelector<T, K> keySelector) {
    +		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, getType());
    +		String callLocation = Utils.getCallLocationName();
    +
    +		// Extract key from input element by keySelector.
    +		KeyExtractorMapper<T, K> keyExtractorMapper = new KeyExtractorMapper<T, K>(keySelector);
    --- End diff --
    
    Yes, it's very low level job abstraction, not sure whether i can get everything required, i didn't find any precedent of this, but it deserve a try. 
    Besides, everything required(ship strategy type / target parallelism) is available at `OptimizedPlan` level, so i think it should be better to inject the sampling and partitionID assignment code by modification of `OptimizedPlan` at the  begining of `JobGraphGenerator::compileJobGraph` instead of the previous inject point as the next comment mentioned. The previous inject point is at the middle stage of building `JobGraph`, and require rewriting of `JobGraph`,  even lower level than `OptimizedPlan`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44405116
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    --- End diff --
    
    `ShipStrategy` can be `FORWARD`, because parallelism is 1.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43228263
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/DataSet.java ---
    @@ -1223,6 +1230,51 @@ public long count() throws Exception {
     		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, getType());
     		return new PartitionOperator<T>(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys<T, K>(clean(keyExtractor), this.getType(), keyType), Utils.getCallLocationName());
     	}
    +
    +	/**
    +	 * Range-partitions a DataSet using the specified KeySelector.
    +	 * <p>
    +	 * <b>Important:</b>This operation shuffles the whole DataSet over the network and can take significant amount of time.
    +	 *
    +	 * @param keySelector The KeySelector with which the DataSet is range-partitioned.
    +	 * @return The partitioned DataSet.
    +	 *
    +	 * @see KeySelector
    +	 */
    +	public <K extends Comparable<K>> DataSet<T> partitionByRange(KeySelector<T, K> keySelector) {
    +		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, getType());
    +		String callLocation = Utils.getCallLocationName();
    +
    +		// Extract key from input element by keySelector.
    +		KeyExtractorMapper<T, K> keyExtractorMapper = new KeyExtractorMapper<T, K>(keySelector);
    --- End diff --
    
    Sounds good to me.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44407007
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, channel.getTempMode());
    +		partChannel.setDataExchangeMode(channel.getDataExchangeMode());
    +		final FieldList keys = new FieldList(0);
    +		final boolean[] sortDirection = { true };
    +		partChannel.setShipStrategy(channel.getShipStrategy(), keys, sortDirection, channel.getPartitioner(), channel.getDataExchangeMode());
    +		ariPlanNode.addOutgoingChannel(channel);
    +		partChannel.setLocalStrategy(channel.getLocalStrategy(), keys, sortDirection);
    +		this.plan.getAllNodes().remove(targetNode);
    --- End diff --
    
    I was wrong about removing the target node. We have to keep it, in case it has a local strategy set.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43004309
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.DataDistribution;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * This mapPartition function require a DataSet with DataDistribution as broadcast input, it read
    + * target parallelism from parameter, build partition boundaries with input DataDistribution, then
    + * compute the range index for each record.
    + *
    + * @param <IN> The original data type.
    + * @param <K> The key type.
    + */
    +public class AssignRangeIndex<IN, K extends Comparable<K>>
    +	extends RichMapPartitionFunction<Tuple2<K, IN>, Tuple2<Integer, IN>> {
    +
    +	private List<K> partitionBoundaries;
    +	private int numberChannels;
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    +		this.numberChannels = parameters.getInteger("TargetParallelism", 1);
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<Tuple2<K, IN>> values, Collector<Tuple2<Integer, IN>> out) throws Exception {
    +
    +		List<Object> broadcastVariable = getRuntimeContext().getBroadcastVariable("DataDistribution");
    +		if (broadcastVariable == null || broadcastVariable.size() != 1) {
    --- End diff --
    
    Nevermind, I thought you were using a MapFunction, but its a MapPartitionFunction. So this is only done once.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43956876
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java ---
    @@ -148,9 +147,6 @@ else if (pKeys instanceof Keys.SelectorFunctionKeys) {
     			}
     			
     		} 
    -		else if (pMethod == PartitionMethod.RANGE) {
    --- End diff --
    
    We need to add a check here to ensure that `pKeys.getKeyType().isSortKeyType() == true`. The reason for this is that some keys can be used for grouping but not for sorting because their order is not well defined. For example POJO types have multiple fields on which they can be grouped, but the order of their fields is not defined so the order of POJO objects isn't defined either.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-161251178
  
    What is your timeline on this? I would like to use it for FLINK-2946.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43986746
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java ---
    @@ -148,9 +147,6 @@ else if (pKeys instanceof Keys.SelectorFunctionKeys) {
     			}
     			
     		} 
    -		else if (pMethod == PartitionMethod.RANGE) {
    --- End diff --
    
    Do you mean that if `pKeys.getKeyType().isSortKeyType() == false` and it's range partitioner, we should throw a exception?
    If yes, I think it should be ok, for the range partitioner, it does not require the range to be sorted in meaningful way, it just try to make each target partition have roughly same size data. As long as we can compare key fields, we should be able to use range partitioner. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-163873769
  
    Thanks for the update. Looks good. Will try it once more on a cluster.
    
    @uce, @StephanEwen , we inject a `DataExchangeMode.BATCH` for range partitioning (`RangePartitionRewriter`, line 168). IIRC, there are some implication wrt. to iterations. Will that work?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46566247
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +	final static long SEED = org.apache.flink.api.java.Utils.RNG.nextLong();
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    --- End diff --
    
    This will continue in case of a Record type and cause silently wrong results, no? Can we do this check earlier, e.g., in the `PartitionOperator` and throw an `UnsupportedOperationException`? We should also add a note to FLINK-1278 in order to remember to remove the exception later when FLINK-1278 is resolved.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-153905286
  
    Hi @ChengXiangLi, sorry for the long reviewing delay.
    The PR is definitely moving in the right direction! I added a couple of comments in-line. Please let me know if you have questions or would like to discuss some of the comments.
    
    We will also need a few tests that check the OptimizedPlan that is returned by the optimizer. We have a few tests in place that check for example that correct execution strategies are chosen by the optimizer. Since we manually rewrite the plan, it would be good to have tests that verify our changes, for example for cases where an operator has multiple outputs of which only one or all require range partitioning. Please have a look at `PartitionOperatorTest` and other tests in the `flink-optimizer` module.
    
    Best, Fabian


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44408781
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java ---
    @@ -231,4 +268,44 @@ private int murmurHash(int k) {
     			throw new RuntimeException("Error while calling custom partitioner.", t);
     		}
     	}
    +
    +	private final int compareRecordAndBoundary(T record, Object[] boundary) {
    +		TypeComparator[] flatComparators = this.comparator.getFlatComparators();
    --- End diff --
    
    can we cache the `flatComparators` and `keys` to avoid many object instantiations?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46564786
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +	final static long SEED = org.apache.flink.api.java.Utils.RNG.nextLong();
    --- End diff --
    
    This seed won't be deterministic, because `RNG` is initialized as `new java.util.Random()` which uses mixes `System.nanoTime()` in its seed. I'd prefer just some static `long` value.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44406470
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, channel.getTempMode());
    +		partChannel.setDataExchangeMode(channel.getDataExchangeMode());
    +		final FieldList keys = new FieldList(0);
    +		final boolean[] sortDirection = { true };
    +		partChannel.setShipStrategy(channel.getShipStrategy(), keys, sortDirection, channel.getPartitioner(), channel.getDataExchangeMode());
    +		ariPlanNode.addOutgoingChannel(channel);
    +		partChannel.setLocalStrategy(channel.getLocalStrategy(), keys, sortDirection);
    --- End diff --
    
    Do not set the local strategy. This should be done in the channel that forwards to the target node.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-155403470
  
    Thanks, @fhueske , I've updated the PR and verified in test environment that the deadlock issue has fixed on latest code.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43952751
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java ---
    @@ -121,6 +121,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
     			Channel toReducer = new Channel(combiner);
     			toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
     									in.getShipStrategySortOrder(), in.getDataExchangeMode());
    +			if (in.getShipStrategy() == ShipStrategyType.PARTITION_RANGE) {
    --- End diff --
    
    Why this change?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-165732606
  
    Range partitioning serves two purposes:
    
    1. producing fully sorted results.
    2. evenly balancing the load in case of skewed key distributions.
    
    Producing sorted results is working fine. However, producing balanced partitions does not seem to work so well. Looking at the numbers I posted, the partitions produced by the range partitioner are less balanced than the hash partitioned ones (records-in / bytes-in). The difference is not huge, but still range partitioning should be able to do better than hash partitioning.
    
    I proposed to increase the sample size, because this should improve the accuracy of the histogram without having a (measurable) impact on the performance. If we pay so much time to generate a histogram, the histogram should be accurate enough to result in balanced partitions.
    
    Can you explain how you calculated the sample size of `parallelism * 20`? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403668
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/CoGroupDescriptor.java ---
    @@ -152,8 +152,7 @@ else if(produced1.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED &&
     				produced2.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED) {
     
     			// both are range partitioned, check that partitioning fields are equivalently chosen
    --- End diff --
    
    Please adapt the comment


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43951890
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/RangeBoundaryBuilder.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.CommonRangeBoundaries;
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +
    +/**
    + * Build RangeBoundaries with input records. First, sort the input records, and then select
    + * the boundaries with same interval.
    + *
    + * @param <T>
    + */
    +public class RangeBoundaryBuilder<T> extends RichMapPartitionFunction<T, RangeBoundaries<T>> {
    +
    +	private int parallelism;
    +	private final TypeComparatorFactory<T> comparatorFactory;
    +
    +	public RangeBoundaryBuilder(TypeComparatorFactory<T> comparator, int parallelism) {
    +		this.comparatorFactory = comparator;
    +		this.parallelism = parallelism;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<T> values, Collector<RangeBoundaries<T>> out) throws Exception {
    +		final TypeComparator<T> comparator = this.comparatorFactory.createComparator();
    +		List<T> sampledData = new ArrayList<>();
    +		for (T value : values) {
    +			sampledData.add(value);
    +		}
    +		Collections.sort(sampledData, new Comparator<T>() {
    +			@Override
    +			public int compare(T first, T second) {
    +				return comparator.compare(first, second);
    +			}
    +		});
    +
    +		List<T> boundaries = new ArrayList<>();
    +		double avgRange = sampledData.size() / (double) parallelism;
    +		for (int i = 1; i < parallelism; i++) {
    +			boundaries.add(sampledData.get((int) (i * avgRange)));
    +		}
    +
    +		RangeBoundaries<T> rangeBoundaries = new CommonRangeBoundaries<>(boundaries);
    --- End diff --
    
    It might be better to send the boundaries as individual records instead of a single big record.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43954245
  
    --- 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();
    --- End diff --
    
    I would use deterministic value here. This will make the behavior more deterministic and testing the feature easier.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43008620
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java ---
    @@ -1264,11 +1264,10 @@ public static void logAndThrowException(Exception ex, AbstractInvokable parent)
     					oe = new OutputEmitter<T>(strategy);
     				}
     				else {
    -					final DataDistribution dataDist = config.getOutputDataDistribution(i, cl);
    --- End diff --
    
    Can we keep the interface with the `DataDistribution` and simply inject a simple int distribution (1,2,3,...,n)?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43982452
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java ---
    @@ -135,7 +116,7 @@ public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, Pa
     		case PARTITION_CUSTOM:
     			return customPartition(record.getInstance(), numberOfChannels);
     		case PARTITION_RANGE:
    -			return rangePartition(record.getInstance(), numberOfChannels);
    +			return rangePartition((Tuple2<Integer, ?>)record.getInstance(), numberOfChannels);
    --- End diff --
    
    I got your point, @fhueske , for the reason i implement in current way:
    1. This PR only includes the part of range partition based on automatic sampling, to only execute source node(before range partition) once, we send the sampled distribution data by broadcast, so take the partition id part of the shuffle data. In this case, the type of 'OutputEmitter' would definitely be `Tuple2<Integer, ?>`.
    2. Actually there would be a following up task to support range partition by given `DataDistribution`. Current `DataDistribution.getBucketBoundary()` return `Key<?>[]`, while `TypeComparator.extractKeys()` return `Object[]` which does match each other. Besides, there are exists implementations based on `DataDistribution` and `GeneralDataSinkBase.setGlobalOrder()`, and a bunch of related logic. 
    Involved `DataDistribution` in `OutputEmitter` would basically bring case 2 into this PR, which i plan to implement as a followup work. Do you prefer that we do both together?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-151177199
  
    Hi, I left a few comments inside. 
    I don't think we need the special user code to extract keys. The comparators provide this functionality and can be generated within the JobGraphGenerator.
    
    Let me know what you think.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46555497
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/distributions/CommonRangeBoundaries.java ---
    @@ -0,0 +1,76 @@
    +/*
    + * 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.api.common.distributions;
    +
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +
    +public class CommonRangeBoundaries<T> implements RangeBoundaries<T> {
    +	final private TypeComparator<T> typeComparator;
    +	final private Object[][] boundaries;
    +	final private TypeComparator[] flatComparators;
    +
    +	public CommonRangeBoundaries(TypeComparator<T> typeComparators, Object[][] boundaries) {
    +		this.typeComparator = typeComparators;
    +		this.flatComparators = typeComparators.getFlatComparators();
    +		this.boundaries = boundaries;
    +	}
    +
    +	@Override
    +	public int getRangeIndex(T record) {
    +		return binarySearch(record);
    +	}
    +
    +	// Search the range index of input record.
    +	private int binarySearch(T record) {
    +		int low = 0;
    +		int high = this.boundaries.length - 1;
    +		Object[] keys = new Object[flatComparators.length];
    --- End diff --
    
    We can initialize the `keys` array in the constructor and reuse it across invocations of `binarySearch`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44405408
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    --- End diff --
    
    DataExchangeMode should be `PIPELINED`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-155431001
  
    Thanks for the fast update @ChengXiangLi! I added a few comments, mostly regarding the shipping strategies, data exchange modes, and object instantiations. I think, we're almost there. After the next update, I'll play around with range partitioning on a cluster and see how it performs. We might want to add a few more tests, esp. to ensure the plan rewriting performs well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43005310
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.DataDistribution;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * This mapPartition function require a DataSet with DataDistribution as broadcast input, it read
    + * target parallelism from parameter, build partition boundaries with input DataDistribution, then
    + * compute the range index for each record.
    + *
    + * @param <IN> The original data type.
    + * @param <K> The key type.
    + */
    +public class AssignRangeIndex<IN, K extends Comparable<K>>
    +	extends RichMapPartitionFunction<Tuple2<K, IN>, Tuple2<Integer, IN>> {
    +
    +	private List<K> partitionBoundaries;
    +	private int numberChannels;
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    +		this.numberChannels = parameters.getInteger("TargetParallelism", 1);
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<Tuple2<K, IN>> values, Collector<Tuple2<Integer, IN>> out) throws Exception {
    +
    +		List<Object> broadcastVariable = getRuntimeContext().getBroadcastVariable("DataDistribution");
    +		if (broadcastVariable == null || broadcastVariable.size() != 1) {
    +			throw new RuntimeException("AssignRangePartition require a single DataDistribution as broadcast input.");
    +		}
    +		DataDistribution<K> dataDistribution = (DataDistribution<K>) broadcastVariable.get(0);
    +
    +		partitionBoundaries = new ArrayList<>(numberChannels);
    +		for (int i=0; i<numberChannels - 1; i++) {
    --- End diff --
    
    Each Assigner will independently initialize the boundaries and have its own copy of it. It would be better if you could do that only once in a single operator (I believe you had an AllReduce, i.e., a reduce without groupBy() before) and broadcast the result of this operator. The benefit is that the boundaries are only built once and all tasks on a TaskManager share the same broadcast variable, i.e., there are not multiple copies of the boundaries.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43953613
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java ---
    @@ -135,7 +116,7 @@ public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, Pa
     		case PARTITION_CUSTOM:
     			return customPartition(record.getInstance(), numberOfChannels);
     		case PARTITION_RANGE:
    -			return rangePartition(record.getInstance(), numberOfChannels);
    +			return rangePartition((Tuple2<Integer, ?>)record.getInstance(), numberOfChannels);
    --- End diff --
    
    I would not make the `OutputEmitter` type specific, i.e., expect a `Tuple2<Integer, ?>`. Instead I would use a Integer DataDistribution, and a `TypeComparator<Tuple2<Integer,?>` that compares on field `f0`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46642181
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +	final static long SEED = org.apache.flink.api.java.Utils.RNG.nextLong();
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    --- End diff --
    
    No silently wrong result, for `Record` type data, range partition is handled in `RecordOutputEmitter`, it does not need rewriting. And `GenericDataSinkBase.setRangePartitioned()` would set range partition as well, so we can not  just move this verification to `PartitionOperator`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44522180
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, channel.getTempMode());
    +		partChannel.setDataExchangeMode(channel.getDataExchangeMode());
    +		final FieldList keys = new FieldList(0);
    +		final boolean[] sortDirection = { true };
    +		partChannel.setShipStrategy(channel.getShipStrategy(), keys, sortDirection, channel.getPartitioner(), channel.getDataExchangeMode());
    +		ariPlanNode.addOutgoingChannel(channel);
    +		partChannel.setLocalStrategy(channel.getLocalStrategy(), keys, sortDirection);
    +		this.plan.getAllNodes().remove(targetNode);
    --- End diff --
    
    `Channel.setLocalStrategy()` happens during the period of `OptimizedNode.getAlternativePlans()` which is before the `RangePartitionRewriter` optimization,  so we have to set `LocalStrategy` even while keep the partition node and reuse the previous channel, as the key information has changed already. It seems removing partition node works well now, do you have some good reason why we should not implement in this way?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43953917
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java ---
    @@ -71,6 +71,30 @@ public void testHashPartitionByKeyField() throws Exception {
     	}
     
     	@Test
    +	public void testRangePartitionByKeyField() throws Exception {
    +		/*
    +		 * Test hash partition by key field
    +		 */
    +
    +		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +
    +		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
    +		DataSet<Long> uniqLongs = ds
    +			.partitionByRange(1)
    +			.mapPartition(new UniqueLongMapper());
    +		List<Long> result = uniqLongs.collect();
    +
    +		String expected = "1\n" +
    +			"2\n" +
    +			"3\n" +
    +			"4\n" +
    +			"5\n" +
    +			"6\n";
    +
    +		compareResultAsText(result, expected);
    +	}
    +
    --- End diff --
    
    Please add another test with composite partition keys (`.partitionByRange(0,2)`).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43004130
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.DataDistribution;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/**
    + * This mapPartition function require a DataSet with DataDistribution as broadcast input, it read
    + * target parallelism from parameter, build partition boundaries with input DataDistribution, then
    + * compute the range index for each record.
    + *
    + * @param <IN> The original data type.
    + * @param <K> The key type.
    + */
    +public class AssignRangeIndex<IN, K extends Comparable<K>>
    +	extends RichMapPartitionFunction<Tuple2<K, IN>, Tuple2<Integer, IN>> {
    +
    +	private List<K> partitionBoundaries;
    +	private int numberChannels;
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    +		this.numberChannels = parameters.getInteger("TargetParallelism", 1);
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<Tuple2<K, IN>> values, Collector<Tuple2<Integer, IN>> out) throws Exception {
    +
    +		List<Object> broadcastVariable = getRuntimeContext().getBroadcastVariable("DataDistribution");
    +		if (broadcastVariable == null || broadcastVariable.size() != 1) {
    --- End diff --
    
    You can move the broadcast variable initialization into the open method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-155052860
  
    I just realized we also need to make the optimizer aware that two auto-sampled range partitionings are not equivalent. These checks the to be added to the `areCompatible()` methods of the `AbstractJoinDescriptor` and `CoGroupDescriptor` to avoid incorrect joins and coGroups.
    
    For now, it is sufficient to simply return `false` if a range partitioning is observed for a join or coGroup. Later, we want to add a property to the `GlobalProperties` that identifies the data distribution which is used for the range partitioning.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403726
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java ---
    @@ -121,6 +121,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
     			Channel toReducer = new Channel(combiner);
     			toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
     									in.getShipStrategySortOrder(), in.getDataExchangeMode());
    +			if (in.getShipStrategy() == ShipStrategyType.PARTITION_RANGE) {
    --- End diff --
    
    Can you explain why this change is necessary? Thanks


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-162608149
  
    Hi @ChengXiangLi, I tried the PR locally on my machine and it was working quite well.
    I found a few minor issues:
    - You don't need to set keys and sort order for `FORWARD` shipping strategies in `RangePartitionRewriter` line 118
    - You set the `DataExchangeMode` twice in line 163 and 166 or `RangePartitionRewriter`
    - We need to set costs and properties for the PlanNodes that we create. Otherwise `ExecutionEnvironment.getExecutionPlan()` fails with a NPE. Costs can be 0, IMO but we should set the correct physical properties because these are visualized and might confuse users.
    - The injected plan node should have good names that indicate that they belong to the range partitioner. I suggest:
      - `sipPlanNode`: "RangePartition: LocalSample"
      - `sicPlanNode`: "RangePartition: GlobalSample"
      - `rbPlanNode`: "RangePartition: Histogram"
      - `ariPlanNode`: "RangePartition: Partition"
      - `prPlanNode`: "RangePartition: Partition" (IMO its fine to have the same name here. Too much detail such as assign id and unwrap might just confuse)
    
    What do you think?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46556467
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/distributions/SampledDataDistribution.java ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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.api.common.distributions;
    +
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * While range partition automatically, Flink sample the source data, and assign range index for each record,
    + * so its data distribution is very simple, just return the bucket index as its boundary.
    + */
    +public class SampledDataDistribution implements DataDistribution {
    +	@Override
    +	public Integer[] getBucketBoundary(int bucketNum, int totalNumBuckets) {
    +		return new Integer[] { bucketNum };
    +	}
    +
    +	@Override
    +	public int getNumberOfFields() {
    +		return -1;
    --- End diff --
    
    `getNumberOfFields()` should return `1` (instead of `-1`) according to the JavaDocs of `DataDistribution`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43952628
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/PartitionNode.java ---
    @@ -119,7 +121,14 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
     				rgps.setCustomPartitioned(this.keys, this.customPartitioner);
     				break;
     			case RANGE:
    -				throw new UnsupportedOperationException("Not yet supported");
    +				// Initiate Ordering as ascending here as no order parameter in API level,
    +				// we could revisit this while order is required in future optimization.
    +				Ordering ordering = new Ordering();
    +				for (int filed : this.keys) {
    --- End diff --
    
    typo: `filed` -> `field`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43953382
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java ---
    @@ -67,27 +67,11 @@ public OutputEmitter(ShipStrategyType strategy) {
     	 * @param comparator The comparator used to hash / compare the records.
     	 */
     	public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator) {
    -		this(strategy, comparator, null, null);
    +		this(strategy, comparator, null);
     	}
     	
    -	/**
    -	 * Creates a new channel selector that uses the given strategy (broadcasting, partitioning, ...)
    -	 * and uses the supplied comparator to hash / compare records for partitioning them deterministically.
    -	 * 
    -	 * @param strategy The distribution strategy to be used.
    -	 * @param comparator The comparator used to hash / compare the records.
    -	 * @param distr The distribution pattern used in the case of a range partitioning.
    -	 */
    -	public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, DataDistribution distr) {
    -		this(strategy, comparator, null, distr);
    -	}
    -	
    -	public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, Partitioner<?> partitioner) {
    -		this(strategy, comparator, partitioner, null);
    -	}
    -		
     	@SuppressWarnings("unchecked")
    -	public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, Partitioner<?> partitioner, DataDistribution distr) {
    +	public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, Partitioner<?> partitioner) {
    --- End diff --
    
    I would keep the `DataDistribution` in the `OutputEmitter`. 
    We can later adapt this such that we can set the `DataDistribution` at execution time from the broadcasted set and avoid the partition ID assignment.
    
    For now, we can simply set an Integer distribution with boundaries 1,2,3...,n. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46575587
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +	final static long SEED = org.apache.flink.api.java.Utils.RNG.nextLong();
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, SEED);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, SEED);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, DataExchangeMode.PIPELINED);
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.PIPELINED);
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, DataExchangeMode.PIPELINED);
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, TempMode.NONE);
    +		partChannel.setDataExchangeMode(DataExchangeMode.PIPELINED);
    +		final FieldList keys = new FieldList(0);
    +		final boolean[] sortDirection = { true };
    +		partChannel.setShipStrategy(ShipStrategyType.PARTITION_RANGE, keys, sortDirection, null, DataExchangeMode.PIPELINED);
    +		ariPlanNode.addOutgoingChannel(channel);
    +		partChannel.setLocalStrategy(channel.getLocalStrategy(), keys, sortDirection);
    --- End diff --
    
    We cannot remove the target node and apply its local strategy on the `PartitionIDRemoveWrapper` because
    1. the keys of the local strategy are changed (here keys are fixed to `{0}` and order to `{true}`.
    2. the local strategy is applied before the user functions, i.e., originally it is applied on the `SourceOut` type and here it is applied on the `Tuple2<Int, SourceOut>` type.
    
    I would like to 
    - keep the `target` node and its local strategy
    - connect the `PartitionIDRemoveWrapper` to the `target` node
    - change the ship strategy of the channel to the target node from `PARTITION_RANGE` to `FORWARD` and the data exchange mode to `PIPELINED`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43989719
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.api.java.functions;
    --- End diff --
    
    Ok, make sense to me.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43986963
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/RangeBoundaryBuilder.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.CommonRangeBoundaries;
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +
    +/**
    + * Build RangeBoundaries with input records. First, sort the input records, and then select
    + * the boundaries with same interval.
    + *
    + * @param <T>
    + */
    +public class RangeBoundaryBuilder<T> extends RichMapPartitionFunction<T, RangeBoundaries<T>> {
    +
    +	private int parallelism;
    +	private final TypeComparatorFactory<T> comparatorFactory;
    +
    +	public RangeBoundaryBuilder(TypeComparatorFactory<T> comparator, int parallelism) {
    +		this.comparatorFactory = comparator;
    +		this.parallelism = parallelism;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<T> values, Collector<RangeBoundaries<T>> out) throws Exception {
    +		final TypeComparator<T> comparator = this.comparatorFactory.createComparator();
    +		List<T> sampledData = new ArrayList<>();
    +		for (T value : values) {
    +			sampledData.add(value);
    +		}
    +		Collections.sort(sampledData, new Comparator<T>() {
    +			@Override
    +			public int compare(T first, T second) {
    +				return comparator.compare(first, second);
    +			}
    +		});
    +
    +		List<T> boundaries = new ArrayList<>();
    +		double avgRange = sampledData.size() / (double) parallelism;
    +		for (int i = 1; i < parallelism; i++) {
    +			boundaries.add(sampledData.get((int) (i * avgRange)));
    +		}
    +
    +		RangeBoundaries<T> rangeBoundaries = new CommonRangeBoundaries<>(boundaries);
    --- End diff --
    
    I am in favor of keeping `RangeBoundary` as well. But I think we can build the `RangeBoundary` in the `AssignRangeId` from the individual Key records, no?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-148070644
  
    Hi @ChengXiangLi, thanks a lot for the PR! 
    Range partitioning has been very long on our feature list ;-)
    I'll look into this in the next days.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by uce <gi...@git.apache.org>.
Github user uce commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-165400271
  
    Sorry for not responding earlier. I've missed the notification somehow.
    
    Batch exchange (the blocking intermediate results) do not work within iterations. Therefore the optimizer never sets the exchange mode to batch within iterations currently.
    
    I think we need to suppress range partitionings within iterations for the time being.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44124427
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java ---
    @@ -134,6 +134,10 @@ public Channel(PlanNode sourceNode, TempMode tempMode) {
     	public PlanNode getSource() {
     		return this.source;
     	}
    +
    +	public void setSource(PlanNode source) {
    --- End diff --
    
    Create new channels means that we need to update the input channel existing `PlanNode`, there is no interface to set input channel in `PlanNode` as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-161465668
  
    Sorry, I forgot about this PR. Thanks for reminding me.
    Will check it in the next days.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44125013
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    --- End diff --
    
    Both output channels of source node is `DataExchangeMode,PIPELINED`, i think set the `DataExchangeMode` of the channel which connect to `AssignRangeIndex` node to `BATCH` may avoid the deadlock, I would launch a test cluster to verify this, let you know when i got more information.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43956183
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java ---
    @@ -134,6 +134,10 @@ public Channel(PlanNode sourceNode, TempMode tempMode) {
     	public PlanNode getSource() {
     		return this.source;
     	}
    +
    +	public void setSource(PlanNode source) {
    --- End diff --
    
    I think we do not need this method, if we create completely new channels in the `RangePartitionRewriter`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44257060
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java ---
    @@ -148,9 +147,6 @@ else if (pKeys instanceof Keys.SelectorFunctionKeys) {
     			}
     			
     		} 
    -		else if (pMethod == PartitionMethod.RANGE) {
    --- End diff --
    
    I see your point. It's probably OK to leave out this check. `sortPartition()` is doing the same check and would fail if the same non-sortable key is used.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43991097
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java ---
    @@ -148,9 +147,6 @@ else if (pKeys instanceof Keys.SelectorFunctionKeys) {
     			}
     			
     		} 
    -		else if (pMethod == PartitionMethod.RANGE) {
    --- End diff --
    
    Should we check this in some else place? I mean it's more like a sort issue, not a range partitioner issue. If  `pKeys.getKeyType().isSortKeyType() == false` and not fully sort, user just want to balance skew data through range partitioner shuffle, we should support it, right?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43987990
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    --- End diff --
    
    The deadlock will only occur for later data sets because some of the the data can be buffered in the memory of Flink's network stack. The network buffers are filled up if a receiver is blocked and cannot receive data. Once the buffers are full, it produces backpressure and the producing task stalls. Since the producer needs to finish to before the receiver can receive data, we have a deadlock.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46792344
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +	final static long SEED = org.apache.flink.api.java.Utils.RNG.nextLong();
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    --- End diff --
    
    Thanks for the remind, @fhueske , I've added verification on `PartitionOperator` level and related unit test. BTW, would the `Record` type be fully discarded finally? it seems been broken on many features, but there are still lots of related code remain in Flink.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403659
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/AbstractJoinDescriptor.java ---
    @@ -143,8 +143,7 @@ else if(produced1.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED &&
     					produced2.getPartitioning() == PartitioningProperty.RANGE_PARTITIONED) {
     
     				// both are range partitioned, check that partitioning fields are equivalently chosen
    --- End diff --
    
    Please adapt the comment


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44406299
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, channel.getTempMode());
    +		partChannel.setDataExchangeMode(channel.getDataExchangeMode());
    +		final FieldList keys = new FieldList(0);
    +		final boolean[] sortDirection = { true };
    +		partChannel.setShipStrategy(channel.getShipStrategy(), keys, sortDirection, channel.getPartitioner(), channel.getDataExchangeMode());
    --- End diff --
    
    ShipStrategy should be `RANGE` (to make it more clear what we are doing), the Partitioner can be `null` (we are not using a custom partitioner here), DataExchangeMode should be `PIPELINED`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-166297616
  
    Sorry, @fhueske , i misunderstood your test data, the keys should be skewed on some value, while in my previous test, the keys are now skewed. it's complicate to calculate how many samples should be taken from a dataset to meet an a priori specified accuracy guarantee, one of the algorithm is described at http://research.microsoft.com/pubs/159275/MSR-TR-2012-18.pdf which i used before, but it should not totally fit into the case which keys are skewed.
    Would you continue to test how much it required to make partition roughly balanced? Raise the sample number should not add much overhead, i'm totally support of it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43987736
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/shipping/OutputEmitter.java ---
    @@ -135,7 +116,7 @@ public OutputEmitter(ShipStrategyType strategy, TypeComparator<T> comparator, Pa
     		case PARTITION_CUSTOM:
     			return customPartition(record.getInstance(), numberOfChannels);
     		case PARTITION_RANGE:
    -			return rangePartition(record.getInstance(), numberOfChannels);
    +			return rangePartition((Tuple2<Integer, ?>)record.getInstance(), numberOfChannels);
    --- End diff --
    
    Yes, the current version of `DataDistribution` was supported by a previous version of the API (even pre-Flink). With the generalized data model, `DataDistribution` should be backed by a `Object[][]` instead of `Key<?>[]` and the `TypeComparator`s should be used to compare the individual key fields.
    
    I think if we change DataDistribution now to operate on `Object[][]`, the follow up task would become much easier. Also if you plan to add a `DataDistribution` later, wouldn't it make more sense to update it now, instead of removing it first and adding an updated version later again?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44405811
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    --- End diff --
    
    DataExchangeMode should be `PIPELINED`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43951499
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/distributions/CommonRangeBoundaries.java ---
    @@ -0,0 +1,57 @@
    +/*
    + * 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.api.common.distributions;
    +
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +
    +import java.util.List;
    +
    +public class CommonRangeBoundaries<T> implements RangeBoundaries<T> {
    +	private List<T> boundaries;
    --- End diff --
    
    If we extract keys with `TypeComparator.extractKeys()` we can store the boundaries as `Object[][]` and do the comparisons field-wise with the comparators returned by `TypeComparator.getFlatComparators()`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-148716888
  
    Hi @ChengXiangLi, I had a look at your PR and I think we need to change the implementation a bit.
    Right now, it executes an additional job for each range partition operator to obtain a sample. The additional job executes the full program and samples at the end. Imagine a complex program that includes for instance several joins and wants to write out the result in a total order, i.e., range partitions and sorts the result before it writes to the final sink. With the current implementation, this would mean that the expensive job is executed twice.
    
    It would be better to inject the sampling into the actual job. This can be done for example as follow.
    For a program such as:
    ```
    DataSet x = ...
    x.rangePartition(0).reduce(...)
    ```
    could be translated into:
    ```
    DataSet<X> x = ...
    DataSet<Distr> dist = x.mapPartition("sample").reduce("collect samples and build distribution");
    DataSet<Tuple2<Integer,X>> xWithPIDs = x
      .map("assign PartitionIDs).withBroadcastSet(dist, "distribution");
    ```
    
    This would inject the sampling into the original program. The sampling is done as before, but the data distribution is broadcasted to a map operator that uses the distribution to assign partition IDs to records and converts the `DataSet<X>` into a `DataSet<Tuple2<Integer, X>>` similar as the `KeySelector`. Once the partition IDs are assigned, a RangePartitionOperator could partition the tuples on the first field (f0) with a simple Int-DataDistribution (0,1,2,3,4,..., n). Finally, the DataSet needs to be unwrapped, i.e, converted from `DataSet<Tuple2<Integer,X>>` to `DataSet<X>`. 
    
    I agree it is not super nice, but this implementationx would cache the intermediate result instead of recomputing it. In addition it barely touches the internals.
    
    It is also possible to integrate the partitioning more tightly into the runtime by providing the data distribution directly to the Partitioner. However, that would mean we need to implement a partitioning operator for the runtime (instead of using the regular operator and a NOOP driver).
    
    Btw. I have some code lying around (for a not-yet-completed features) to extract keys from a record given the key specification. Let me know if that would help for your implementation. 
    
    Regarding the implementation of the `Partitioner` and `OutputEmitter`, I am very open for suggestions for how to improve the design. As you said, I would bring this discussion to the dev mailing list or open a JIRA and start a discussion there.
    
    What do you think? Thanks, Fabian


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-149074329
  
    Thanks, @fhueske , it's definitely great that if we can only execute the pre-sample logic just once, i would update the code later.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44406066
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, channel.getTempMode());
    --- End diff --
    
    TempMode should be `NONE`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403298
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/distributions/CommonRangeBoundaries.java ---
    @@ -0,0 +1,73 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.flink.api.common.distributions;
    +
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +
    +public class CommonRangeBoundaries<T> implements RangeBoundaries<T> {
    +	private Object[][] boundaries;
    +
    +	public CommonRangeBoundaries(Object[][] boundaries) {
    +		this.boundaries = boundaries;
    +	}
    +
    +	@Override
    +	public int getRangeIndex(T record, TypeComparator<T> typeComparator) {
    +		return binarySearch(record, typeComparator);
    +	}
    +
    +	// Search the range index of input record.
    +	private int binarySearch(T record, TypeComparator<T> typeComparator) {
    +		int low = 0;
    +		int high = this.boundaries.length - 1;
    +		TypeComparator[] flatComparators = typeComparator.getFlatComparators();
    --- End diff --
    
    Can you add the `TypeComparator` to the constructor (it is tied to the boundaries anyways).
    Then we can initialize `flatComparators` and `keys` once and reuse them in all `binarySearch()` calls.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43953776
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/PartitionITCase.java ---
    @@ -71,6 +71,30 @@ public void testHashPartitionByKeyField() throws Exception {
     	}
     
     	@Test
    +	public void testRangePartitionByKeyField() throws Exception {
    +		/*
    +		 * Test hash partition by key field
    +		 */
    +
    +		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    +
    +		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
    --- End diff --
    
    You can use `env.generateSequence()` to generate more input. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43971562
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.api.java.functions;
    --- End diff --
    
    Should they be part of `flink-optimizer`? The rewrite happens during optimization phase, we have not touch runtime yet.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44521192
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/operators/GroupReduceWithCombineProperties.java ---
    @@ -121,6 +121,9 @@ public SingleInputPlanNode instantiate(Channel in, SingleInputNode node) {
     			Channel toReducer = new Channel(combiner);
     			toReducer.setShipStrategy(in.getShipStrategy(), in.getShipStrategyKeys(),
     									in.getShipStrategySortOrder(), in.getDataExchangeMode());
    +			if (in.getShipStrategy() == ShipStrategyType.PARTITION_RANGE) {
    --- End diff --
    
    In this case, a sourceNode-->targetNode would be transferred into sourceNode --> combineNode --> targetNode, if it's range partition between sourceNode and targetNode, the data distribution is forget to copy to the new created channel.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44406114
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    +		final int sampleSize = 20 * targetParallelism;
    +		final SampleInPartition sampleInPartition = new SampleInPartition(false, sampleSize, seed);
    +		final TypeInformation<?> sourceOutputType = sourceNode.getOptimizerNode().getOperator().getOperatorInfo().getOutputType();
    +		final TypeInformation<IntermediateSampleData> isdTypeInformation = TypeExtractor.getForClass(IntermediateSampleData.class);
    +		final UnaryOperatorInformation sipOperatorInformation = new UnaryOperatorInformation(sourceOutputType, isdTypeInformation);
    +		final MapPartitionOperatorBase sipOperatorBase = new MapPartitionOperatorBase(sampleInPartition, sipOperatorInformation, "Sample in partitions");
    +		final MapPartitionNode sipNode = new MapPartitionNode(sipOperatorBase);
    +		final Channel sipChannel = new Channel(sourceNode, TempMode.NONE);
    +		sipChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sipPlanNode = new SingleInputPlanNode(sipNode, "SampleInPartition PlanNode", sipChannel, DriverStrategy.MAP_PARTITION);
    +		sipPlanNode.setParallelism(sourceParallelism);
    +		sipChannel.setTarget(sipPlanNode);
    +		this.plan.getAllNodes().add(sipPlanNode);
    +		sourceNewOutputChannels.add(sipChannel);
    +
    +		// 2. Fixed size sample in a single coordinator.
    +		final SampleInCoordinator sampleInCoordinator = new SampleInCoordinator(false, sampleSize, seed);
    +		final UnaryOperatorInformation sicOperatorInformation = new UnaryOperatorInformation(isdTypeInformation, sourceOutputType);
    +		final GroupReduceOperatorBase sicOperatorBase = new GroupReduceOperatorBase(sampleInCoordinator, sicOperatorInformation, "Sample in coordinator");
    +		final GroupReduceNode sicNode = new GroupReduceNode(sicOperatorBase);
    +		final Channel sicChannel = new Channel(sipPlanNode, TempMode.NONE);
    +		sicChannel.setShipStrategy(ShipStrategyType.PARTITION_HASH, channel.getShipStrategyKeys(), channel.getShipStrategySortOrder(), null, channel.getDataExchangeMode());
    +		final SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		final RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		final TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		final UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		final MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		final MapPartitionNode rbNode = new MapPartitionNode(rbOperatorBase);
    +		final Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		final SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		final AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		final TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		final UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		final MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		final MapPartitionNode ariNode = new MapPartitionNode(ariOperatorBase);
    +		final Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		// To avoid deadlock, set the DataExchangeMode of channel between source node and this to Batch.
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, DataExchangeMode.BATCH);
    +		final SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +		sourceNewOutputChannels.add(ariChannel);
    +
    +		final NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		final Channel partChannel = new Channel(ariPlanNode, channel.getTempMode());
    +		partChannel.setDataExchangeMode(channel.getDataExchangeMode());
    --- End diff --
    
    DataExchangeMode should be `PIPELINED`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44263202
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/plan/Channel.java ---
    @@ -134,6 +134,10 @@ public Channel(PlanNode sourceNode, TempMode tempMode) {
     	public PlanNode getSource() {
     		return this.source;
     	}
    +
    +	public void setSource(PlanNode source) {
    --- End diff --
    
    Fair enough. Adding the `setSource()` method seems to be less invasive compared to adding several update channel methods for different `PlanNode`s.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/1255


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46674245
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +	final static long SEED = org.apache.flink.api.java.Utils.RNG.nextLong();
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    --- End diff --
    
    I think the `RecordOutputEmitter` would fail to partition by range because it does not have access to the generated data distribution. You are right, it would not silently produce wrong results, but probably fail at runtime with a `NullPointerException` in line 193 of `RecordOutputEmitter`. 
    
    Good observation with the `GenericDataSinkBase`! The methods that relate to range partitioning and global order are artifacts from the recently removed Record API and not publicly accessible. In fact, they are useless, because they lack the definition of a key on which the output should be partitioned / ordered. 
    I will open a PR to remove the `setGlobalOrder()`, `setRangePartitioned()`, `getDataDistribution()`, and `getPartitioningOrder()` methods from the `GenericDataSinkBase`. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43987268
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		appendOutgoingChannels.add(ariChannel);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +
    +		channel.setSource(ariPlanNode);
    +		ariPlanNode.addOutgoingChannel(channel);
    +
    +		NamedChannel broadcastChannel = new NamedChannel("RangeBoundaries", rbPlanNode);
    +		broadcastChannel.setShipStrategy(ShipStrategyType.BROADCAST, channel.getDataExchangeMode());
    +		broadcastChannel.setTarget(ariPlanNode);
    +		List<NamedChannel> broadcastChannels = new ArrayList<>(1);
    +		broadcastChannels.add(broadcastChannel);
    +		ariPlanNode.setBroadcastInputs(broadcastChannels);
    +
    +		// 5. Remove the partition id.
    +		PartitionIDRemoveWrapper partitionIDRemoveWrapper = new PartitionIDRemoveWrapper();
    --- End diff --
    
    Ah, yes. You are right! I for got about the Partition node. However, the Partition node operates with a `NoOpDriver` which means that it does nothing. Hence, I think we can safely replace this node by the `PartitionIDRemoveWrapper` which removes the ID after the partitioning instead of nothing. This will avoid some serialization overhead induced by the additional operator.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43973217
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/RangeBoundaryBuilder.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.CommonRangeBoundaries;
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +
    +/**
    + * Build RangeBoundaries with input records. First, sort the input records, and then select
    + * the boundaries with same interval.
    + *
    + * @param <T>
    + */
    +public class RangeBoundaryBuilder<T> extends RichMapPartitionFunction<T, RangeBoundaries<T>> {
    +
    +	private int parallelism;
    +	private final TypeComparatorFactory<T> comparatorFactory;
    +
    +	public RangeBoundaryBuilder(TypeComparatorFactory<T> comparator, int parallelism) {
    +		this.comparatorFactory = comparator;
    +		this.parallelism = parallelism;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<T> values, Collector<RangeBoundaries<T>> out) throws Exception {
    +		final TypeComparator<T> comparator = this.comparatorFactory.createComparator();
    +		List<T> sampledData = new ArrayList<>();
    +		for (T value : values) {
    +			sampledData.add(value);
    +		}
    +		Collections.sort(sampledData, new Comparator<T>() {
    +			@Override
    +			public int compare(T first, T second) {
    +				return comparator.compare(first, second);
    +			}
    +		});
    +
    +		List<T> boundaries = new ArrayList<>();
    +		double avgRange = sampledData.size() / (double) parallelism;
    +		for (int i = 1; i < parallelism; i++) {
    +			boundaries.add(sampledData.get((int) (i * avgRange)));
    +		}
    +
    +		RangeBoundaries<T> rangeBoundaries = new CommonRangeBoundaries<>(boundaries);
    --- End diff --
    
    I would prefer to keep the `RangeBoundary` abstraction, it actually does not add any real extra effort, we may need this flexibility for further optimization. For example, we can add a `WeightedRangBoundary` to support weighted range partitioner easily.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-148290945
  
    Thanks for the review in advance, @fhueske . Something i want to mention here is that, the range partitioner implementation in this PR follow the existed partitioner design pattern, which in my personal view, is not well designed by the principle of object-oriented programing, the parameters of different partitioners are passed through Flink all together, and all partitioners logic mixed in OutputEmitter, i'm not very satisfied with the implementation in this PR actually. I think we need refactor the partitioner as a followup work, but this should not block this PR. Range partitioner is a little different from other partitioners, with it enabled, we could have a better overview about how to design the abstraction of partitioner. The refactor should require a design doc and fully discussed in community, what do you think about this?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403447
  
    --- Diff: docs/apis/programming_guide.md ---
    @@ -742,6 +742,17 @@ DataSet<Integer> result = in.partitionByHash(0)
           </td>
         </tr>
         <tr>
    +      <td><strong>Range-Partition</strong></td>
    +      <td>
    +        <p>Range-partitions a data set on a given key. Keys can be specified as key-selector functions or field position keys.</p>
    --- End diff --
    
    should be "position keys, expression keys, and key selector functions".


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43006432
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/DataSet.java ---
    @@ -1223,6 +1230,51 @@ public long count() throws Exception {
     		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, getType());
     		return new PartitionOperator<T>(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys<T, K>(clean(keyExtractor), this.getType(), keyType), Utils.getCallLocationName());
     	}
    +
    +	/**
    +	 * Range-partitions a DataSet using the specified KeySelector.
    +	 * <p>
    +	 * <b>Important:</b>This operation shuffles the whole DataSet over the network and can take significant amount of time.
    +	 *
    +	 * @param keySelector The KeySelector with which the DataSet is range-partitioned.
    +	 * @return The partitioned DataSet.
    +	 *
    +	 * @see KeySelector
    +	 */
    +	public <K extends Comparable<K>> DataSet<T> partitionByRange(KeySelector<T, K> keySelector) {
    +		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, getType());
    +		String callLocation = Utils.getCallLocationName();
    +
    +		// Extract key from input element by keySelector.
    +		KeyExtractorMapper<T, K> keyExtractorMapper = new KeyExtractorMapper<T, K>(keySelector);
    --- End diff --
    
    It would be good to inject the sampling and partition ID assignment code in the `JobGraphGenerator` and not at the API level. The `JobGraphGenerator` is called after the `Optimizer` and translates the optimized plan into a parallel data flow called `JobGraph` which is executed by the runtime. The benefit of injecting the code at this point is that any range partitioning can be handled transparently within the optimizer. This means also other operators except the explicit `partitionByRange()` such as Join, CoGroup, and Reduce can benefit from range partitioning. In addition this makes the injected code a part of the runtime which can be more transparently improved later on. 
    
    The downside (for you) is that the job abstraction is much lower at this level. However, you have still access to the chosen key fields and type information of all operators. See the `JavaApiPostPass` class to learn how to generate serializers and comparators at this level.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-165453966
  
    I executed the Flink WordCount example on 4 nodes with 8 parallel tasks and roughly 17GB of input data once with hash partitioning and once with range partitioning. Both times no combiner was used.
    First of all, both programs compute the same result and the result of the range partitioned WordCount is ordered. So the implementation is correct.
    
    The hash partitioned WC took 8:00 mins and the range partitioned 13:17 mins. 
    The breakdown of the range partitioned WC looks as follows:
    
    1. Source+FlatMap: 3:01 mins
    2. LocalSample: 3:01 mins
    3. GlobalSample: 0:15 mins
    4. Histogram: 24 ms
    5. PreparePartition: 8:49 mins
    6. Partition: 8:48 mins
    7. GroupReduce: 10:14 mins
    8. Sink: 1:09 mins
    
    The breakdown of the hash partitioned WC is:
    
    1. Source + FlatMap: 6:26 mins
    2. Partition: 6:25 mins
    3. GroupReduce: 7:58 mins
    4. Sink: 1:21 mins
    
    So, the overhead of the range partitioned WC comes from additional IO of reading the flatMapped words and the additional 4-byte integer. Also the sorting of the group reduce does not happen concurrently with the source IO. Reducing (w/o sort) and sink take about the same amount of time.
    
    I also check the distribution of input and output records / bytes for the GroupReduce.
    
    |  | min records-in | min bytes-in | max records-in | max bytes-in | 
    | --- | --- | --- | --- | --- |
    | Hash | 197M | 1.82GB | 346M | 2.90GB |  
    | Range | 177M | 1.41GB | 352M | 2.90GB |
    
    |  | min records-out | min bytes-out | max records-out | max bytes-out | 
    | --- | --- | --- | --- | --- |
    Hash | 2.5M | 26.5MB | 2.5M | 26.5MB
    Range | 2.3K | 28.2KB | 14M | 154MB
    
    We see that the range partitioner does not perform better (in fact a bit worse) than the hash partitioner (the differences for output records are expected). Maybe increasing the sample size helps? The overhead of reading the the intermediate data set from disk is so high, that a more fine-grained histogram can be justified, IMO. How about increasing the sample size from `parallelism * 20` to `parallelism * 1000`?
    
    Other thoughts?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-149849817
  
    Implemented the range partitioner with KeySelector based on broadcast data distribution, it's not fully finished yet, @fhueske , for the range partitioner with filed index or field name, it should require the tool you have mentioned before about extracting keys from a record.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43988062
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java ---
    @@ -148,9 +147,6 @@ else if (pKeys instanceof Keys.SelectorFunctionKeys) {
     			}
     			
     		} 
    -		else if (pMethod == PartitionMethod.RANGE) {
    --- End diff --
    
    Yes, that's my intention. Range partitioning is often used to produce fully sorted output. This does not work if the partition keys have no defined sort order.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43950406
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.api.java.functions;
    --- End diff --
    
    The range partition functions (`AssignRangeIndex`, `RangeBoundaryBuilder`, and `PartitionIDRemoveWrapper`) should be moved from `flink-java` to `flink-runtime` because they are rather part of the runtime than of the user-facing API.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43990006
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/RangeBoundaryBuilder.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.CommonRangeBoundaries;
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparator;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +
    +/**
    + * Build RangeBoundaries with input records. First, sort the input records, and then select
    + * the boundaries with same interval.
    + *
    + * @param <T>
    + */
    +public class RangeBoundaryBuilder<T> extends RichMapPartitionFunction<T, RangeBoundaries<T>> {
    +
    +	private int parallelism;
    +	private final TypeComparatorFactory<T> comparatorFactory;
    +
    +	public RangeBoundaryBuilder(TypeComparatorFactory<T> comparator, int parallelism) {
    +		this.comparatorFactory = comparator;
    +		this.parallelism = parallelism;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<T> values, Collector<RangeBoundaries<T>> out) throws Exception {
    +		final TypeComparator<T> comparator = this.comparatorFactory.createComparator();
    +		List<T> sampledData = new ArrayList<>();
    +		for (T value : values) {
    +			sampledData.add(value);
    +		}
    +		Collections.sort(sampledData, new Comparator<T>() {
    +			@Override
    +			public int compare(T first, T second) {
    +				return comparator.compare(first, second);
    +			}
    +		});
    +
    +		List<T> boundaries = new ArrayList<>();
    +		double avgRange = sampledData.size() / (double) parallelism;
    +		for (int i = 1; i < parallelism; i++) {
    +			boundaries.add(sampledData.get((int) (i * avgRange)));
    +		}
    +
    +		RangeBoundaries<T> rangeBoundaries = new CommonRangeBoundaries<>(boundaries);
    --- End diff --
    
    Ok, seems no difference from my side.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-162935419
  
    Thanks for the fast update!
    Your last commit seems to include changes that have already been merged to the master branch. Can you revert it and rebase the second last commit on the current master?
    
    I also got feedback from @StephanEwen about the use of the IntDistribution. He shares your opinion, that it is not really needed and just incurs additional overhead due to the second binary search. He proposes to use a `PARTITION_CUSTOM` strategy instead of a `PARTITION_RANGE` strategy with a `Partitioner` that simply returns the partition id. You basically can replace 
    `partChannel.setShipStrategy(ShipStrategyType.PARTITION_RANGE, keys, sortDirection, null, DataExchangeMode.PIPELINED);`
    by 
    `partChannel.setShipStrategy(ShipStrategyType.PARTITION_CUSTOM, keys, new IdPartitioner(), DataExchangeMode.PIPELINED);`
    and provide an implementation for IdPartitioner. This way, we have still support for generic DataDistribution range partitioning (although not being used ATM) and a more efficient automatic range partitioning. 
    
    On last very small remark. Could you please set the parallelism also for the `OptimizerNodes` (`sipNode`, `sicNode`, etc.)? Otherwise, the JSON output will have some `"parallelism": "default"` values.
    
    I think we're almost there. :-)
    Thanks a lot, Fabian


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43987723
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    --- End diff --
    
    We have go through the integration test, so i think it's resolved in some way. Quite interesting, i would debug to see what's going on.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-152108364
  
    I add the `RangePartitionRewriter` at `Optimizer.comple()` finally, as it rewrite `OptimizedPlan`, and utilize the `JavaApiPostPass`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43954968
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    +		ariChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode ariPlanNode = new SingleInputPlanNode(ariNode, "AssignRangeIndex PlanNode", ariChannel, DriverStrategy.MAP_PARTITION);
    +		ariPlanNode.setParallelism(sourceParallelism);
    +		ariChannel.setTarget(ariPlanNode);
    +		appendOutgoingChannels.add(ariChannel);
    +		this.plan.getAllNodes().add(ariPlanNode);
    +
    +		channel.setSource(ariPlanNode);
    --- End diff --
    
    I would not adapt the given channel but completely replace it. In that case you also do not need to add the `setSource` method to `Channel`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44404272
  
    --- Diff: flink-optimizer/src/main/java/org/apache/flink/optimizer/traversals/RangePartitionRewriter.java ---
    @@ -0,0 +1,194 @@
    +/*
    + * 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.operators.util.FieldList;
    +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.typeutils.RecordTypeInfo;
    +import org.apache.flink.runtime.io.network.DataExchangeMode;
    +import org.apache.flink.runtime.operators.udf.AssignRangeIndex;
    +import org.apache.flink.runtime.operators.udf.PartitionIDRemoveWrapper;
    +import org.apache.flink.runtime.operators.udf.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> {
    +
    +	final OptimizedPlan plan;
    +
    +	public RangePartitionRewriter(OptimizedPlan plan) {
    +		this.plan = plan;
    +	}
    +
    +	@Override
    +	public boolean preVisit(PlanNode visitable) {
    +		return true;
    +	}
    +
    +	@Override
    +	public void postVisit(PlanNode visitable) {
    +		final List<Channel> outgoingChannels = visitable.getOutgoingChannels();
    +		final List<Channel> newOutGoingChannels = new LinkedList<>();
    +		final List<Channel> toBeRemoveChannels = new ArrayList<>();
    +		for (Channel channel : outgoingChannels) {
    +			ShipStrategyType shipStrategy = channel.getShipStrategy();
    +			if (shipStrategy == ShipStrategyType.PARTITION_RANGE) {
    +				TypeInformation<?> outputType = channel.getSource().getProgramOperator().getOperatorInfo().getOutputType();
    +				// Do not optimize for record type, it's a special case for range partitioner, and should be removed later.
    +				if (!(outputType instanceof RecordTypeInfo)) {
    +					newOutGoingChannels.addAll(rewriteRangePartitionChannel(channel));
    +					toBeRemoveChannels.add(channel);
    +				}
    +			}
    +		}
    +
    +		for (Channel chan : toBeRemoveChannels) {
    +			outgoingChannels.remove(chan);
    +		}
    +		outgoingChannels.addAll(newOutGoingChannels);
    +	}
    +
    +	private List<Channel> rewriteRangePartitionChannel(Channel channel) {
    +		final List<Channel> sourceNewOutputChannels = new ArrayList<>();
    +		final PlanNode sourceNode = channel.getSource();
    +		final PlanNode targetNode = channel.getTarget();
    +		final int sourceParallelism = sourceNode.getParallelism();
    +		final int targetParallelism = targetNode.getParallelism();
    +		final TypeComparatorFactory<?> comparator = Utils.getShipComparator(channel, this.plan.getOriginalPlan().getExecutionConfig());
    +		// 1. Fixed size sample in each partitions.
    +		final long seed = org.apache.flink.api.java.Utils.RNG.nextLong();
    --- End diff --
    
    Can we use a static seed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/1255#issuecomment-166330992
  
    Thanks @ChengXiangLi. 
    I will increase the sample size to 1000 elements per partition and finally merge this PR :-)
    
    Thanks for your patience!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43156865
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/DataSet.java ---
    @@ -1223,6 +1230,51 @@ public long count() throws Exception {
     		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, getType());
     		return new PartitionOperator<T>(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys<T, K>(clean(keyExtractor), this.getType(), keyType), Utils.getCallLocationName());
     	}
    +
    +	/**
    +	 * Range-partitions a DataSet using the specified KeySelector.
    +	 * <p>
    +	 * <b>Important:</b>This operation shuffles the whole DataSet over the network and can take significant amount of time.
    +	 *
    +	 * @param keySelector The KeySelector with which the DataSet is range-partitioned.
    +	 * @return The partitioned DataSet.
    +	 *
    +	 * @see KeySelector
    +	 */
    +	public <K extends Comparable<K>> DataSet<T> partitionByRange(KeySelector<T, K> keySelector) {
    +		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keySelector, getType());
    +		String callLocation = Utils.getCallLocationName();
    +
    +		// Extract key from input element by keySelector.
    +		KeyExtractorMapper<T, K> keyExtractorMapper = new KeyExtractorMapper<T, K>(keySelector);
    --- End diff --
    
    I think you would still have the nodes and all the information of the `OptimizedPlan` available in `connectJobVertices()`. However, I would also be OK to do it as a preprocessing step in `compileJobGraph()`. 
    Let me know if you face any obstacles or have any questions.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43986789
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.api.java.functions;
    --- End diff --
    
    My intention was to move the classes to `flink-runtime` because this is where most of the execution code is located, including the execution vertices. `flink-optimizer` does not contain execution code AFAIK and does only translates the API code into an executable representation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r44403456
  
    --- Diff: docs/apis/programming_guide.md ---
    @@ -1020,6 +1031,17 @@ val result = in.partitionByHash(0).mapPartition { ... }
     {% endhighlight %}
           </td>
         </tr>
    +    <tr>
    +      <td><strong>Range-Partition</strong></td>
    +      <td>
    +        <p>Range-partitions a data set on a given key. Keys can be specified as key-selector functions, tuple positions
    --- End diff --
    
    should be "position keys, expression keys, and key selector functions".


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by ChengXiangLi <gi...@git.apache.org>.
Github user ChengXiangLi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43992361
  
    --- 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());
    +		SingleInputPlanNode sicPlanNode = new SingleInputPlanNode(sicNode, "SampleInCoordinator PlanNode", sicChannel, DriverStrategy.ALL_GROUP_REDUCE);
    +		sicPlanNode.setParallelism(1);
    +		sicChannel.setTarget(sicPlanNode);
    +		sipPlanNode.addOutgoingChannel(sicChannel);
    +		this.plan.getAllNodes().add(sicPlanNode);
    +
    +		// 3. Use sampled data to build range boundaries.
    +		RangeBoundaryBuilder rangeBoundaryBuilder = new RangeBoundaryBuilder(comparator, targetParallelism);
    +		TypeInformation<CommonRangeBoundaries> rbTypeInformation = TypeExtractor.getForClass(CommonRangeBoundaries.class);
    +		UnaryOperatorInformation rbOperatorInformation = new UnaryOperatorInformation(sourceOutputType, rbTypeInformation);
    +		MapPartitionOperatorBase rbOperatorBase = new MapPartitionOperatorBase(rangeBoundaryBuilder, rbOperatorInformation, "RangeBoundaryBuilder");
    +		MapPartitionNode rbNode= new MapPartitionNode(rbOperatorBase);
    +		Channel rbChannel = new Channel(sicPlanNode, TempMode.NONE);
    +		rbChannel.setShipStrategy(ShipStrategyType.FORWARD, channel.getDataExchangeMode());
    +		SingleInputPlanNode rbPlanNode = new SingleInputPlanNode(rbNode, "RangeBoundary PlanNode", rbChannel, DriverStrategy.MAP_PARTITION);
    +		rbPlanNode.setParallelism(1);
    +		rbChannel.setTarget(rbPlanNode);
    +		sicPlanNode.addOutgoingChannel(rbChannel);
    +		this.plan.getAllNodes().add(rbPlanNode);
    +
    +		// 4. Take range boundaries as broadcast input and take the tuple of partition id and record as output.
    +		AssignRangeIndex assignRangeIndex = new AssignRangeIndex(comparator);
    +		TypeInformation<Tuple2> ariOutputTypeInformation = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, sourceOutputType);
    +		UnaryOperatorInformation ariOperatorInformation = new UnaryOperatorInformation(sourceOutputType, ariOutputTypeInformation);
    +		MapPartitionOperatorBase ariOperatorBase = new MapPartitionOperatorBase(assignRangeIndex, ariOperatorInformation, "Assign Range Index");
    +		MapPartitionNode ariNode= new MapPartitionNode(ariOperatorBase);
    +		Channel ariChannel = new Channel(sourceNode, TempMode.NONE);
    --- End diff --
    
    I got it, so this mean we can't reuse the source node output through broadcast range boundaries,  is there any other solution? otherwise, we have to launch a separate job to sample data and build range boundaries. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r46558726
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/DataSet.java ---
    @@ -1246,7 +1246,46 @@ public long count() throws Exception {
     		final TypeInformation<K> keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, getType());
     		return new PartitionOperator<T>(this, PartitionMethod.HASH, new Keys.SelectorFunctionKeys<T, K>(clean(keyExtractor), this.getType(), keyType), Utils.getCallLocationName());
     	}
    -	
    +
    +	/**
    +	 * Range-partitions a DataSet on the specified key fields.
    +	 * <p>
    +	 * <b>Important:</b>This operation shuffles the whole DataSet over the network and can take significant amount of time.
    --- End diff --
    
    We should add that range partitioning requires an additional pass over the data to obtain the distribution.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-7] [Runtime] Enable Range Partitioner.

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1255#discussion_r43952488
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/functions/AssignRangeIndex.java ---
    @@ -0,0 +1,55 @@
    +/*
    + * 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.api.java.functions;
    +
    +import org.apache.flink.api.common.distributions.RangeBoundaries;
    +import org.apache.flink.api.common.functions.RichMapPartitionFunction;
    +import org.apache.flink.api.common.typeutils.TypeComparatorFactory;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.util.Collector;
    +
    +import java.util.List;
    +
    +/**
    + * This mapPartition function require a DataSet with RangeBoundaries as broadcast input, it
    + * generate Tuple2 which includes range index and record itself as output.
    + *
    + * @param <IN> The original data type.
    + */
    +public class AssignRangeIndex<IN> extends RichMapPartitionFunction<IN, Tuple2<Integer, IN>> {
    +
    +	private TypeComparatorFactory<IN> typeComparator;
    +
    +	public AssignRangeIndex(TypeComparatorFactory<IN> typeComparator) {
    +		this.typeComparator = typeComparator;
    +	}
    +
    +	@Override
    +	public void mapPartition(Iterable<IN> values, Collector<Tuple2<Integer, IN>> out) throws Exception {
    +
    +		List<Object> broadcastVariable = getRuntimeContext().getBroadcastVariable("RangeBoundaries");
    --- End diff --
    
    FYI, the broadcasted `List` (and it's content) is reused by all parallel tasks on the same task manager. In this case it is fine, because the `RangeBoundaries` are used read-only.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---