Skip to content

Commit

Permalink
[Enhancement] Add pendingTime/netTime/netComputeTime to queryDetail (…
Browse files Browse the repository at this point in the history
…backport #53322) (#53377)

Co-authored-by: satanson <[email protected]>
  • Loading branch information
mergify[bot] and satanson authored Dec 2, 2024
1 parent 9096acf commit ac3aef8
Show file tree
Hide file tree
Showing 7 changed files with 82 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

Expand All @@ -42,6 +43,10 @@ public Timer scope(long time, String name) {
return t;
}

public Optional<Timer> getTimer(String name) {
return Optional.ofNullable(timers.get(name));
}

public List<Timer> getAllTimerWithOrder() {
return timers.values().stream().sorted(Comparator.comparingLong(o -> o.firstTimePoints))
.collect(Collectors.toList());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.function.Function;

public abstract class Tracer {
Expand Down Expand Up @@ -71,4 +72,8 @@ public String printReasons() {

public void toRuntimeProfile(RuntimeProfile parent) {
}

public Optional<Timer> getSpecifiedTimer(String name) {
return Optional.empty();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
Expand Down Expand Up @@ -242,4 +243,9 @@ public void toRuntimeProfile(RuntimeProfile parent) {
buildVars(parent);
buildReasons(parent);
}

@Override
public Optional<Timer> getSpecifiedTimer(String name) {
return watcher.getTimer(name);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.commons.lang3.StringUtils;

import java.util.List;
import java.util.Optional;
import java.util.function.Function;

public class Tracers {
Expand Down Expand Up @@ -258,6 +259,11 @@ public static void toRuntimeProfile(RuntimeProfile profile) {
tracers.allTracer[1].toRuntimeProfile(profile);
}

public static Optional<Timer> getSpecifiedTimer(String name) {
Tracers tracers = THREAD_LOCAL.get();
return tracers.allTracer[1].getSpecifiedTimer(name);
}

public static String getTrace(Mode mode) {
switch (mode) {
case TIMER:
Expand Down
28 changes: 28 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/qe/QueryDetail.java
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,10 @@ public enum QueryMemState {
// default value will set to be minus one(-1).
private long endTime;
private long latency;

private long pendingTime = -1;
private long netTime = -1;
private long netComputeTime = -1;
private QueryMemState state;
private String database;
private String sql;
Expand Down Expand Up @@ -207,6 +211,30 @@ public long getLatency() {
return latency;
}

public long getPendingTime() {
return pendingTime;
}

public void setPendingTime(long pendingTime) {
this.pendingTime = pendingTime;
}

public long getNetTime() {
return netTime;
}

public void setNetTime(long netTime) {
this.netTime = netTime;
}

public long getNetComputeTime() {
return netComputeTime;
}

public void setNetComputeTime(long netComputeTime) {
this.netComputeTime = netComputeTime;
}

public void setState(QueryMemState state) {
this.state = state;
}
Expand Down
9 changes: 9 additions & 0 deletions fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java
Original file line number Diff line number Diff line change
Expand Up @@ -2860,6 +2860,15 @@ public void addFinishedQueryDetail() {
}
queryDetail.setEndTime(endTime);
queryDetail.setLatency(elapseMs);
long pendingTime = ctx.getAuditEventBuilder().build().pendingTimeMs;
pendingTime = pendingTime < 0 ? 0 : pendingTime;
queryDetail.setPendingTime(pendingTime);
queryDetail.setNetTime(elapseMs - pendingTime);
long parseTime = Tracers.getSpecifiedTimer("Parser").map(Timer::getTotalTime).orElse(0L);
long planTime = Tracers.getSpecifiedTimer("Total").map(Timer::getTotalTime).orElse(0L);
long prepareTime = Tracers.getSpecifiedTimer("Prepare").map(Timer::getTotalTime).orElse(0L);
long deployTime = Tracers.getSpecifiedTimer("Deploy").map(Timer::getTotalTime).orElse(0L);
queryDetail.setNetComputeTime(elapseMs - parseTime - planTime - prepareTime - pendingTime - deployTime);
queryDetail.setResourceGroupName(ctx.getResourceGroup() != null ? ctx.getResourceGroup().getName() : "");
// add execution statistics into queryDetail
queryDetail.setReturnRows(ctx.getReturnRows());
Expand Down
41 changes: 23 additions & 18 deletions fe/fe-core/src/test/java/com/starrocks/qe/QueryDetailQueueTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -63,24 +63,29 @@ public void testQueryDetailQueue() {

Gson gson = new Gson();
String jsonString = gson.toJson(queryDetails);
String queryDetailString = "[{\"eventTime\":" + startQueryDetail.getEventTime() + ","
+ "\"queryId\":\"219a2d5443c542d4-8fc938db37c892e3\","
+ "\"isQuery\":false,"
+ "\"remoteIP\":\"127.0.0.1\","
+ "\"connId\":1,"
+ "\"startTime\":" + startQueryDetail.getStartTime() + ",\"endTime\":-1,\"latency\":-1,"
+ "\"state\":\"RUNNING\",\"database\":\"testDb\","
+ "\"sql\":\"select * from table1 limit 1\","
+ "\"user\":\"root\","
+ "\"scanRows\":100,"
+ "\"scanBytes\":10001,"
+ "\"returnRows\":1,"
+ "\"cpuCostNs\":1002,"
+ "\"memCostBytes\":100003,"
+ "\"spillBytes\":-1,"
+ "\"warehouse\":\"default_warehouse\","
+ "\"catalog\":\"default_catalog\""
+ "}]";
String queryDetailString = "[{\"eventTime\":" + startQueryDetail.getEventTime() + "," +
"\"queryId\":\"219a2d5443c542d4-8fc938db37c892e3\"," +
"\"isQuery\":false," +
"\"remoteIP\":\"127.0.0.1\"," +
"\"connId\":1," +
"\"startTime\":" + startQueryDetail.getStartTime() + "," +
"\"endTime\":-1," +
"\"latency\":-1," +
"\"pendingTime\":-1," +
"\"netTime\":-1," +
"\"netComputeTime\":-1," +
"\"state\":\"RUNNING\"," +
"\"database\":\"testDb\"," +
"\"sql\":\"select * from table1 limit 1\"," +
"\"user\":\"root\"," +
"\"scanRows\":100," +
"\"scanBytes\":10001," +
"\"returnRows\":1," +
"\"cpuCostNs\":1002," +
"\"memCostBytes\":100003," +
"\"spillBytes\":-1," +
"\"warehouse\":\"default_warehouse\"," +
"\"catalog\":\"default_catalog\"}]";
Assert.assertEquals(jsonString, queryDetailString);

queryDetails = QueryDetailQueue.getQueryDetailsAfterTime(startQueryDetail.getEventTime());
Expand Down

0 comments on commit ac3aef8

Please sign in to comment.