@ -34,7 +34,6 @@ import org.apache.logging.log4j.Logger;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@ -67,10 +66,6 @@ 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) {
|
||||
executionProfile = new RuntimeProfile("Execution Profile " + DebugUtil.printId(queryId));
|
||||
RuntimeProfile fragmentsProfile = new RuntimeProfile("Fragments");
|
||||
@ -82,7 +77,6 @@ public class ExecutionProfile {
|
||||
}
|
||||
loadChannelProfile = new RuntimeProfile("LoadChannels");
|
||||
executionProfile.addChild(loadChannelProfile);
|
||||
this.queryId = queryId;
|
||||
}
|
||||
|
||||
public RuntimeProfile getAggregatedFragmentsProfile(Map<Integer, String> planNodeMap) {
|
||||
@ -148,16 +142,12 @@ public class ExecutionProfile {
|
||||
if (profileDoneSignal != null) {
|
||||
// count down to zero to notify all objects waiting for this
|
||||
profileDoneSignal.countDownToZero(new Status());
|
||||
LOG.info("Query {} unfinished instance: {}", DebugUtil.printId(queryId), profileDoneSignal.getLeftMarks()
|
||||
.stream().map(e -> DebugUtil.printId(e.getKey())).toArray());
|
||||
}
|
||||
}
|
||||
|
||||
public void markOneInstanceDone(TUniqueId fragmentInstanceId) {
|
||||
if (profileDoneSignal != null) {
|
||||
if (profileDoneSignal.markedCountDown(fragmentInstanceId, -1L)) {
|
||||
LOG.info("Mark instance {} done succeed", DebugUtil.printId(fragmentInstanceId));
|
||||
} else {
|
||||
if (!profileDoneSignal.markedCountDown(fragmentInstanceId, -1L)) {
|
||||
LOG.warn("Mark instance {} done failed", DebugUtil.printId(fragmentInstanceId));
|
||||
}
|
||||
}
|
||||
@ -167,16 +157,6 @@ 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);
|
||||
}
|
||||
|
||||
|
||||
@ -106,7 +106,7 @@ public class Transaction {
|
||||
|
||||
coordinator.exec();
|
||||
int execTimeout = ctx.getExecTimeout();
|
||||
LOG.info("Insert {} execution timeout:{}", DebugUtil.printId(ctx.queryId()), execTimeout);
|
||||
LOG.debug("Insert {} execution timeout:{}", DebugUtil.printId(ctx.queryId()), execTimeout);
|
||||
boolean notTimeout = coordinator.join(execTimeout);
|
||||
if (!coordinator.isDone()) {
|
||||
coordinator.cancel();
|
||||
|
||||
@ -2532,12 +2532,7 @@ 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()));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -140,7 +140,9 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
public void unregisterQuery(TUniqueId queryId) {
|
||||
QueryInfo queryInfo = coordinatorMap.remove(queryId);
|
||||
if (queryInfo != null) {
|
||||
LOG.info("Deregister query id {}", DebugUtil.printId(queryId));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Deregister query id {}", DebugUtil.printId(queryId));
|
||||
}
|
||||
|
||||
if (queryInfo.getConnectContext() != null
|
||||
&& !Strings.isNullOrEmpty(queryInfo.getConnectContext().getQualifiedUser())
|
||||
@ -159,7 +161,9 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
LOG.warn("not found query {} when unregisterQuery", DebugUtil.printId(queryId));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("not found query {} when unregisterQuery", DebugUtil.printId(queryId));
|
||||
}
|
||||
}
|
||||
|
||||
// commit hive tranaction if needed
|
||||
@ -190,10 +194,6 @@ 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),
|
||||
@ -220,7 +220,7 @@ public final class QeProcessorImpl implements QeProcessor {
|
||||
writeProfileExecutor.submit(new WriteProfileTask(params, info));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Report response: {}, query: {}, instance: {}", result.toString(),
|
||||
LOG.warn("Exception during handle report, response: {}, query: {}, instance: {}", result.toString(),
|
||||
DebugUtil.printId(params.query_id), DebugUtil.printId(params.fragment_instance_id));
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -1363,12 +1363,16 @@ 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));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Handling query {} with query id {}",
|
||||
originStmt.originStmt, DebugUtil.printId(context.queryId));
|
||||
}
|
||||
|
||||
if (context.getConnectType() == ConnectType.MYSQL) {
|
||||
// Every time set no send flag and clean all data in buffer
|
||||
context.getMysqlChannel().reset();
|
||||
}
|
||||
|
||||
Queriable queryStmt = (Queriable) parsedStmt;
|
||||
|
||||
QueryDetail queryDetail = new QueryDetail(context.getStartTime(),
|
||||
@ -1419,7 +1423,10 @@ public class StmtExecutor {
|
||||
if (channel != null && parsedStmt instanceof SelectStmt) {
|
||||
SelectStmt parsedSelectStmt = (SelectStmt) parsedStmt;
|
||||
if (parsedSelectStmt.getLimit() == 0) {
|
||||
LOG.info("ignore handle limit 0 ,sql:{}", parsedSelectStmt.toSql());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("ignore handle limit 0 ,sql:{}", parsedSelectStmt.toSql());
|
||||
}
|
||||
|
||||
sendFields(queryStmt.getColLabels(), exprToType(queryStmt.getResultExprs()));
|
||||
context.getState().setEof();
|
||||
LOG.info("Query {} finished", DebugUtil.printId(context.queryId));
|
||||
|
||||
Reference in New Issue
Block a user