Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Test 0307 #48825

Open
wants to merge 5 commits into
base: branch-3.0
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 5 additions & 19 deletions fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
Original file line number Diff line number Diff line change
Expand Up @@ -228,6 +228,7 @@ public class Coordinator implements CoordInterface {
private final List<PipelineExecContext> needCheckPipelineExecContexts = Lists.newArrayList();
private List<ResultReceiver> receivers = Lists.newArrayList();
protected final List<ScanNode> scanNodes;
private ResultReceiverConsumer receiverConsumer;
private int scanRangeNum = 0;
// number of instances of this query, equals to
// number of backends executing plan fragments on behalf of this query;
Expand Down Expand Up @@ -279,8 +280,6 @@ public class Coordinator implements CoordInterface {

private StatsErrorEstimator statsErrorEstimator;

private int receiverOffset = 0;

// A countdown latch to mark the completion of each instance.
// use for old pipeline
// instance id -> dummy value
Expand Down Expand Up @@ -754,6 +753,7 @@ private void execInternal() throws Exception {
toArrowFlightHost(param.host), toBrpcHost(param.host), fragments.get(0).getOutputExprs()));
}
}
receiverConsumer = new ResultReceiverConsumer(receivers, timeoutDeadline);

LOG.info("dispatch result sink of query {} to {}", DebugUtil.printId(queryId),
topParams.instanceExecParams.get(0).host);
Expand Down Expand Up @@ -1161,10 +1161,8 @@ public RowBatch getNext() throws Exception {
throw new UserException("There is no receiver.");
}

