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 AE534200C3F for ; Wed, 22 Mar 2017 13:42:40 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id ACBCF160B86; Wed, 22 Mar 2017 12:42:40 +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 C881D160B83 for ; Wed, 22 Mar 2017 13:42:39 +0100 (CET) Received: (qmail 90383 invoked by uid 500); 22 Mar 2017 12:42:39 -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 90374 invoked by uid 99); 22 Mar 2017 12:42:39 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 22 Mar 2017 12:42:38 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 982ECC0578 for ; Wed, 22 Mar 2017 12:42:38 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.021 X-Spam-Level: X-Spam-Status: No, score=-4.021 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=-0.001] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id g4gV1GIe4wWU for ; Wed, 22 Mar 2017 12:42:37 +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 0BF425FCBA for ; Wed, 22 Mar 2017 12:42:35 +0000 (UTC) Received: (qmail 90342 invoked by uid 99); 22 Mar 2017 12:42:30 -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; Wed, 22 Mar 2017 12:42:30 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 30672DFBD3; Wed, 22 Mar 2017 12:42:30 +0000 (UTC) From: rtudoran To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #3590: [FLINK-5654] - Add processing time OVER RANGE BETW... Content-Type: text/plain Message-Id: <20170322124230.30672DFBD3@git1-us-west.apache.org> Date: Wed, 22 Mar 2017 12:42:30 +0000 (UTC) archived-at: Wed, 22 Mar 2017 12:42:40 -0000 Github user rtudoran commented on a diff in the pull request: https://github.com/apache/flink/pull/3590#discussion_r107402759 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedProcessingOverProcessFunction.scala --- @@ -0,0 +1,141 @@ +/* + * 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 org.apache.flink.api.common.state.{ListState, ListStateDescriptor} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.state.{FunctionInitializationContext, FunctionSnapshotContext} +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.functions.{Accumulator, AggregateFunction} +import org.apache.flink.types.Row +import org.apache.flink.util.{Collector, Preconditions} +import org.apache.flink.api.common.state.ValueState +import org.apache.flink.api.common.state.ValueStateDescriptor +import scala.util.control.Breaks._ + +/** + * Process Function used for the aggregate in partitioned bounded windows in + * [[org.apache.flink.streaming.api.datastream.DataStream]] + * + * @param aggregates the list of all [[org.apache.flink.table.functions.AggregateFunction]] + * used for this aggregation + * @param aggFields the position (in the input Row) of the input value for each aggregate + * @param forwardedFieldCount Is used to indicate fields in the current element to forward + * @param rowTypeInfo Is used to indicate the field schema + * @param time_boundary Is used to indicate the processing time boundaries + */ +class ProcTimeBoundedProcessingOverProcessFunction( + private val aggregates: Array[AggregateFunction[_]], + private val aggFields: Array[Int], + private val forwardedFieldCount: Int, + private val rowTypeInfo: RowTypeInfo, + private val time_boundary: Long) + extends ProcessFunction[Row, Row] { + + Preconditions.checkNotNull(aggregates) + Preconditions.checkNotNull(aggFields) + Preconditions.checkArgument(aggregates.length == aggFields.length) + + private var accumulators: Row = _ + private var output: Row = _ + private var windowBuffer: ListState[Tuple2[Long,Row]] = null + private var state: ValueState[Row] = _ + + + override def open(config: Configuration) { + output = new Row(forwardedFieldCount + aggregates.length) + + accumulators = new Row(aggregates.length) + var i = 0 + while (i < aggregates.length) { + accumulators.setField(i, aggregates(i).createAccumulator()) + i += 1 + } + + // We keep the elements received in a list state + // together with the ingestion time in the operator + val bufferDescriptor: ListStateDescriptor[Tuple2[Long,Row]] = + new ListStateDescriptor[Tuple2[Long,Row]]("windowBufferState", classOf[Tuple2[Long,Row]]) + windowBuffer = getRuntimeContext.getListState(bufferDescriptor) + + val stateDescriptor: ValueStateDescriptor[Row] = + new ValueStateDescriptor[Row]("overState", classOf[Row] , accumulators) + state = getRuntimeContext.getState(stateDescriptor) + } + + override def processElement( + input: Row, + ctx: ProcessFunction[Row, Row]#Context, + out: Collector[Row]): Unit = { + + var current_time = System.currentTimeMillis() + //buffer the event incoming event + windowBuffer.add(new Tuple2( + current_time, + input)) + + var i = 0 + + var accumulators = state.value() + + //set the fields of the last event to carry on with the aggregates + i = 0 + while (i < forwardedFieldCount) { + output.setField(i, input.getField(i)) + i += 1 + } + + //update the elements to be removed and retract them from aggregators + var iter = windowBuffer.get.iterator() + var continue:Boolean = true --- End diff -- @fhueske 2 points: -If we traverse the whole list of all elements - than what is the difference compared to having the window as before and going through the whole list to aggregate?...complexity would be the same... -There is no need to go through the whole list. The list is sorted based on processing time (i.e. based on incoming order of events). As we bound things based on time it means that we only need to go through the oldest elements until we find one which is still within the scope of the window. When we find this we can stop the search as we ensured we have only the right elements remaining in the buffer. Considering that the reason to switch from window to process function was to reduce the number of operations - i would say we need to keep this...otherwise it is basically the same and in the case i would ask you to merge the window implementation --- 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. ---