You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/17 04:46:54 UTC

[GitHub] [flink] TsReaper commented on a change in pull request #12073: [FLINK-17735][streaming] Add specialized collecting iterator

TsReaper commented on a change in pull request #12073:
URL: https://github.com/apache/flink/pull/12073#discussion_r426216888



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectResultFetcher.java
##########
@@ -0,0 +1,343 @@
+/*
+ * 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.streaming.api.operators.collect;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.accumulators.SerializedListAccumulator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestGateway;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * A fetcher which fetches query results from sink and provides exactly-once semantics.
+ */
+public class CollectResultFetcher<T> {
+
+	private static final int DEFAULT_RETRY_MILLIS = 100;
+	private static final long DEFAULT_ACCUMULATOR_GET_MILLIS = 10000;
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectResultFetcher.class);
+
+	private final CompletableFuture<OperatorID> operatorIdFuture;
+	private final String accumulatorName;
+	private final int retryMillis;
+
+	private ResultBuffer buffer;
+
+	private JobClient jobClient;
+	private boolean terminated;
+	private boolean closed;
+
+	public CollectResultFetcher(
+			CompletableFuture<OperatorID> operatorIdFuture,
+			TypeSerializer<T> serializer,
+			String accumulatorName) {
+		this(
+			operatorIdFuture,
+			serializer,
+			accumulatorName,
+			DEFAULT_RETRY_MILLIS);
+	}
+
+	@VisibleForTesting
+	public CollectResultFetcher(
+			CompletableFuture<OperatorID> operatorIdFuture,
+			TypeSerializer<T> serializer,
+			String accumulatorName,
+			int retryMillis) {
+		this.operatorIdFuture = operatorIdFuture;
+		this.accumulatorName = accumulatorName;
+		this.retryMillis = retryMillis;
+
+		this.buffer = new ResultBuffer(serializer);
+
+		this.terminated = false;
+	}
+
+	public void setJobClient(JobClient jobClient) {
+		Preconditions.checkArgument(
+			jobClient instanceof CoordinationRequestGateway,
+			"Job client must be a CoordinationRequestGateway. This is a bug.");
+		this.jobClient = jobClient;
+	}
+
+	@SuppressWarnings("unchecked")
+	public T next() {
+		if (closed) {
+			return null;
+		}
+
+		T res = buffer.next();
+		if (res != null) {
+			// we still have user-visible results, just use them
+			return res;
+		} else if (terminated) {
+			// no user-visible results, but job has terminated, we have to return
+			return null;
+		}
+
+		// we're going to fetch some more
+		while (true) {
+			if (isJobTerminated()) {
+				// job terminated, read results from accumulator
+				terminated = true;
+				Tuple2<Long, CollectCoordinationResponse> accResults = getAccumulatorResults();
+				if (accResults != null) {
+					buffer.dealWithResponse(accResults.f1, accResults.f0);
+				}
+				buffer.complete();
+			} else {
+				// job still running, try to fetch some results
+				CollectCoordinationResponse<T> response;
+				try {
+					response = sendRequest(buffer.version, buffer.offset);
+				} catch (Exception e) {
+					LOG.warn("An exception occurs when fetching query results", e);
+					sleepBeforeRetry();
+					continue;
+				}
+				buffer.dealWithResponse(response);
+			}
+
+			// try to return results after fetching
+			res = buffer.next();
+			if (res != null) {
+				// ok, we have results this time
+				return res;
+			} else if (terminated) {
+				// still no results, but job has terminated, we have to return
+				return null;
+			} else {
+				// still no results, but job is still running, retry
+				sleepBeforeRetry();
+			}
+		}
+	}
+
+	public void close() {
+		if (closed) {
+			return;
+		}
+
+		cancelJob();
+		closed = true;
+	}
+
+	@Override
+	protected void finalize() throws Throwable {
+		// in case that user neither reads all data nor closes the iterator
+		close();
+	}
+
+	@SuppressWarnings("unchecked")
+	private CollectCoordinationResponse<T> sendRequest(
+			String version,
+			long offset) throws InterruptedException, ExecutionException {
+		checkJobClientConfigured();
+		CoordinationRequestGateway gateway = (CoordinationRequestGateway) jobClient;
+
+		OperatorID operatorId = operatorIdFuture.getNow(null);
+		Preconditions.checkNotNull(operatorId, "Unknown operator ID. This is a bug.");
+
+		CollectCoordinationRequest request = new CollectCoordinationRequest(version, offset);
+		return (CollectCoordinationResponse) gateway.sendCoordinationRequest(operatorId, request).get();
+	}
+
+	@Nullable
+	private Tuple2<Long, CollectCoordinationResponse> getAccumulatorResults() {
+		checkJobClientConfigured();
+
+		JobExecutionResult executionResult;
+		try {
+			// this timeout is sort of hack, see comments in isJobTerminated for explanation
+			executionResult = jobClient.getJobExecutionResult(getClass().getClassLoader()).get(
+				DEFAULT_ACCUMULATOR_GET_MILLIS, TimeUnit.MILLISECONDS);
+		} catch (InterruptedException | ExecutionException | TimeoutException e) {
+			throw new RuntimeException("Failed to fetch job execution result", e);
+		}
+
+		ArrayList<byte[]> accResults = executionResult.getAccumulatorResult(accumulatorName);
+		if (accResults == null) {
+			// job terminates abnormally
+			return null;
+		}
+
+		try {
+			List<byte[]> serializedResults =
+				SerializedListAccumulator.deserializeList(accResults, BytePrimitiveArraySerializer.INSTANCE);
+			byte[] serializedResult = serializedResults.get(0);
+			return CollectSinkFunction.deserializeAccumulatorResult(serializedResult);
+		} catch (ClassNotFoundException | IOException e) {
+			// this is impossible
+			throw new RuntimeException("Failed to deserialize accumulator results", e);
+		}
+	}
+
+	private boolean isJobTerminated() {
+		checkJobClientConfigured();
+
+		try {
+			JobStatus status = jobClient.getJobStatus().get();
+			return status.isGloballyTerminalState();
+		} catch (Exception e) {
+			// TODO
+			//  This is sort of hack.
+			//  Currently different execution environment will have different behaviors
+			//  when fetching a finished job status.
+			//  For example, standalone session cluster will return a normal FINISHED,
+			//  while mini cluster will throw IllegalStateException,
+			//  and yarn per job will throw ApplicationNotFoundException.
+			//  We have to assume that job has finished in this case.
+			//  Change this when these behaviors are unified.
+			LOG.warn("Failed to get job status so we assume that the job has terminated. Some data might be lost.", e);
+			return true;
+		}
+	}
+
+	private void cancelJob() {
+		checkJobClientConfigured();
+
+		if (!isJobTerminated()) {
+			jobClient.cancel();
+		}
+	}
+
+	private void sleepBeforeRetry() {
+		if (retryMillis <= 0) {
+			return;
+		}
+
+		try {
+			// TODO a more proper retry strategy?
+			Thread.sleep(retryMillis);
+		} catch (InterruptedException e) {
+			LOG.warn("Interrupted when sleeping before a retry", e);
+		}
+	}
+
+	private void checkJobClientConfigured() {
+		Preconditions.checkNotNull(jobClient, "Job client must be configured before first use.");
+	}
+
+	private class ResultBuffer {
+
+		private static final String INIT_VERSION = "";
+
+		private final LinkedList<T> buffer;
+		private final TypeSerializer<T> serializer;
+
+		private String version;
+		private long offset;
+		private long lastCheckpointedOffset;
+		private long userHead;
+		private long userTail;
+
+		private ResultBuffer(TypeSerializer<T> serializer) {
+			this.buffer = new LinkedList<>();
+			this.serializer = serializer;
+
+			this.version = INIT_VERSION;
+			this.offset = 0;
+			this.lastCheckpointedOffset = 0;
+			this.userHead = 0;
+			this.userTail = 0;
+		}
+
+		private T next() {
+			if (userHead == userTail) {
+				return null;
+			}
+			T ret = buffer.removeFirst();
+			userHead++;
+
+			sanityCheck();
+			return ret;
+		}
+
+		private void dealWithResponse(CollectCoordinationResponse<T> response) {
+			dealWithResponse(response, offset);
+		}
+
+		private void dealWithResponse(CollectCoordinationResponse<T> response, long responseOffset) {
+			String responseVersion = response.getVersion();
+			long responseLastCheckpointedOffset = response.getLastCheckpointedOffset();
+			List<T> results;
+			try {
+				results = response.getResults(serializer);
+			} catch (IOException e) {
+				LOG.warn("An exception occurs when deserializing query results. Some results might be lost.", e);
+				results = Collections.emptyList();
+			}
+
+			if (responseLastCheckpointedOffset > lastCheckpointedOffset) {

Review comment:
       It is impossible for `responseLastCheckpointedOffset` to be smaller than `lastCheckpointedOffset`, offsets are always non-decreasing.




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