Skip to content

Commit

Permalink
add new field updatedRows to QueryStatistics.java where it's availabl…
Browse files Browse the repository at this point in the history
…e in EventListener callbacks. It comes from TableMutationOperator.java where update/delete queries are issued.
  • Loading branch information
bhzaeri committed Jan 21, 2025
1 parent af08b56 commit b45d6dc
Show file tree
Hide file tree
Showing 36 changed files with 154 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -332,6 +332,7 @@ private static QueryStats immediateFailureQueryStats()
DataSize.ofBytes(0),
0,
0,
0,
new Duration(0, MILLISECONDS),
new Duration(0, MILLISECONDS),
DataSize.ofBytes(0),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -231,6 +231,7 @@ public void queryImmediateFailureEvent(BasicQueryInfo queryInfo, ExecutionFailur
0,
0,
0,
0,
ImmutableList.of(),
0,
true,
Expand Down Expand Up @@ -343,6 +344,7 @@ private QueryStatistics createQueryStatistics(QueryInfo queryInfo)
queryStats.getRawInputPositions(),
queryStats.getOutputDataSize().toBytes(),
queryStats.getOutputPositions(),
queryStats.getUpdatedPositions(),
queryStats.getLogicalWrittenDataSize().toBytes(),
queryStats.getWrittenPositions(),
queryStats.getSpilledDataSize().toBytes(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ public static OperatorStats pruneOperatorStats(OperatorStats operatorStats)
operatorStats.getGetOutputCpu(),
operatorStats.getOutputDataSize(),
operatorStats.getOutputPositions(),
operatorStats.getUpdatedPositions(),
operatorStats.getDynamicFilterSplitsProcessed(),
pruneMetrics(operatorStats.getMetrics()),
pruneMetrics(operatorStats.getConnectorMetrics()),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -686,6 +686,7 @@ private QueryStats getQueryStats(Optional<StageInfo> rootStage, List<StageInfo>
long outputDataSize = 0;
long failedOutputDataSize = 0;
long outputPositions = 0;
long updatedPositions = 0;
long failedOutputPositions = 0;

long outputBlockedTime = 0;
Expand Down Expand Up @@ -772,6 +773,7 @@ private QueryStats getQueryStats(Optional<StageInfo> rootStage, List<StageInfo>
outputDataSize += outputStageStats.getOutputDataSize().toBytes();
failedOutputDataSize += outputStageStats.getFailedOutputDataSize().toBytes();
outputPositions += outputStageStats.getOutputPositions();
updatedPositions += outputStageStats.getUpdatedPositions();
failedOutputPositions += outputStageStats.getFailedOutputPositions();
}

Expand Down Expand Up @@ -898,6 +900,7 @@ private QueryStats getQueryStats(Optional<StageInfo> rootStage, List<StageInfo>
succinctBytes(outputDataSize),
succinctBytes(failedOutputDataSize),
outputPositions,
updatedPositions,
failedOutputPositions,

new Duration(outputBlockedTime, NANOSECONDS).convertToMostSuccinctTimeUnit(),
Expand Down Expand Up @@ -1502,6 +1505,7 @@ private static QueryStats pruneQueryStats(QueryStats queryStats)
queryStats.getOutputDataSize(),
queryStats.getFailedOutputDataSize(),
queryStats.getOutputPositions(),
queryStats.getUpdatedPositions(),
queryStats.getFailedOutputPositions(),
queryStats.getOutputBlockedTime(),
queryStats.getFailedOutputBlockedTime(),
Expand Down
10 changes: 10 additions & 0 deletions core/trino-main/src/main/java/io/trino/execution/QueryStats.java
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ public class QueryStats
private final DataSize outputDataSize;
private final DataSize failedOutputDataSize;
private final long outputPositions;
private final long updatedPositions;
private final long failedOutputPositions;

private final Duration outputBlockedTime;
Expand Down Expand Up @@ -213,6 +214,7 @@ public QueryStats(
@JsonProperty("outputDataSize") DataSize outputDataSize,
@JsonProperty("failedOutputDataSize") DataSize failedOutputDataSize,
@JsonProperty("outputPositions") long outputPositions,
@JsonProperty("updatedPositions") long updatedPositions,
@JsonProperty("failedOutputPositions") long failedOutputPositions,

@JsonProperty("outputBlockedTime") Duration outputBlockedTime,
Expand Down Expand Up @@ -323,6 +325,8 @@ public QueryStats(
this.failedOutputDataSize = requireNonNull(failedOutputDataSize, "failedOutputDataSize is null");
checkArgument(outputPositions >= 0, "outputPositions is negative");
this.outputPositions = outputPositions;
checkArgument(updatedPositions >= 0, "updatedPositions is negative");
this.updatedPositions = updatedPositions;
checkArgument(failedOutputPositions >= 0, "failedOutputPositions is negative");
this.failedOutputPositions = failedOutputPositions;

Expand Down Expand Up @@ -743,6 +747,12 @@ public long getOutputPositions()
return outputPositions;
}

@JsonProperty
public long getUpdatedPositions()
{
return updatedPositions;
}

@JsonProperty
public long getFailedOutputPositions()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -473,6 +473,7 @@ public StageInfo getStageInfo(Supplier<Iterable<TaskInfo>> taskInfosSupplier)
long outputDataSize = 0;
long failedOutputDataSize = 0;
long outputPositions = 0;
long updatedPositions = 0;
long failedOutputPositions = 0;
Metrics.Accumulator outputBufferMetrics = Metrics.accumulator();

Expand Down Expand Up @@ -553,6 +554,7 @@ public StageInfo getStageInfo(Supplier<Iterable<TaskInfo>> taskInfosSupplier)
taskInfo.outputBuffers().getUtilization().ifPresent(bufferUtilizationHistograms::add);
outputDataSize += taskStats.getOutputDataSize().toBytes();
outputPositions += taskStats.getOutputPositions();
updatedPositions += taskStats.getUpdatedPositions();
bufferMetrics.ifPresent(outputBufferMetrics::add);

outputBlockedTime += taskStats.getOutputBlockedTime().roundTo(NANOSECONDS);
Expand Down Expand Up @@ -660,6 +662,7 @@ public StageInfo getStageInfo(Supplier<Iterable<TaskInfo>> taskInfosSupplier)
succinctBytes(outputDataSize),
succinctBytes(failedOutputDataSize),
outputPositions,
updatedPositions,
failedOutputPositions,
outputBufferMetrics.get(),
succinctDuration(outputBlockedTime, NANOSECONDS),
Expand Down
11 changes: 11 additions & 0 deletions core/trino-main/src/main/java/io/trino/execution/StageStats.java
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ public class StageStats
private final DataSize outputDataSize;
private final DataSize failedOutputDataSize;
private final long outputPositions;
private final long updatedPositions;
private final long failedOutputPositions;
private final Metrics outputBufferMetrics;

Expand Down Expand Up @@ -182,6 +183,7 @@ public StageStats(
@JsonProperty("outputDataSize") DataSize outputDataSize,
@JsonProperty("failedOutputDataSize") DataSize failedOutputDataSize,
@JsonProperty("outputPositions") long outputPositions,
@JsonProperty("updatedPositions") long updatedPositions,
@JsonProperty("failedOutputPositions") long failedOutputPositions,
@JsonProperty("outputBufferMetrics") Metrics outputBufferMetrics,

Expand Down Expand Up @@ -273,6 +275,8 @@ public StageStats(
this.failedOutputDataSize = requireNonNull(failedOutputDataSize, "failedOutputDataSize is null");
checkArgument(outputPositions >= 0, "outputPositions is negative");
this.outputPositions = outputPositions;
checkArgument(updatedPositions >= 0, "updatedPositions is negative");
this.updatedPositions = updatedPositions;
checkArgument(failedOutputPositions >= 0, "failedOutputPositions is negative");
this.failedOutputPositions = failedOutputPositions;
this.outputBufferMetrics = requireNonNull(outputBufferMetrics, "outputBufferMetrics is null");
Expand Down Expand Up @@ -588,6 +592,12 @@ public long getOutputPositions()
return outputPositions;
}

@JsonProperty
public long getUpdatedPositions()
{
return updatedPositions;
}

@JsonProperty
public long getFailedOutputPositions()
{
Expand Down Expand Up @@ -736,6 +746,7 @@ public static StageStats createInitial()
zeroBytes,
0,
0,
0,
Metrics.EMPTY,
zeroSeconds,
zeroSeconds,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -346,6 +346,7 @@ public DriverStats getDriverStats()
Duration inputBlockedTime;
DataSize outputDataSize;
long outputPositions;
long updatedPositions;
Duration outputBlockedTime;
if (inputOperator != null) {
physicalInputDataSize = inputOperator.getPhysicalInputDataSize();
Expand All @@ -364,6 +365,8 @@ public DriverStats getDriverStats()

inputBlockedTime = inputOperator.getBlockedWall();

updatedPositions = inputOperator.getUpdatedPositions();

OperatorStats outputOperator = requireNonNull(getLast(operators, null));
outputDataSize = outputOperator.getOutputDataSize();
outputPositions = outputOperator.getOutputPositions();
Expand All @@ -389,6 +392,7 @@ public DriverStats getDriverStats()

outputDataSize = DataSize.ofBytes(0);
outputPositions = 0;
updatedPositions = 0;

outputBlockedTime = new Duration(0, MILLISECONDS);
}
Expand Down Expand Up @@ -428,6 +432,7 @@ public DriverStats getDriverStats()
inputBlockedTime,
outputDataSize.succinct(),
outputPositions,
updatedPositions,
outputBlockedTime,
succinctBytes(physicalWrittenDataSize),
operators);
Expand Down
11 changes: 11 additions & 0 deletions core/trino-main/src/main/java/io/trino/operator/DriverStats.java
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ public class DriverStats

private final DataSize outputDataSize;
private final long outputPositions;
private final long updatedPositions;

private final Duration outputBlockedTime;

Expand Down Expand Up @@ -109,6 +110,7 @@ public DriverStats()

this.outputDataSize = DataSize.ofBytes(0);
this.outputPositions = 0;
this.updatedPositions = 0;

this.outputBlockedTime = new Duration(0, MILLISECONDS);

Expand Down Expand Up @@ -152,6 +154,7 @@ public DriverStats(

@JsonProperty("outputDataSize") DataSize outputDataSize,
@JsonProperty("outputPositions") long outputPositions,
@JsonProperty("updatedPositions") long updatedPositions,

@JsonProperty("outputBlockedTime") Duration outputBlockedTime,

Expand Down Expand Up @@ -197,6 +200,8 @@ public DriverStats(
this.outputDataSize = requireNonNull(outputDataSize, "outputDataSize is null");
checkArgument(outputPositions >= 0, "outputPositions is negative");
this.outputPositions = outputPositions;
checkArgument(updatedPositions >= 0, "updatedPositions is negative");
this.updatedPositions = updatedPositions;

this.outputBlockedTime = requireNonNull(outputBlockedTime, "outputBlockedTime is null");

Expand Down Expand Up @@ -357,6 +362,12 @@ public long getOutputPositions()
return outputPositions;
}

@JsonProperty
public long getUpdatedPositions()
{
return updatedPositions;
}

@JsonProperty
public Duration getOutputBlockedTime()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@ public class OperatorContext
private final OperationTiming getOutputTiming = new OperationTiming();
private final CounterStat outputDataSize = new CounterStat();
private final CounterStat outputPositions = new CounterStat();
private final CounterStat updatedPositions = new CounterStat();

private final AtomicLong dynamicFilterSplitsProcessed = new AtomicLong();
private final AtomicReference<Metrics> metrics = new AtomicReference<>(Metrics.EMPTY); // this is not incremental, but gets overwritten by the latest value.
Expand Down Expand Up @@ -490,6 +491,11 @@ public CounterStat getOutputPositions()
return outputPositions;
}

public CounterStat getUpdatedPositions()
{
return updatedPositions;
}

public long getWriterInputDataSize()
{
return writerInputDataSize.get();
Expand Down Expand Up @@ -554,6 +560,7 @@ public OperatorStats getOperatorStats()
new Duration(getOutputTiming.getCpuNanos(), NANOSECONDS).convertToMostSuccinctTimeUnit(),
DataSize.ofBytes(outputDataSize.getTotalCount()),
outputPositions.getTotalCount(),
updatedPositions.getTotalCount(),

dynamicFilterSplitsProcessed.get(),
getOperatorMetrics(
Expand Down
15 changes: 15 additions & 0 deletions core/trino-main/src/main/java/io/trino/operator/OperatorStats.java
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ public class OperatorStats
private final Duration getOutputCpu;
private final DataSize outputDataSize;
private final long outputPositions;
private final long updatedPositions;

private final long dynamicFilterSplitsProcessed;
private final Metrics metrics;
Expand Down Expand Up @@ -117,6 +118,7 @@ public OperatorStats(
@JsonProperty("getOutputCpu") Duration getOutputCpu,
@JsonProperty("outputDataSize") DataSize outputDataSize,
@JsonProperty("outputPositions") long outputPositions,
@JsonProperty("updatedPositions") long updatedPositions,

@JsonProperty("dynamicFilterSplitsProcessed") long dynamicFilterSplitsProcessed,
@JsonProperty("metrics") Metrics metrics,
Expand Down Expand Up @@ -174,6 +176,8 @@ public OperatorStats(
this.outputDataSize = requireNonNull(outputDataSize, "outputDataSize is null");
checkArgument(outputPositions >= 0, "outputPositions is negative");
this.outputPositions = outputPositions;
checkArgument(updatedPositions >= 0, "updatedPositions is negative");
this.updatedPositions = updatedPositions;

this.dynamicFilterSplitsProcessed = dynamicFilterSplitsProcessed;
this.metrics = requireNonNull(metrics, "metrics is null");
Expand Down Expand Up @@ -340,6 +344,12 @@ public long getOutputPositions()
return outputPositions;
}

@JsonProperty
public long getUpdatedPositions()
{
return updatedPositions;
}

@JsonProperty
public long getDynamicFilterSplitsProcessed()
{
Expand Down Expand Up @@ -485,6 +495,7 @@ private OperatorStats add(Iterable<OperatorStats> operators, Optional<Metrics> p
long getOutputCpu = this.getOutputCpu.roundTo(NANOSECONDS);
long outputDataSize = this.outputDataSize.toBytes();
long outputPositions = this.outputPositions;
long updatedPositions = this.updatedPositions;

long dynamicFilterSplitsProcessed = this.dynamicFilterSplitsProcessed;
Metrics.Accumulator metricsAccumulator = Metrics.accumulator().add(this.getMetrics());
Expand Down Expand Up @@ -535,6 +546,7 @@ private OperatorStats add(Iterable<OperatorStats> operators, Optional<Metrics> p
getOutputCpu += operator.getGetOutputCpu().roundTo(NANOSECONDS);
outputDataSize += operator.getOutputDataSize().toBytes();
outputPositions += operator.getOutputPositions();
updatedPositions += operator.getUpdatedPositions();

dynamicFilterSplitsProcessed += operator.getDynamicFilterSplitsProcessed();
metricsAccumulator.add(operator.getMetrics());
Expand Down Expand Up @@ -596,6 +608,7 @@ private OperatorStats add(Iterable<OperatorStats> operators, Optional<Metrics> p
new Duration(getOutputCpu, NANOSECONDS).convertToMostSuccinctTimeUnit(),
DataSize.ofBytes(outputDataSize),
outputPositions,
updatedPositions,

dynamicFilterSplitsProcessed,
metricsAccumulator.get(),
Expand Down Expand Up @@ -672,6 +685,7 @@ public OperatorStats summarize()
getOutputCpu,
outputDataSize,
outputPositions,
updatedPositions,
dynamicFilterSplitsProcessed,
metrics,
connectorMetrics,
Expand Down Expand Up @@ -717,6 +731,7 @@ public OperatorStats withPipelineMetrics(Metrics pipelineMetrics)
getOutputCpu,
outputDataSize,
outputPositions,
updatedPositions,
dynamicFilterSplitsProcessed,
metrics,
connectorMetrics,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,7 @@ public class PipelineContext

private final CounterStat outputDataSize = new CounterStat();
private final CounterStat outputPositions = new CounterStat();
private final CounterStat updatedPositions = new CounterStat();

private final AtomicLong outputBlockedTime = new AtomicLong();

Expand Down Expand Up @@ -237,6 +238,7 @@ public void driverFinished(DriverContext driverContext)

outputDataSize.update(driverStats.getOutputDataSize().toBytes());
outputPositions.update(driverStats.getOutputPositions());
updatedPositions.update(driverStats.getUpdatedPositions());

outputBlockedTime.getAndAdd(driverStats.getOutputBlockedTime().roundTo(NANOSECONDS));

Expand Down Expand Up @@ -403,6 +405,7 @@ public PipelineStats getPipelineStats()

long outputDataSize = this.outputDataSize.getTotalCount();
long outputPositions = this.outputPositions.getTotalCount();
long updatedPositions = this.updatedPositions.getTotalCount();

long outputBlockedTime = this.outputBlockedTime.get();

Expand Down Expand Up @@ -455,6 +458,7 @@ public PipelineStats getPipelineStats()

outputDataSize += driverStats.getOutputDataSize().toBytes();
outputPositions += driverStats.getOutputPositions();
updatedPositions += driverStats.getUpdatedPositions();

outputBlockedTime += driverStats.getOutputBlockedTime().roundTo(NANOSECONDS);

Expand Down Expand Up @@ -538,6 +542,7 @@ else if (pipelineLevelMetrics != Metrics.EMPTY) {

succinctBytes(outputDataSize),
outputPositions,
updatedPositions,

new Duration(outputBlockedTime, NANOSECONDS).convertToMostSuccinctTimeUnit(),

Expand Down
Loading

0 comments on commit b45d6dc

Please sign in to comment.