TsReaper commented on a change in pull request #12073: URL: https://github.com/apache/flink/pull/12073#discussion_r426230954
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectResultFetcher.java ########## @@ -0,0 +1,359 @@ +/* + * 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.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.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; + + @Nullable + private JobClient jobClient; + @Nullable + private CoordinationRequestGateway gateway; + + private boolean jobTerminated; + private boolean closed; + + public CollectResultFetcher( + CompletableFuture<OperatorID> operatorIdFuture, + TypeSerializer<T> serializer, + String accumulatorName) { + this( + operatorIdFuture, + serializer, + accumulatorName, + DEFAULT_RETRY_MILLIS); + } + + 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.jobTerminated = false; + this.closed = false; + } + + public void setJobClient(JobClient jobClient) { + Preconditions.checkArgument( + jobClient instanceof CoordinationRequestGateway, + "Job client must be a CoordinationRequestGateway. This is a bug."); + this.jobClient = jobClient; + this.gateway = (CoordinationRequestGateway) 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 (jobTerminated) { + // 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 + jobTerminated = 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 (jobTerminated) { + // still no results, but job has terminated, we have to return + return null; + } else { Review comment: This will cause iterator to sleep before the first try, which is undesired. ---------------------------------------------------------------- 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: us...@infra.apache.org