Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Add recording distribution of outputBuffer utilization #13463

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

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 @@ -24,11 +24,13 @@
import io.trino.operator.OperatorStats;
import io.trino.operator.PipelineStats;
import io.trino.operator.TaskStats;
import io.trino.plugin.base.metrics.TDigestHistogram;
import io.trino.spi.eventlistener.StageGcStatistics;
import io.trino.sql.planner.PlanFragment;
import io.trino.sql.planner.plan.PlanNodeId;
import io.trino.sql.planner.plan.TableScanNode;
import io.trino.util.Failures;
import io.trino.util.Optionals;
import org.joda.time.DateTime;

import javax.annotation.concurrent.ThreadSafe;
Expand Down Expand Up @@ -420,6 +422,7 @@ public StageInfo getStageInfo(Supplier<Iterable<TaskInfo>> taskInfosSupplier)
long failedInputBlockedTime = 0;

long bufferedDataSize = 0;
Optional<TDigestHistogram> outputBufferUtilization = Optional.empty();
long outputDataSize = 0;
long failedOutputDataSize = 0;
long outputPositions = 0;
Expand Down Expand Up @@ -495,6 +498,7 @@ public StageInfo getStageInfo(Supplier<Iterable<TaskInfo>> taskInfosSupplier)
inputBlockedTime += taskStats.getInputBlockedTime().roundTo(NANOSECONDS);

bufferedDataSize += taskInfo.getOutputBuffers().getTotalBufferedBytes();
outputBufferUtilization = Optionals.combine(outputBufferUtilization, taskInfo.getOutputBuffers().getUtilization(), TDigestHistogram::mergeWith);
outputDataSize += taskStats.getOutputDataSize().toBytes();
outputPositions += taskStats.getOutputPositions();

Expand Down Expand Up @@ -596,6 +600,7 @@ public StageInfo getStageInfo(Supplier<Iterable<TaskInfo>> taskInfosSupplier)
succinctDuration(inputBlockedTime, NANOSECONDS),
succinctDuration(failedInputBlockedTime, NANOSECONDS),
succinctBytes(bufferedDataSize),
outputBufferUtilization,
succinctBytes(outputDataSize),
succinctBytes(failedOutputDataSize),
outputPositions,
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 @@ -22,12 +22,14 @@
import io.airlift.units.Duration;
import io.trino.operator.BlockedReason;
import io.trino.operator.OperatorStats;
import io.trino.plugin.base.metrics.TDigestHistogram;
import io.trino.spi.eventlistener.StageGcStatistics;
import org.joda.time.DateTime;

import javax.annotation.concurrent.Immutable;

import java.util.List;
import java.util.Optional;
import java.util.OptionalDouble;
import java.util.Set;

Expand Down Expand Up @@ -96,6 +98,7 @@ public class StageStats
private final Duration failedInputBlockedTime;

private final DataSize bufferedDataSize;
private final Optional<TDigestHistogram> outputBufferUtilization;
private final DataSize outputDataSize;
private final DataSize failedOutputDataSize;
private final long outputPositions;
Expand Down Expand Up @@ -170,6 +173,7 @@ public StageStats(
@JsonProperty("failedInputBlockedTime") Duration failedInputBlockedTime,

@JsonProperty("bufferedDataSize") DataSize bufferedDataSize,
@JsonProperty("outputBufferUtilization") Optional<TDigestHistogram> outputBufferUtilization,
@JsonProperty("outputDataSize") DataSize outputDataSize,
@JsonProperty("failedOutputDataSize") DataSize failedOutputDataSize,
@JsonProperty("outputPositions") long outputPositions,
Expand Down Expand Up @@ -258,6 +262,7 @@ public StageStats(
this.failedInputBlockedTime = requireNonNull(failedInputBlockedTime, "failedInputBlockedTime is null");

this.bufferedDataSize = requireNonNull(bufferedDataSize, "bufferedDataSize is null");
this.outputBufferUtilization = requireNonNull(outputBufferUtilization, "outputBufferUtilization is null");
this.outputDataSize = requireNonNull(outputDataSize, "outputDataSize is null");
this.failedOutputDataSize = requireNonNull(failedOutputDataSize, "failedOutputDataSize is null");
checkArgument(outputPositions >= 0, "outputPositions is negative");
Expand Down Expand Up @@ -552,6 +557,12 @@ public DataSize getBufferedDataSize()
return bufferedDataSize;
}

@JsonProperty
public Optional<TDigestHistogram> getOutputBufferUtilization()
{
return outputBufferUtilization;
}

@JsonProperty
public DataSize getOutputDataSize()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ public boolean isNeedsPlan()
public TaskInfo summarize()
{
if (taskStatus.getState().isDone()) {
return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarize(), noMoreSplits, stats.summarizeFinal(), estimatedMemory, needsPlan);
return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarizeFinal(), noMoreSplits, stats.summarizeFinal(), estimatedMemory, needsPlan);
}
return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarize(), noMoreSplits, stats.summarize(), estimatedMemory, needsPlan);
}
Expand All @@ -130,7 +130,7 @@ public static TaskInfo createInitialTask(TaskId taskId, URI location, String nod
return new TaskInfo(
initialTaskStatus(taskId, location, nodeId),
DateTime.now(),
new OutputBufferInfo("UNINITIALIZED", OPEN, true, true, 0, 0, 0, 0, bufferStates),
new OutputBufferInfo("UNINITIALIZED", OPEN, true, true, 0, 0, 0, 0, bufferStates, Optional.empty()),
ImmutableSet.of(),
taskStats,
Optional.empty(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.execution.buffer.SerializedPageReference.PagesReleasedListener;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.plugin.base.metrics.TDigestHistogram;

import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
Expand Down Expand Up @@ -150,7 +151,8 @@ public OutputBufferInfo getInfo()
totalBufferedPages,
totalRowsAdded.get(),
totalPagesAdded.get(),
infos.build());
infos.build(),
Optional.of(new TDigestHistogram(memoryManager.getUtilizationHistogram())));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.execution.buffer.SerializedPageReference.PagesReleasedListener;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.plugin.base.metrics.TDigestHistogram;

import javax.annotation.concurrent.GuardedBy;

Expand Down Expand Up @@ -142,7 +143,8 @@ public OutputBufferInfo getInfo()
totalPagesAdded.get(),
buffers.stream()
.map(ClientBuffer::getInfo)
.collect(toImmutableList()));
.collect(toImmutableList()),
Optional.of(new TDigestHistogram(memoryManager.getUtilizationHistogram())));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,8 @@ public OutputBufferInfo getInfo()
0,
0,
0,
ImmutableList.of());
ImmutableList.of(),
Optional.empty());
}
return outputBuffer.getInfo();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,11 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import io.trino.plugin.base.metrics.TDigestHistogram;

