flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] wuchong commented on a change in pull request #8244: [FLINK-11945] [table-runtime-blink] Support over aggregation for blink streaming runtime
Date Sun, 05 May 2019 13:53:07 GMT
wuchong commented on a change in pull request #8244: [FLINK-11945] [table-runtime-blink] Support
over aggregation for blink streaming runtime
URL: https://github.com/apache/flink/pull/8244#discussion_r281020930
 
 

 ##########
 File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.java
 ##########
 @@ -0,0 +1,221 @@
+/*
+ * 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.MapState;
+import org.apache.flink.api.common.state.MapStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.typeutils.ListTypeInfo;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.JoinedRow;
+import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.generated.AggsHandleFunction;
+import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.type.InternalType;
+import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.util.Collector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Process Function used for the aggregate in bounded proc-time OVER window.
+ */
+public class ProcTimeBoundedRangeOver<K>
+	extends ProcessFunctionWithCleanupState<K, BaseRow, BaseRow> {
+	private static final Logger LOG = LoggerFactory.getLogger(ProcTimeBoundedRangeOver.class);
+
+	private GeneratedAggsHandleFunction genAggsHandler;
+	private InternalType[] accTypes;
+	private InternalType[] inputFieldTypes;
+	private Long precedingTimeBoundary;
+
+	private ValueState<BaseRow> accState;
+	private MapState<Long, List<BaseRow>> inputState;
+
+	private AggsHandleFunction function;
+	private JoinedRow output;
+
+	public ProcTimeBoundedRangeOver(
+		GeneratedAggsHandleFunction genAggsHandler,
+		InternalType[] accTypes,
+		InternalType[] inputFieldTypes,
+		Long precedingTimeBoundary,
+		TableConfig tableConfig) {
+		super(tableConfig);
+		this.genAggsHandler = genAggsHandler;
+		this.accTypes = accTypes;
+		this.inputFieldTypes = inputFieldTypes;
+		this.precedingTimeBoundary = precedingTimeBoundary;
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		LOG.debug("Compiling AggregateHelper: " + genAggsHandler.getClassName() + " \n\n" +
+			"Code:\n" + genAggsHandler.getCode());
+		function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader());
+		function.open(new PerKeyStateDataViewStore(getRuntimeContext()));
+
+		output = new JoinedRow();
+
+		// input element are all binary row as they are came from network
+		BaseRowTypeInfo inputType = new BaseRowTypeInfo(inputFieldTypes);
+		// we keep the elements received in a map state indexed based on their ingestion time
+		ListTypeInfo<BaseRow> rowListTypeInfo = new ListTypeInfo<BaseRow>(inputType);
+		MapStateDescriptor<Long, List<BaseRow>> mapStateDescriptor = new MapStateDescriptor<Long,
List<BaseRow>>(
+			"inputState", BasicTypeInfo.LONG_TYPE_INFO, rowListTypeInfo);
+		inputState = getRuntimeContext().getMapState(mapStateDescriptor);
+
+		BaseRowTypeInfo accTypeInfo = new BaseRowTypeInfo(accTypes);
+		ValueStateDescriptor<BaseRow> stateDescriptor =
+			new ValueStateDescriptor<BaseRow>("accState", accTypeInfo);
+		accState = getRuntimeContext().getState(stateDescriptor);
+
+		initCleanupTimeState("ProcTimeBoundedRangeOverCleanupTime");
+	}
+
+	@Override
+	public void processElement(
+		BaseRow input,
+		KeyedProcessFunction<K, BaseRow, BaseRow>.Context ctx,
+		Collector<BaseRow> out) throws Exception {
+		Long currentTime = ctx.timerService().currentProcessingTime();
+		// register state-cleanup timer
+		registerProcessingCleanupTimer(ctx, currentTime);
+
+		// buffer the event incoming event
+
+		// add current element to the window list of elements with corresponding timestamp
+		List<BaseRow> rowList = inputState.get(currentTime);
+		// null value means that this si the first event received for this timestamp
+		if (rowList == null) {
+			rowList = new ArrayList<BaseRow>();
+			// register timer to process event once the current millisecond passed
+			ctx.timerService().registerProcessingTimeTimer(currentTime + 1);
+		}
+		rowList.add(input);
+		inputState.put(currentTime, rowList);
+	}
+
+	@Override
+	public void onTimer(
+		long timestamp,
+		KeyedProcessFunction<K, BaseRow, BaseRow>.OnTimerContext ctx,
+		Collector<BaseRow> out) throws Exception {
+		if (needToCleanupState(timestamp)) {
+			// clean up and return
+			cleanupState(inputState, accState);
+			function.cleanup();
+			return;
+		}
+
+		// remove timestamp set outside of ProcessFunction.
+		((TimestampedCollector) out).eraseTimestamp();
+
+		// we consider the original timestamp of events
+		// that have registered this time trigger 1 ms ago
+
+		long currentTime = timestamp - 1;
+		// get the list of elements of current proctime
+		List<BaseRow> currentElements = inputState.get(currentTime);
+
+		// Expired clean-up timers pass the needToCleanupState check.
+		// Perform a null check to verify that we have data to process.
+		if (null == currentElements) {
+			return;
+		}
+
+		// initialize the accumulators
+		BaseRow accumulators = accState.value();
+
+		if (null == accumulators) {
+			accumulators = function.createAccumulators();
+		}
+
+//		// set accumulators in context first
+//		function.setAccumulators(accumulators);
+
+		// update the elements to be removed and retract them from aggregators
+		long limit = currentTime - precedingTimeBoundary;
+
+		// we iterate through all elements in the window buffer based on timestamp keys
+		// when we find timestamps that are out of interest, we retrieve corresponding elements
+		// and eliminate them. Multiple elements could have been received at the same timestamp
+		// the removal of old elements happens only once per proctime as onTimer is called only
once
+		Iterator<Map.Entry<Long, List<BaseRow>>> iter = inputState.iterator();
+		while (iter.hasNext()) {
+			Map.Entry<Long, List<BaseRow>> entry = iter.next();
+			Long elementKey = entry.getKey();
+			if (elementKey < limit) {
+				// element key outside of window. Retract values
+				List<BaseRow> elementsRemove = entry.getValue();
+				int iRemove = 0;
+				while (iRemove < elementsRemove.size()) {
+					BaseRow retractRow = elementsRemove.get(iRemove);
+					function.retract(retractRow);
+					iRemove += 1;
+				}
+				iter.remove();
+			}
+		}
+
+		// set accumulators in context first
+		function.setAccumulators(accumulators);
 
 Review comment:
   I think `setAccumulators` should happen before the while iteration.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message