RowBatch resultBatch;
Status status = new Status();
ResultReceiver receiver = receivers.get(receiverOffset);
resultBatch = receiver.getNext(status);
RowBatch resultBatch = receiverConsumer.getNext(status);
if (!status.ok()) {
LOG.warn("Query {} coordinator get next fail, {}, need cancel.",
DebugUtil.printId(queryId), status.getErrorMsg());
Expand Down Expand Up @@ -1208,20 +1206,8 @@ public RowBatch getNext() throws Exception {
boolean reachedLimit = LimitUtils.cancelIfReachLimit(
resultBatch, limitRows, numReceivedRows, this::cancelInternal);

if (resultBatch.isEos()) {
receivers.remove(receiver);
if (receivers.isEmpty()) {
returnedAllResults = true;
} else if (!reachedLimit) {
// if reachedLimit is true, which means this query has been cancelled.
// so no need to set eos to false again.
resultBatch.setEos(false);
}
}

if (!returnedAllResults) {
receiverOffset += 1;
receiverOffset %= receivers.size();
if (reachedLimit) {
resultBatch.setEos(true);
}
return resultBatch;
}
Expand Down
29 changes: 19 additions & 10 deletions fe/fe-core/src/main/java/org/apache/doris/qe/ResultReceiver.java
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import org.apache.doris.thrift.TStatusCode;
import org.apache.doris.thrift.TUniqueId;

import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.FutureCallback;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.thrift.TDeserializer;
Expand Down Expand Up @@ -76,20 +78,31 @@ Types.PUniqueId getRealFinstId() {
return finstId;
}

public void createFuture(
FutureCallback<InternalService.PFetchDataResult> callback) throws RpcException {
InternalService.PFetchDataRequest request = InternalService.PFetchDataRequest.newBuilder()
.setFinstId(getRealFinstId())
.setRespInAttachment(false)
.build();
try {
fetchDataAsyncFuture = BackendServiceProxy.getInstance().fetchDataAsyncWithCallback(address, request,
callback);
} catch (RpcException e) {
LOG.warn("fetch result rpc exception, finstId={}", DebugUtil.printId(finstId), e);
SimpleScheduler.addToBlacklist(backendId, e.getMessage());
throw e;
}
}

public RowBatch getNext(Status status) throws TException {
if (isDone) {
return null;
}
final RowBatch rowBatch = new RowBatch();
try {
while (!isDone && runStatus.ok()) {
InternalService.PFetchDataRequest request = InternalService.PFetchDataRequest.newBuilder()
.setFinstId(getRealFinstId())
.setRespInAttachment(false)
.build();

currentThread = Thread.currentThread();
fetchDataAsyncFuture = BackendServiceProxy.getInstance().fetchDataAsync(address, request);
Preconditions.checkNotNull(fetchDataAsyncFuture);
InternalService.PFetchDataResult pResult = null;

while (pResult == null) {
Expand Down Expand Up @@ -172,10 +185,6 @@ public RowBatch getNext(Status status) throws TException {
return rowBatch;
}
}
} catch (RpcException e) {
LOG.warn("fetch result rpc exception, finstId={}", DebugUtil.printId(finstId), e);
status.updateStatus(TStatusCode.THRIFT_RPC_ERROR, e.getMessage());
SimpleScheduler.addToBlacklist(backendId, e.getMessage());
} catch (ExecutionException e) {
LOG.warn("fetch result execution exception, finstId={}", DebugUtil.printId(finstId), e);
if (e.getMessage().contains("time out")) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
// 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.doris.qe;

import org.apache.doris.common.Status;
import org.apache.doris.common.UserException;
import org.apache.doris.proto.InternalService;
import org.apache.doris.rpc.RpcException;

import com.google.common.collect.Lists;
import com.google.common.util.concurrent.FutureCallback;
import org.apache.thrift.TException;

import java.util.List;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutionException;

public class ResultReceiverConsumer {
class ReceiverContext {
public ReceiverContext(ResultReceiver receiver, int offset) {
this.receiver = receiver;
this.offset = offset;
}

public void createFuture() {
if (errMsg != null) {
return;
}
try {
receiver.createFuture(new FutureCallback<InternalService.PFetchDataResult>() {
@Override
public void onSuccess(InternalService.PFetchDataResult result) {
readyOffsets.offer(offset);
}

@Override
public void onFailure(Throwable t) {
readyOffsets.offer(offset);
}
});
} catch (RpcException e) {
setErrMsg(e.getMessage());
readyOffsets.offer(offset);
}
}

ResultReceiver receiver;
final int offset;
}

private List<ReceiverContext> contexts = Lists.newArrayList();
private boolean futureInitialized = false;
private String errMsg;
private final long timeoutTs;

void setErrMsg(String errMsg) {
this.errMsg = errMsg;
}

BlockingQueue<Integer> readyOffsets;
int finishedReceivers = 0;

public ResultReceiverConsumer(List<ResultReceiver> resultReceivers, long timeoutDeadline) {
for (int i = 0; i < resultReceivers.size(); i++) {
ReceiverContext context = new ReceiverContext(resultReceivers.get(i), i);
contexts.add(context);
}
this.readyOffsets = new ArrayBlockingQueue<>(resultReceivers.size());
timeoutTs = timeoutDeadline;
}

public boolean isEos() {
return finishedReceivers == contexts.size();
}

public RowBatch getNext(Status status) throws TException, InterruptedException, ExecutionException, UserException {
if (!futureInitialized) {
futureInitialized = true;
for (ReceiverContext context : contexts) {
context.createFuture();
}
}

Integer offset = readyOffsets.poll(timeoutTs - System.currentTimeMillis(),
java.util.concurrent.TimeUnit.MILLISECONDS);
if (offset == null) {
throw new TException("query timeout");
}
if (errMsg != null) {
throw new UserException(errMsg);
}

ReceiverContext context = contexts.get(offset);
RowBatch rowBatch = context.receiver.getNext(status);
if (!status.ok() || rowBatch == null) {
return rowBatch;
}
if (rowBatch.isEos()) {
finishedReceivers++;
rowBatch.setEos(isEos());
} else {
context.createFuture();
}

return rowBatch;
}

public synchronized void cancel(Status reason) {
for (ReceiverContext context : contexts) {
context.receiver.cancel(reason);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,8 @@ public ListenableFuture<InternalService.PCancelPlanFragmentResult> cancelPlanFra
.cancelPlanFragment(request);
}

public Future<InternalService.PFetchDataResult> fetchDataAsync(InternalService.PFetchDataRequest request) {
public ListenableFuture<InternalService.PFetchDataResult> fetchDataAsync(
InternalService.PFetchDataRequest request) {
return stub.fetchData(request);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@
import org.apache.doris.thrift.TUniqueId;

import com.google.common.collect.Maps;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.protobuf.ByteString;
import org.apache.logging.log4j.LogManager;
Expand Down Expand Up @@ -295,6 +297,23 @@ public Future<InternalService.PFetchDataResult> fetchDataAsync(
}
}

public Future<InternalService.PFetchDataResult> fetchDataAsyncWithCallback(
TNetworkAddress address, InternalService.PFetchDataRequest request,
FutureCallback<InternalService.PFetchDataResult> callback) throws RpcException {
try {
final BackendServiceClient client = getProxy(address);
ListenableFuture<InternalService.PFetchDataResult> future = client.fetchDataAsync(request);
Futures.addCallback(
future, callback,
grpcThreadPool);
return future;
} catch (Throwable e) {
LOG.warn("fetch data catch a exception, address={}:{}",
address.getHostname(), address.getPort(), e);
throw new RpcException(address.hostname, e.getMessage());
}
}

public Future<InternalService.PTabletKeyLookupResponse> fetchTabletDataAsync(
TNetworkAddress address, InternalService.PTabletKeyLookupRequest request) throws RpcException {
try {
Expand Down
Loading
Loading