import java.util.List;
import java.util.Objects;
import java.util.Optional;

import static com.google.common.base.MoreObjects.toStringHelper;

Expand All @@ -33,6 +35,7 @@ public final class OutputBufferInfo
private final long totalRowsSent;
private final long totalPagesSent;
private final List<BufferInfo> buffers;
private final Optional<TDigestHistogram> utilization;

@JsonCreator
public OutputBufferInfo(
Expand All @@ -44,7 +47,8 @@ public OutputBufferInfo(
@JsonProperty("totalBufferedPages") long totalBufferedPages,
@JsonProperty("totalRowsSent") long totalRowsSent,
@JsonProperty("totalPagesSent") long totalPagesSent,
@JsonProperty("buffers") List<BufferInfo> buffers)
@JsonProperty("buffers") List<BufferInfo> buffers,
@JsonProperty("utilization") Optional<TDigestHistogram> utilization)
{
this.type = type;
this.state = state;
Expand All @@ -55,6 +59,7 @@ public OutputBufferInfo(
this.totalRowsSent = totalRowsSent;
this.totalPagesSent = totalPagesSent;
this.buffers = ImmutableList.copyOf(buffers);
this.utilization = utilization;
}

@JsonProperty
Expand Down Expand Up @@ -111,9 +116,20 @@ public long getTotalPagesSent()
return totalPagesSent;
}

@JsonProperty
public Optional<TDigestHistogram> getUtilization()
{
return utilization;
}

public OutputBufferInfo summarize()
{
return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of());
return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of(), Optional.empty());
}

public OutputBufferInfo summarizeFinal()
{
return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of(), utilization);
}

@Override
Expand All @@ -134,13 +150,14 @@ public boolean equals(Object o)
Objects.equals(totalRowsSent, that.totalRowsSent) &&
Objects.equals(totalPagesSent, that.totalPagesSent) &&
state == that.state &&
Objects.equals(buffers, that.buffers);
Objects.equals(buffers, that.buffers) &&
Objects.equals(utilization, that.utilization);
}

@Override
public int hashCode()
{
return Objects.hash(state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, buffers);
return Objects.hash(state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, buffers, utilization);
}

@Override
Expand All @@ -156,6 +173,7 @@ public String toString()
.add("totalRowsSent", totalRowsSent)
.add("totalPagesSent", totalPagesSent)
.add("buffers", buffers)
.add("bufferUtilization", utilization)
.toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,10 @@

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Suppliers;
import com.google.common.base.Ticker;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.airlift.stats.TDigest;
import io.trino.memory.context.LocalMemoryContext;

import javax.annotation.Nullable;
Expand Down Expand Up @@ -54,18 +56,29 @@ class OutputBufferMemoryManager
@GuardedBy("this")
private ListenableFuture<Void> blockedOnMemory = NOT_BLOCKED;

private final Ticker ticker = Ticker.systemTicker();

private final AtomicBoolean blockOnFull = new AtomicBoolean(true);

