[chore](log) Add log to trace query execution #25739
This commit is contained in:
@ -31,6 +31,7 @@ import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@ -63,6 +64,8 @@ public class ExecutionProfile {
|
||||
// instance id -> dummy value
|
||||
private MarkedCountDownLatch<TUniqueId, Long> profileDoneSignal;
|
||||
|
||||
private int waitCount = 0;
|
||||
|
||||
private TUniqueId queryId;
|
||||
|
||||
public ExecutionProfile(TUniqueId queryId, int fragmentNum) {
|
||||
@ -127,7 +130,11 @@ public class ExecutionProfile {
|
||||
|
||||
public void markOneInstanceDone(TUniqueId fragmentInstanceId) {
|
||||
if (profileDoneSignal != null) {
|
||||
profileDoneSignal.markedCountDown(fragmentInstanceId, -1L);
|
||||
if (profileDoneSignal.markedCountDown(fragmentInstanceId, -1L)) {
|
||||
LOG.info("Mark instance {} done succeed", DebugUtil.printId(fragmentInstanceId));
|
||||
} else {
|
||||
LOG.warn("Mark instance {} done failed", DebugUtil.printId(fragmentInstanceId));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -135,6 +142,16 @@ public class ExecutionProfile {
|
||||
if (profileDoneSignal == null) {
|
||||
return true;
|
||||
}
|
||||
|
||||
waitCount++;
|
||||
|
||||
for (Entry<TUniqueId, Long> entry : profileDoneSignal.getLeftMarks()) {
|
||||
if (waitCount > 2) {
|
||||
LOG.info("Query {} waiting instance {}, waitCount: {}",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(entry.getKey()), waitCount);
|
||||
}
|
||||
}
|
||||
|
||||
return profileDoneSignal.await(waitTimeS, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
|
||||
@ -105,7 +105,7 @@ public class Transaction {
|
||||
|
||||
coordinator.exec();
|
||||
int execTimeout = ctx.getExecTimeout();
|
||||
LOG.debug("Insert execution timeout:{}", execTimeout);
|
||||
LOG.info("Insert {} execution timeout:{}", DebugUtil.printId(ctx.queryId()), execTimeout);
|
||||
boolean notTimeout = coordinator.join(execTimeout);
|
||||
if (!coordinator.isDone()) {
|
||||
coordinator.cancel();
|
||||
|
||||
@ -530,13 +530,18 @@ public class Coordinator implements CoordInterface {
|
||||
|
||||
this.idToBackend = Env.getCurrentSystemInfo().getIdToBackend();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Query {} idToBackend size={}", DebugUtil.printId(queryId), idToBackend.size());
|
||||
int backendNum = idToBackend.size();
|
||||
StringBuilder backendInfos = new StringBuilder("backends info:");
|
||||
for (Map.Entry<Long, Backend> entry : idToBackend.entrySet()) {
|
||||
Long backendID = entry.getKey();
|
||||
Backend backend = entry.getValue();
|
||||
LOG.debug("Query {}, backend: {}-{}-{}-{}", DebugUtil.printId(queryId),
|
||||
backendID, backend.getHost(), backend.getBePort(), backend.getProcessEpoch());
|
||||
backendInfos.append(' ').append(backendID).append("-")
|
||||
.append(backend.getHost()).append("-")
|
||||
.append(backend.getBePort()).append("-")
|
||||
.append(backend.getProcessEpoch());
|
||||
}
|
||||
LOG.debug("query {}, backend size: {}, {}",
|
||||
DebugUtil.printId(queryId), backendNum, backendInfos.toString());
|
||||
}
|
||||
}
|
||||
|
||||
@ -631,7 +636,7 @@ public class Coordinator implements CoordInterface {
|
||||
resultInternalServiceAddr = toBrpcHost(execBeAddr);
|
||||
resultOutputExprs = fragments.get(0).getOutputExprs();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("dispatch query job: {} to {}", DebugUtil.printId(queryId),
|
||||
LOG.debug("dispatch result sink of query {} to {}", DebugUtil.printId(queryId),
|
||||
topParams.instanceExecParams.get(0).host);
|
||||
}
|
||||
|
||||
@ -857,7 +862,8 @@ public class Coordinator implements CoordInterface {
|
||||
}
|
||||
}
|
||||
|
||||
// 3. group BackendExecState by BE. So that we can use one RPC to send all fragment instances of a BE.
|
||||
// 3. group PipelineExecContext by BE.
|
||||
// So that we can use one RPC to send all fragment instances of a BE.
|
||||
for (Map.Entry<TNetworkAddress, TPipelineFragmentParams> entry : tParams.entrySet()) {
|
||||
Long backendId = this.addressToBackendID.get(entry.getKey());
|
||||
PipelineExecContext pipelineExecContext = new PipelineExecContext(fragment.getFragmentId(),
|
||||
@ -905,6 +911,16 @@ public class Coordinator implements CoordInterface {
|
||||
span = ConnectContext.get().getTracer().spanBuilder("execRemoteFragmentsAsync")
|
||||
.setParent(parentSpanContext).setSpanKind(SpanKind.CLIENT).startSpan();
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
String infos = "";
|
||||
for (PipelineExecContext pec : ctxs.ctxs) {
|
||||
infos += pec.fragmentId + " ";
|
||||
}
|
||||
LOG.debug("query {}, sending pipeline fragments: {} to be {} bprc address {}",
|
||||
DebugUtil.printId(queryId), infos, ctxs.beId, ctxs.brpcAddr.toString());
|
||||
}
|
||||
|
||||
ctxs.scopedSpan = new ScopedSpan(span);
|
||||
ctxs.unsetFields();
|
||||
BackendServiceProxy proxy = BackendServiceProxy.getInstance();
|
||||
@ -2415,9 +2431,12 @@ public class Coordinator implements CoordInterface {
|
||||
if (params.isSetErrorTabletInfos()) {
|
||||
updateErrorTabletInfos(params.getErrorTabletInfos());
|
||||
}
|
||||
|
||||
Preconditions.checkArgument(params.isSetDetailedReport());
|
||||
for (TDetailedReportParams param : params.detailed_report) {
|
||||
if (ctx.fragmentInstancesMap.get(param.fragment_instance_id).getIsDone()) {
|
||||
LOG.debug("Query {} instance {} is marked done",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()));
|
||||
executionProfile.markOneInstanceDone(param.getFragmentInstanceId());
|
||||
}
|
||||
}
|
||||
@ -2442,9 +2461,11 @@ public class Coordinator implements CoordInterface {
|
||||
// and returned_all_results_ is true.
|
||||
// (UpdateStatus() initiates cancellation, if it hasn't already been initiated)
|
||||
if (!(returnedAllResults && status.isCancelled()) && !status.ok()) {
|
||||
LOG.warn("one instance report fail, query_id={} instance_id={}, error message: {}",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()),
|
||||
status.getErrorMsg());
|
||||
LOG.warn("one instance report fail, query_id={} fragment_id={} instance_id={}, be={},"
|
||||
+ " error message: {}",
|
||||
DebugUtil.printId(queryId), params.getFragmentId(),
|
||||
DebugUtil.printId(params.getFragmentInstanceId()),
|
||||
params.getBackendId(), status.getErrorMsg());
|
||||
updateStatus(status, params.getFragmentInstanceId());
|
||||
}
|
||||
if (ctx.fragmentInstancesMap.get(params.fragment_instance_id).getIsDone()) {
|
||||
@ -2466,11 +2487,17 @@ public class Coordinator implements CoordInterface {
|
||||
if (params.isSetErrorTabletInfos()) {
|
||||
updateErrorTabletInfos(params.getErrorTabletInfos());
|
||||
}
|
||||
LOG.debug("Query {} instance {} is marked done",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()));
|
||||
executionProfile.markOneInstanceDone(params.getFragmentInstanceId());
|
||||
} else {
|
||||
LOG.debug("Query {} instance {} is not marked done",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()));
|
||||
}
|
||||
} else {
|
||||
if (params.backend_num >= backendExecStates.size()) {
|
||||
LOG.warn("unknown backend number: {}, expected less than: {}",
|
||||
LOG.warn("Query {} instance {} unknown backend number: {}, expected less than: {}",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()),
|
||||
params.backend_num, backendExecStates.size());
|
||||
return;
|
||||
}
|
||||
@ -2518,7 +2545,12 @@ public class Coordinator implements CoordInterface {
|
||||
if (params.isSetErrorTabletInfos()) {
|
||||
updateErrorTabletInfos(params.getErrorTabletInfos());
|
||||
}
|
||||
LOG.info("Query {} instance {} is marked done",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()));
|
||||
executionProfile.markOneInstanceDone(params.getFragmentInstanceId());
|
||||
} else {
|
||||
LOG.info("Query {} instance {} is not marked done",
|
||||
DebugUtil.printId(queryId), DebugUtil.printId(params.getFragmentInstanceId()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -3121,9 +3153,10 @@ public class Coordinator implements CoordInterface {
|
||||
for (TPipelineInstanceParams localParam : rpcParams.local_params) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("cancelRemoteFragments initiated={} done={} hasCanceled={} backend: {},"
|
||||
+ " fragment instance id={}, reason: {}",
|
||||
+ " fragment instance id={} query={}, reason: {}",
|
||||
this.initiated, this.done, this.hasCanceled, backend.getId(),
|
||||
DebugUtil.printId(localParam.fragment_instance_id), cancelReason.name());
|
||||
DebugUtil.printId(localParam.fragment_instance_id),
|
||||
DebugUtil.printId(queryId), cancelReason.name());
|
||||
}
|
||||
|
||||
RuntimeProfile profile = fragmentInstancesMap.get(localParam.fragment_instance_id);
|
||||
|
||||
@ -140,9 +140,8 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
public void unregisterQuery(TUniqueId queryId) {
|
||||
QueryInfo queryInfo = coordinatorMap.remove(queryId);
|
||||
if (queryInfo != null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("deregister query id {}", DebugUtil.printId(queryId));
|
||||
}
|
||||
LOG.info("Deregister query id {}", DebugUtil.printId(queryId));
|
||||
|
||||
if (queryInfo.getConnectContext() != null
|
||||
&& !Strings.isNullOrEmpty(queryInfo.getConnectContext().getQualifiedUser())
|
||||
) {
|
||||
@ -160,9 +159,7 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("not found query {} when unregisterQuery", DebugUtil.printId(queryId));
|
||||
}
|
||||
LOG.warn("not found query {} when unregisterQuery", DebugUtil.printId(queryId));
|
||||
}
|
||||
|
||||
// commit hive tranaction if needed
|
||||
@ -193,6 +190,10 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
|
||||
@Override
|
||||
public TReportExecStatusResult reportExecStatus(TReportExecStatusParams params, TNetworkAddress beAddr) {
|
||||
LOG.info("Processing report exec status, query {} instance {} from {}",
|
||||
DebugUtil.printId(params.query_id), DebugUtil.printId(params.fragment_instance_id),
|
||||
beAddr.toString());
|
||||
|
||||
if (params.isSetProfile()) {
|
||||
LOG.info("ReportExecStatus(): fragment_instance_id={}, query id={}, backend num: {}, ip: {}",
|
||||
DebugUtil.printId(params.fragment_instance_id), DebugUtil.printId(params.query_id),
|
||||
@ -219,7 +220,8 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
writeProfileExecutor.submit(new WriteProfileTask(params, info));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.warn(e.getMessage());
|
||||
LOG.warn("Report response: {}, query: {}, instance: {}", result.toString(),
|
||||
DebugUtil.printId(params.query_id), DebugUtil.printId(params.fragment_instance_id));
|
||||
return result;
|
||||
}
|
||||
result.setStatus(new TStatus(TStatusCode.OK));
|
||||
|
||||
@ -204,7 +204,6 @@ public class SimpleScheduler {
|
||||
try {
|
||||
Thread.sleep(1000L);
|
||||
SystemInfoService clusterInfoService = Env.getCurrentSystemInfo();
|
||||
LOG.debug("UpdateBlacklistThread retry begin");
|
||||
|
||||
Iterator<Map.Entry<Long, Pair<Integer, String>>> iterator = blacklistBackends.entrySet().iterator();
|
||||
while (iterator.hasNext()) {
|
||||
@ -227,9 +226,6 @@ public class SimpleScheduler {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
LOG.debug("UpdateBlacklistThread retry end");
|
||||
|
||||
} catch (Throwable ex) {
|
||||
LOG.warn("blacklist thread exception", ex);
|
||||
}
|
||||
|
||||
@ -1353,6 +1353,8 @@ public class StmtExecutor {
|
||||
|
||||
// Process a select statement.
|
||||
private void handleQueryStmt() throws Exception {
|
||||
LOG.info("Handling query {} with query id {}",
|
||||
originStmt.originStmt, DebugUtil.printId(context.queryId));
|
||||
// Every time set no send flag and clean all data in buffer
|
||||
context.getMysqlChannel().reset();
|
||||
Queriable queryStmt = (Queriable) parsedStmt;
|
||||
@ -1369,6 +1371,7 @@ public class StmtExecutor {
|
||||
if (queryStmt.isExplain()) {
|
||||
String explainString = planner.getExplainString(queryStmt.getExplainOptions());
|
||||
handleExplainStmt(explainString, false);
|
||||
LOG.info("Query {} finished", DebugUtil.printId(context.queryId));
|
||||
return;
|
||||
}
|
||||
|
||||
@ -1376,6 +1379,7 @@ public class StmtExecutor {
|
||||
Optional<ResultSet> resultSet = planner.handleQueryInFe(parsedStmt);
|
||||
if (resultSet.isPresent()) {
|
||||
sendResultSet(resultSet.get());
|
||||
LOG.info("Query {} finished", DebugUtil.printId(context.queryId));
|
||||
return;
|
||||
}
|
||||
|
||||
@ -1389,6 +1393,7 @@ public class StmtExecutor {
|
||||
&& context.getSessionVariable().getDefaultOrderByLimit() < 0) {
|
||||
if (queryStmt instanceof QueryStmt || queryStmt instanceof LogicalPlanAdapter) {
|
||||
handleCacheStmt(cacheAnalyzer, channel);
|
||||
LOG.info("Query {} finished", DebugUtil.printId(context.queryId));
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -1400,11 +1405,13 @@ public class StmtExecutor {
|
||||
LOG.info("ignore handle limit 0 ,sql:{}", parsedSelectStmt.toSql());
|
||||
sendFields(queryStmt.getColLabels(), exprToType(queryStmt.getResultExprs()));
|
||||
context.getState().setEof();
|
||||
LOG.info("Query {} finished", DebugUtil.printId(context.queryId));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
sendResult(isOutfileQuery, false, queryStmt, channel, null, null);
|
||||
LOG.info("Query {} finished", DebugUtil.printId(context.queryId));
|
||||
}
|
||||
|
||||
private void sendResult(boolean isOutfileQuery, boolean isSendFields, Queriable queryStmt, MysqlChannel channel,
|
||||
@ -1893,10 +1900,10 @@ public class StmtExecutor {
|
||||
|
||||
coord.exec();
|
||||
int execTimeout = context.getExecTimeout();
|
||||
LOG.debug("Insert execution timeout:{}", execTimeout);
|
||||
LOG.debug("Insert {} execution timeout:{}", DebugUtil.printId(context.queryId()), execTimeout);
|
||||
boolean notTimeout = coord.join(execTimeout);
|
||||
if (!coord.isDone()) {
|
||||
coord.cancel();
|
||||
coord.cancel(Types.PPlanFragmentCancelReason.TIMEOUT);
|
||||
if (notTimeout) {
|
||||
errMsg = coord.getExecStatus().getErrorMsg();
|
||||
ErrorReport.reportDdlException("There exists unhealthy backend. "
|
||||
|
||||
Reference in New Issue
Block a user