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-785) Add Chained operators for AllReduce and AllGroupReduce
Date Fri, 13 Feb 2015 00:04:12 GMT

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

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

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

    https://github.com/apache/flink/pull/370#discussion_r24633778
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedAllGroupReduceCombineDriver.java
---
    @@ -0,0 +1,118 @@
    +/*
    + * 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.runtime.operators.chaining;
    +
    +import java.util.ArrayList;
    +
    +import org.apache.flink.api.common.functions.FlatCombineFunction;
    +import org.apache.flink.api.common.functions.Function;
    +import org.apache.flink.api.common.functions.util.FunctionUtils;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
    +import org.apache.flink.runtime.operators.RegularPactTask;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +public class ChainedAllGroupReduceCombineDriver<T> extends ChainedDriver<T,
T> {
    +	private static final Logger LOG = LoggerFactory.getLogger(ChainedAllGroupReduceCombineDriver.class);
    +
    +	// --------------------------------------------------------------------------------------------
    +	private FlatCombineFunction<T> combiner;
    +	private TypeSerializer<T> serializer;
    +
    +	private volatile boolean running = true;
    +
    +	private final ArrayList<T> values = new ArrayList();
    +
    +	// --------------------------------------------------------------------------------------------
    +	@Override
    +	public void setup(AbstractInvokable parent) {
    +		@SuppressWarnings("unchecked")
    +		final FlatCombineFunction<T> com = RegularPactTask.instantiateUserCode(this.config,
userCodeClassLoader, FlatCombineFunction.class);
    +		this.combiner = com;
    +		FunctionUtils.setFunctionRuntimeContext(com, getUdfRuntimeContext());
    +
    +		this.objectReuseEnabled = this.executionConfig.isObjectReuseEnabled();
    +
    +		final TypeSerializerFactory<T> serializerFactory = this.config.getInputSerializer(0,
this.userCodeClassLoader);
    +		this.serializer = serializerFactory.getSerializer();
    +
    +		if (LOG.isDebugEnabled()) {
    +			LOG.debug("ChainedAllGroupReduceCombineDriver object reuse: " + (this.objectReuseEnabled
? "ENABLED" : "DISABLED") + ".");
    +		}
    +	}
    +
    +	@Override
    +	public void openTask() throws Exception {
    +		final Configuration stubConfig = this.config.getStubParameters();
    +		RegularPactTask.openUserCode(this.combiner, stubConfig);
    +	}
    +
    +	@Override
    +	public void closeTask() throws Exception {
    +		if (!this.running) {
    +			return;
    +		}
    +		RegularPactTask.closeUserCode(this.combiner);
    +	}
    +
    +	@Override
    +	public void cancelTask() {
    +		this.running = false;
    +	}
    +
    +	// --------------------------------------------------------------------------------------------
    +	@Override
    +	public Function getStub() {
    +		return this.combiner;
    +	}
    +
    +	@Override
    +	public String getTaskName() {
    +		return this.taskName;
    +	}
    +
    +	// --------------------------------------------------------------------------------------------
    +	@Override
    +	public void collect(T record) {
    +		try {
    +			values.add(objectReuseEnabled ? record : serializer.copy(record));
    +			if (values.size() > 10000) {
    --- End diff --
    
    is there a more reasonable value for this? (basically anything whose reasoning goes beyond
"i felt like using 10000")


> Add Chained operators for AllReduce and AllGroupReduce
> ------------------------------------------------------
>
>                 Key: FLINK-785
>                 URL: https://issues.apache.org/jira/browse/FLINK-785
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: GitHub Import
>            Assignee: Chesnay Schepler
>              Labels: github-import
>             Fix For: pre-apache
>
>
> Because the operators `AllReduce` and `AllGroupReduce` are used both for the pre-reduce
(combiner side) and the final reduce, they would greatly benefit from a chained version.
> ---------------- Imported from GitHub ----------------
> Url: https://github.com/stratosphere/stratosphere/issues/785
> Created by: [StephanEwen|https://github.com/StephanEwen]
> Labels: runtime, 
> Milestone: Release 0.6 (unplanned)
> Created at: Sun May 11 17:41:12 CEST 2014
> State: open



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

Mime
View raw message