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

[fix](query-forward) Fix forward query exception or stuck or potential query result loss (#41303) #42368

Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -2471,4 +2471,9 @@ public class Config extends ConfigBase {
@ConfField(mutable = true, description = {
"设置为 true,root 和 admin 将跳过 sql block rule", "Set to true, root and admin will skip SQL block rule"})
public static boolean sql_block_rule_ignore_admin = false;

@ConfField(description = {"用于测试,强制将所有的查询forward到master以验证forward query的行为",
"For testing purposes, all queries are forcibly forwarded to the master to verify"
+ "the behavior of forwarding queries."})
public static boolean force_forward_all_queries = false;
}
Original file line number Diff line number Diff line change
Expand Up @@ -581,11 +581,8 @@ private void finalizeCommand() throws IOException {
&& ctx.getState().getStateType() != QueryState.MysqlStateType.ERR) {
ShowResultSet resultSet = executor.getShowResultSet();
if (resultSet == null) {
if (executor.sendProxyQueryResult()) {
packet = getResultPacket();
} else {
packet = executor.getOutputPacket();
}
executor.sendProxyQueryResult();
packet = executor.getOutputPacket();
} else {
executor.sendResultSet(resultSet);
packet = getResultPacket();
Expand Down Expand Up @@ -761,7 +758,12 @@ public TMasterOpResult proxyExecute(TMasterOpRequest request) throws TException
if (ctx.getState().getStateType() == MysqlStateType.OK) {
result.setStatusCode(0);
} else {
result.setStatusCode(ctx.getState().getErrorCode().getCode());
ErrorCode errorCode = ctx.getState().getErrorCode();
if (errorCode != null) {
result.setStatusCode(errorCode.getCode());
} else {
result.setStatusCode(ErrorCode.ERR_UNKNOWN_ERROR.getCode());
}
result.setErrMessage(ctx.getState().getErrorMessage());
}
if (executor != null) {
Expand Down
20 changes: 12 additions & 8 deletions fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
Original file line number Diff line number Diff line change
Expand Up @@ -253,6 +253,7 @@ public class StmtExecutor {
private boolean isExecuteStmt = false;
// The profile of this execution
private final Profile profile;
private Boolean isForwardedToMaster = null;

// The result schema if "dry_run_query" is true.
// Only one column to indicate the real return row numbers.
Expand Down Expand Up @@ -372,13 +373,20 @@ public void setPlanner(Planner planner) {
}

public boolean isForwardToMaster() {
if (isForwardedToMaster == null) {
isForwardedToMaster = shouldForwardToMaster();
}
return isForwardedToMaster;
}

private boolean shouldForwardToMaster() {
if (Env.getCurrentEnv().isMaster()) {
return false;
}

// this is a query stmt, but this non-master FE can not read, forward it to master
if (isQuery() && !Env.getCurrentEnv().isMaster()
&& (!Env.getCurrentEnv().canRead() || debugForwardAllQueries())) {
&& (!Env.getCurrentEnv().canRead() || debugForwardAllQueries() || Config.force_forward_all_queries)) {
return true;
}

Expand All @@ -391,7 +399,7 @@ public boolean isForwardToMaster() {

private boolean debugForwardAllQueries() {
DebugPoint debugPoint = DebugPointUtil.getDebugPoint("StmtExecutor.forward_all_queries");
return debugPoint != null && debugPoint.param("forwardAllQueries", true);
return debugPoint != null && debugPoint.param("forwardAllQueries", false);
}

public ByteBuffer getOutputPacket() {
Expand Down Expand Up @@ -2905,18 +2913,14 @@ public List<ByteBuffer> getProxyQueryResultBufList() {
return ((ProxyMysqlChannel) context.getMysqlChannel()).getProxyResultBufferList();
}

public boolean sendProxyQueryResult() throws IOException {
public void sendProxyQueryResult() throws IOException {
if (masterOpExecutor == null) {
return false;
return;
}
List<ByteBuffer> queryResultBufList = masterOpExecutor.getQueryResultBufList();
if (queryResultBufList.isEmpty()) {
return false;
}
for (ByteBuffer byteBuffer : queryResultBufList) {
context.getMysqlChannel().sendOnePacket(byteBuffer);
}
return true;
}
}

Expand Down
Loading