private final Supplier<LocalMemoryContext> memoryContextSupplier;
private final Executor notificationExecutor;

@GuardedBy("this")
private final TDigest bufferUtilization = new TDigest();
@GuardedBy("this")
private long lastBufferUtilizationRecordTime;
@GuardedBy("this")
private double lastBufferUtilization;

public OutputBufferMemoryManager(long maxBufferedBytes, Supplier<LocalMemoryContext> memoryContextSupplier, Executor notificationExecutor)
{
requireNonNull(memoryContextSupplier, "memoryContextSupplier is null");
checkArgument(maxBufferedBytes > 0, "maxBufferedBytes must be > 0");
this.maxBufferedBytes = maxBufferedBytes;
this.memoryContextSupplier = Suppliers.memoize(memoryContextSupplier::get);
this.notificationExecutor = requireNonNull(notificationExecutor, "notificationExecutor is null");
this.lastBufferUtilizationRecordTime = ticker.read();
this.lastBufferUtilization = 0;
}

public void updateMemoryUsage(long bytesAdded)
Expand Down Expand Up @@ -110,6 +123,7 @@ public void updateMemoryUsage(long bytesAdded)
this.bufferBlockedFuture = null;
}
}
recordBufferUtilization();
}
peakMemoryUsage.accumulateAndGet(currentBufferedBytes, Math::max);
// Notify listeners outside of the critical section
Expand All @@ -119,6 +133,14 @@ public void updateMemoryUsage(long bytesAdded)
}
}

private synchronized void recordBufferUtilization()
{
long recordTime = ticker.read();
bufferUtilization.add(lastBufferUtilization, (double) recordTime - this.lastBufferUtilizationRecordTime);
lastBufferUtilizationRecordTime = recordTime;
lastBufferUtilization = getUtilization();
}

public synchronized ListenableFuture<Void> getBufferBlockedFuture()
{
if (bufferBlockedFuture == null) {
Expand Down Expand Up @@ -155,6 +177,13 @@ public double getUtilization()
return bufferedBytes.get() / (double) maxBufferedBytes;
}

public synchronized TDigest getUtilizationHistogram()
{
// always get most up to date histogram
recordBufferUtilization();
return TDigest.copyOf(bufferUtilization);
}

public boolean isOverutilized()
{
return isBufferFull();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.trino.execution.buffer.OutputBuffers.OutputBufferId;
import io.trino.execution.buffer.SerializedPageReference.PagesReleasedListener;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.plugin.base.metrics.TDigestHistogram;

import java.util.List;
import java.util.Optional;
Expand Down Expand Up @@ -127,7 +128,8 @@ public OutputBufferInfo getInfo()
totalBufferedPages,
totalRowsAdded.get(),
totalPagesAdded.get(),
infos.build());
infos.build(),
Optional.of(new TDigestHistogram(memoryManager.getUtilizationHistogram())));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,8 @@ public OutputBufferInfo getInfo()
totalPagesAdded.get(),
totalRowsAdded.get(),
totalPagesAdded.get(),
ImmutableList.of());
ImmutableList.of(),
Optional.empty());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,12 +18,16 @@
import io.airlift.json.JsonCodec;
import io.airlift.stats.Distribution;
import io.airlift.stats.Distribution.DistributionSnapshot;
import io.airlift.stats.TDigest;
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.trino.plugin.base.metrics.TDigestHistogram;
import io.trino.spi.eventlistener.StageGcStatistics;
import org.joda.time.DateTime;
import org.testng.annotations.Test;

import java.util.Optional;

import static java.util.concurrent.TimeUnit.NANOSECONDS;
import static org.testng.Assert.assertEquals;

Expand Down Expand Up @@ -87,6 +91,7 @@ public class TestStageStats
new Duration(202, NANOSECONDS),

DataSize.ofBytes(34),
Optional.of(getTDigestHistogram(10)),
DataSize.ofBytes(35),
DataSize.ofBytes(36),
37,
Expand Down Expand Up @@ -177,6 +182,7 @@ private static void assertExpectedStageStats(StageStats actual)
assertEquals(actual.getFailedInputBlockedTime(), new Duration(202, NANOSECONDS));

assertEquals(actual.getBufferedDataSize(), DataSize.ofBytes(34));
assertEquals(actual.getOutputBufferUtilization().get().getMax(), 9.0);
assertEquals(actual.getOutputDataSize(), DataSize.ofBytes(35));
assertEquals(actual.getFailedOutputDataSize(), DataSize.ofBytes(36));
assertEquals(actual.getOutputPositions(), 37);
Expand Down Expand Up @@ -205,4 +211,13 @@ private static DistributionSnapshot getTestDistribution(int count)
}
return distribution.snapshot();
}

private static TDigestHistogram getTDigestHistogram(int count)
{
TDigest digest = new TDigest();
for (int i = 0; i < count; i++) {
digest.add(i);
}
return new TDigestHistogram(digest);
}
}
Loading