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 00734200C10 for ; Thu, 19 Jan 2017 10:40:35 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id F3326160B5A; Thu, 19 Jan 2017 09:40:34 +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 25059160B54 for ; Thu, 19 Jan 2017 10:40:33 +0100 (CET) Received: (qmail 94900 invoked by uid 500); 19 Jan 2017 09:40:33 -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 94876 invoked by uid 99); 19 Jan 2017 09:40:32 -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; Thu, 19 Jan 2017 09:40:32 +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 86862180661 for ; Thu, 19 Jan 2017 09:40:32 +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 hnsVkDenUjb8 for ; Thu, 19 Jan 2017 09:40:31 +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 3BA085FCB7 for ; Thu, 19 Jan 2017 09:40:30 +0000 (UTC) Received: (qmail 94430 invoked by uid 99); 19 Jan 2017 09:40:29 -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; Thu, 19 Jan 2017 09:40:29 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 45C02DFB86; Thu, 19 Jan 2017 09:40:29 +0000 (UTC) From: fhueske To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #3150: [FLINK-4693][tableApi] Add session group-windows f... Content-Type: text/plain Message-Id: <20170119094029.45C02DFB86@git1-us-west.apache.org> Date: Thu, 19 Jan 2017 09:40:29 +0000 (UTC) archived-at: Thu, 19 Jan 2017 09:40:35 -0000 Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3150#discussion_r96817585 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregateCombineGroupFunction.scala --- @@ -0,0 +1,135 @@ +/* + * 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.table.runtime.aggregate + +import java.lang.Iterable + +import org.apache.flink.api.common.functions.RichGroupCombineFunction +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ResultTypeQueryable +import org.apache.flink.types.Row +import org.apache.flink.configuration.Configuration +import org.apache.flink.util.{Collector, Preconditions} + +import scala.collection.JavaConversions._ + +/** + * This wraps the aggregate logic inside of + * [[org.apache.flink.api.java.operators.GroupCombineOperator]]. + * + * @param aggregates The aggregate functions. + * @param groupingKeys + * @param intermediateRowArity The intermediate row field count. + * @param gap Session time window gap. + * @param intermediateRowType Intermediate row data type. + */ +class DataSetSessionWindowAggregateCombineGroupFunction( + aggregates: Array[Aggregate[_ <: Any]], + groupingKeys: Array[Int], + intermediateRowArity: Int, + gap: Long, + @transient intermediateRowType: TypeInformation[Row]) + extends RichGroupCombineFunction[Row,Row] with ResultTypeQueryable[Row] { + + private var aggregateBuffer: Row = _ + private var rowTimePos = 0 + + override def open(config: Configuration) { + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(groupingKeys) + aggregateBuffer = new Row(intermediateRowArity) + rowTimePos = intermediateRowArity - 2 + } + + /** + * For sub-grouped intermediate aggregate Rows, divide window based on the row-time + * (current'row-time - previous’row-time > gap), and then merge data (within a unified window) + * into an aggregate buffer. + * + * @param records Sub-grouped intermediate aggregate Rows . + * @return Combined intermediate aggregate Row. + * + */ + override def combine( + records: Iterable[Row], + out: Collector[Row]): Unit = { + + var head:Row = null + var lastRowTime: java.lang.Long = null + var currentRowTime: java.lang.Long = null + + val iterator = records.iterator() + + while (iterator.hasNext) { + val record = iterator.next() + currentRowTime = record.getField(rowTimePos).asInstanceOf[Long] + + // initial traversal or opening a new window + // the session window end is equal to last row-time + gap . + if (null == lastRowTime || + (null != lastRowTime && (currentRowTime > (lastRowTime + gap)))) { + + // calculate the current window and open a new window. + if (null != lastRowTime) { + // emit the current window's merged data + doCollect(out, head, lastRowTime) + } else { + // set group keys to aggregateBuffer. + for (i <- 0 until groupingKeys.length) { + aggregateBuffer.setField(i, record.getField(i)) + } + } + + // initiate intermediate aggregate value. + aggregates.foreach(_.initiate(aggregateBuffer)) + head = record --- End diff -- Do not remember an object that you received from the combine (or reduce) iterator. The iterator may repeatedly serve the same mutable object. If we only need the timestamp, we should remember only this in a `long`. Or we set the start time immediately in the `aggregateBuffer`. See also object reuse mode: https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/batch/index.html#operating-on-data-objects-in-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. ---