flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-7) [GitHub] Enable Range Partitioner
Date Wed, 04 Nov 2015 22:46:27 GMT

    [ https://issues.apache.org/jira/browse/FLINK-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14990614#comment-14990614
] 

ASF GitHub Bot commented on FLINK-7:
------------------------------------

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.


> [GitHub] Enable Range Partitioner
> ---------------------------------
>
>                 Key: FLINK-7
>                 URL: https://issues.apache.org/jira/browse/FLINK-7
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Distributed Runtime
>            Reporter: GitHub Import
>            Assignee: Chengxiang Li
>             Fix For: pre-apache
>
>
> The range partitioner is currently disabled. We need to implement the following aspects:
> 1) Distribution information, if available, must be propagated back together with the
ordering property.
> 2) A generic bucket lookup structure (currently specific to PactRecord).
> Tests to re-enable after fixing this issue:
>  - TeraSortITCase
>  - GlobalSortingITCase
>  - GlobalSortingMixedOrderITCase
> ---------------- Imported from GitHub ----------------
> Url: https://github.com/stratosphere/stratosphere/issues/7
> Created by: [StephanEwen|https://github.com/StephanEwen]
> Labels: core, enhancement, optimizer, 
> Milestone: Release 0.4
> Assignee: [fhueske|https://github.com/fhueske]
> Created at: Fri Apr 26 13:48:24 CEST 2013
> State: open



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message