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:08 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_r281021766
 
 

 ##########
 File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.java
 ##########
 @@ -0,0 +1,292 @@
+/*
+ * 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.Types;
+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.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.apache.flink.util.Preconditions;
+
+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 for ROWS clause event-time bounded OVER window.
+ */
+public class RowTimeBoundedRowsOver<K> extends
+	ProcessFunctionWithCleanupState<K, BaseRow, BaseRow> {
+	private static final Logger LOG = LoggerFactory.getLogger(RowTimeBoundedRowsOver.class);
+
+	private GeneratedAggsHandleFunction genAggsHandler;
+	private InternalType[] accTypes;
+	private InternalType[] inputFieldTypes;
+	private Long precedingOffset;
+	private int rowTimeIdx;
+
+	private JoinedRow output;
+
+	// the state which keeps the last triggering timestamp
+	private ValueState<Long> lastTriggeringTsState;
+
+	// the state which keeps the count of data
+	private ValueState<Long> counterState;
+
+	// the state which used to materialize the accumulator for incremental calculation
+	private ValueState<BaseRow> accState;
+
+	// the state which keeps all the data that are not expired.
+	// The first element (as the mapState key) of the tuple is the time stamp. Per each time
stamp,
+	// the second element of tuple is a list that contains the entire data of all the rows belonging
+	// to this time stamp.
+	private MapState<Long, List<BaseRow>> inputState;
+
+	private AggsHandleFunction function;
+
+	public RowTimeBoundedRowsOver(
+		GeneratedAggsHandleFunction genAggsHandler,
+		InternalType[] accTypes,
+		InternalType[] inputFieldTypes,
+		Long precedingOffset,
+		int rowTimeIdx,
+		TableConfig tableConfig) {
+		super(tableConfig);
+		Preconditions.checkNotNull(precedingOffset);
+		this.genAggsHandler = genAggsHandler;
+		this.accTypes = accTypes;
+		this.inputFieldTypes = inputFieldTypes;
+		this.precedingOffset = precedingOffset;
+		this.rowTimeIdx = rowTimeIdx;
+	}
+
+	@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();
+
+		ValueStateDescriptor<Long> lastTriggeringTsDescriptor = new ValueStateDescriptor<Long>(
+			"lastTriggeringTsState",
+			Types.LONG);
+		lastTriggeringTsState = getRuntimeContext().getState(lastTriggeringTsDescriptor);
+
+		ValueStateDescriptor<Long> dataCountStateDescriptor = new ValueStateDescriptor<Long>(
+			"processedCountState",
+			Types.LONG);
+		counterState = getRuntimeContext().getState(dataCountStateDescriptor);
+
+		BaseRowTypeInfo accTypeInfo = new BaseRowTypeInfo(accTypes);
+		ValueStateDescriptor<BaseRow> accStateDesc = new ValueStateDescriptor<BaseRow>("accState",
accTypeInfo);
+		accState = getRuntimeContext().getState(accStateDesc);
+
+		// input element are all binary row as they are came from network
+		BaseRowTypeInfo inputType = new BaseRowTypeInfo(inputFieldTypes);
+		ListTypeInfo<BaseRow> rowListTypeInfo = new ListTypeInfo<BaseRow>(inputType);
+		MapStateDescriptor<Long, List<BaseRow>> inputStateDesc = new MapStateDescriptor<Long,
List<BaseRow>>(
+			"inputState",
+			Types.LONG,
+			rowListTypeInfo);
+		inputState = getRuntimeContext().getMapState(inputStateDesc);
+
+		initCleanupTimeState("RowTimeBoundedRowsOverCleanupTime");
+	}
+
+	@Override
+	public void processElement(
+		BaseRow input,
+		KeyedProcessFunction<K, BaseRow, BaseRow>.Context ctx,
+		Collector<BaseRow> out) throws Exception {
+		// register state-cleanup timer
+		registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime());
+
+		// triggering timestamp for trigger calculation
+		Long triggeringTs = input.getLong(rowTimeIdx);
+
+		Long lastTriggeringTs = lastTriggeringTsState.value();
+		if (lastTriggeringTs == null) {
+			lastTriggeringTs = 0L;
+		}
+
+		// check if the data is expired, if not, save the data and register event time timer
+		if (triggeringTs > lastTriggeringTs) {
+			List<BaseRow> data;
+			data = inputState.get(triggeringTs);
+			if (null != data) {
+				data.add(input);
+				inputState.put(triggeringTs, data);
+			} else {
+				data = new ArrayList<BaseRow>();
+				data.add(input);
+				inputState.put(triggeringTs, data);
+				// register event time timer
+				ctx.timerService().registerEventTimeTimer(triggeringTs);
+			}
+		}
+	}
+
+	@Override
+	public void onTimer(
+		long timestamp,
+		KeyedProcessFunction<K, BaseRow, BaseRow>.OnTimerContext ctx,
+		Collector<BaseRow> out) throws Exception {
+		if (isProcessingTimeTimer(ctx)) {
+			if (needToCleanupState(timestamp)) {
+
+				Iterator<Map.Entry<Long, List<BaseRow>>> keysIt = inputState.iterator();
+				Long lastProcessedTime = lastTriggeringTsState.value();
+				if (lastProcessedTime == null) {
+					lastProcessedTime = 0L;
+				}
+
+				// is data left which has not been processed yet?
+				Boolean noRecordsToProcess = true;
+				while (keysIt.hasNext() && noRecordsToProcess) {
+					if (keysIt.next().getKey() > lastProcessedTime) {
+						noRecordsToProcess = false;
+					}
+				}
+
+				if (noRecordsToProcess) {
+					// We clean the state
+					cleanupState(inputState, accState, counterState, lastTriggeringTsState);
+					function.cleanup();
+				} else {
+					// There are records left to process because a watermark has not been received yet.
+					// This would only happen if the input stream has stopped. So we don't need to clean
up.
+					// We leave the state as it is and schedule a new cleanup timer
+					registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime());
+				}
+			}
+			return;
+		}
+
+		// gets all window data from state for the calculation
+		List<BaseRow> inputs = inputState.get(timestamp);
+
+		if (null != inputs) {
+
+			Long dataCount = counterState.value();
+			if (dataCount == null) {
+				dataCount = 0L;
+			}
+
+			BaseRow accumulators = accState.value();
+			if (accumulators == null) {
+				accumulators = function.createAccumulators();
+			}
+			// set accumulators in context first
+			function.setAccumulators(accumulators);
+
+			List<BaseRow> retractList = null;
+			Long retractTs = Long.MAX_VALUE;
+			int retractCnt = 0;
+			int i = 0;
+
+			while (i < inputs.size()) {
+				BaseRow input = inputs.get(i);
+				BaseRow retractRow = null;
+				if (dataCount >= precedingOffset) {
+					if (null == retractList) {
+						// find the smallest timestamp
+						retractTs = Long.MAX_VALUE;
+						Iterator<Map.Entry<Long, List<BaseRow>>> dataTimestampIt = inputState.iterator();
+						while (dataTimestampIt.hasNext()) {
+							Map.Entry<Long, List<BaseRow>> data = dataTimestampIt.next();
+							Long dataTs = data.getKey();
+							if (dataTs < retractTs) {
+								retractTs = dataTs;
+								// get the oldest rows to retract them
+								retractList = data.getValue();
+							}
+						}
+					}
+
+					retractRow = retractList.get(retractCnt);
+					retractCnt += 1;
+
+					// remove retracted values from state
+					if (retractList.size() == retractCnt) {
+						inputState.remove(retractTs);
+						retractList = null;
+						retractCnt = 0;
+					}
+				} else {
+					dataCount += 1;
+				}
+
+				// retract old row from accumulators
+				if (null != retractRow) {
+					function.retract(retractRow);
+				}
+
+				// accumulate current row
+				function.accumulate(input);
+
+				// prepare output row
+				output.replace(input, function.getValue());
+				out.collect(output);
+
+				i += 1;
+			}
+
+			// update all states
+			if (inputState.contains(retractTs)) {
+				if (retractCnt > 0) {
+					retractList.subList(0, retractCnt).clear();
+					inputState.put(retractTs, retractList);
+				}
+			}
+			counterState.update(dataCount);
+			// update the value of accumulators for future incremental computation
+			accumulators = function.getAccumulators();
+			accState.update(accumulators);
+		}
+
+		lastTriggeringTsState.update(timestamp);
+
+		// update cleanup timer
+		registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime());
+	}
+
+	@Override
+	public void close() throws Exception {
+		function.close();
 
 Review comment:
   close function only if function is not null.

----------------------------------------------------------------
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