Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id CF426200C07 for ; Sun, 22 Jan 2017 17:25:10 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id CDC41160B45; Sun, 22 Jan 2017 16:25:10 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 22584160B38 for ; Sun, 22 Jan 2017 17:25:09 +0100 (CET) Received: (qmail 51382 invoked by uid 500); 22 Jan 2017 16:25:09 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 51372 invoked by uid 99); 22 Jan 2017 16:25:09 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 22 Jan 2017 16:25:09 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id E36B01806F7 for ; Sun, 22 Jan 2017 16:25:08 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -7.019 X-Spam-Level: X-Spam-Status: No, score=-7.019 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-2.999] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id SEUGKm5-N6yQ for ; Sun, 22 Jan 2017 16:25:07 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id B2D0B5F479 for ; Sun, 22 Jan 2017 16:25:06 +0000 (UTC) Received: (qmail 51362 invoked by uid 99); 22 Jan 2017 16:25:06 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 22 Jan 2017 16:25:06 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CA71CDFCBC; Sun, 22 Jan 2017 16:25:05 +0000 (UTC) From: shaoxuan-wang To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #3186: [FLINK-5582] [streaming] dd a general distributive... Content-Type: text/plain Message-Id: <20170122162505.CA71CDFCBC@git1-us-west.apache.org> Date: Sun, 22 Jan 2017 16:25:05 +0000 (UTC) archived-at: Sun, 22 Jan 2017 16:25:11 -0000 Github user shaoxuan-wang commented on a diff in the pull request: https://github.com/apache/flink/pull/3186#discussion_r97226958 --- Diff: flink-core/src/main/java/org/apache/flink/api/common/functions/AggregateFunction.java --- @@ -0,0 +1,94 @@ +/* + * 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.functions; + +import java.io.Serializable; + +/** + * + *

Aggregation functions must be {@link Serializable} because they are sent around + * between distributed processes during distributed execution. + * + *

An example how to use this interface is below: + * + *

{@code
    + * // the accumulator, which holds the state of the in-flight aggregate
    + * public class AverageAccumulator {
    + *     long count;
    + *     long sum;
    + * }
    + * 
    + * // implementation of an aggregation function for an 'average'
    + * public class Average implements AggregateFunction {
    + * 
    + *     public AverageAccumulator createAccumulator() {
    + *         return new AverageAccumulator();
    + *     }
    + * 
    + *     public AverageAccumulator merge(AverageAccumulator a, AverageAccumulator b) {
    + *         a.count += b.count;
    + *         a.sum += b.sum;
    + *         return a;
    + *     }
    + * 
    + *     public void add(Integer value, AverageAccumulator acc) {
    + *         acc.sum += value;
    + *         acc.count++;
    + *     }
    + * 
    + *     public Double getResult(AverageAccumulator acc) {
    + *         return acc.sum / (double) acc.count;
    + *     }
    + * }
    + * 
    + * // implementation of a weighted average
    + * // this reuses the same accumulator type as the aggregate function for 'average'
    + * public class WeightedAverage implements AggregateFunction {
    + *
    + *     public AverageAccumulator createAccumulator() {
    + *         return new AverageAccumulator();
    + *     }
    + *
    + *     public AverageAccumulator merge(AverageAccumulator a, AverageAccumulator b) {
    + *         a.count += b.count;
    + *         a.sum += b.sum;
    + *         return a;
    + *     }
    + *
    + *     public void add(Datum value, AverageAccumulator acc) {
    + *         acc.count += value.getWeight();
    + *         acc.sum += value.getValue();
    + *     }
    + *
    + *     public Double getResult(AverageAccumulator acc) {
    + *         return acc.sum / (double) acc.count;
    + *     }
    + * }
    + * }
+ */ +public interface AggregateFunction extends Function, Serializable { + + ACC createAccumulator(); + + void add(IN value, ACC accumulator); + + OUT getResult(ACC accumulator); + + ACC merge(ACC a, ACC b); --- End diff -- Do you think it is useful to extend merge function to accept a list of ACC: ACC merge(List a). There are cases where the group merging a list of instances is much more efficient than just merge only two instances. --- 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. ---