Skip to content
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

Use Page#getColumns in TableWriterOperator #17147

Merged
Merged
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 @@ -17,6 +17,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.common.primitives.Ints;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.slice.Slice;
Expand Down Expand Up @@ -154,7 +155,7 @@ private enum State
private final OperatorContext operatorContext;
private final LocalMemoryContext pageSinkMemoryContext;
private final ConnectorPageSink pageSink;
private final List<Integer> columnChannels;
private final int[] columnChannels;
private final AtomicLong pageSinkPeakMemoryUsage = new AtomicLong();
private final Operator statisticAggregationOperator;
private final List<Type> types;
Expand Down Expand Up @@ -183,7 +184,7 @@ public TableWriterOperator(
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.pageSinkMemoryContext = operatorContext.newLocalUserMemoryContext(TableWriterOperator.class.getSimpleName());
this.pageSink = requireNonNull(pageSink, "pageSink is null");
this.columnChannels = requireNonNull(columnChannels, "columnChannels is null");
this.columnChannels = Ints.toArray(requireNonNull(columnChannels, "columnChannels is null"));
this.statisticAggregationOperator = requireNonNull(statisticAggregationOperator, "statisticAggregationOperator is null");
this.types = ImmutableList.copyOf(requireNonNull(types, "types is null"));
this.statisticsCpuTimerEnabled = statisticsCpuTimerEnabled;
Expand Down Expand Up @@ -244,18 +245,14 @@ public void addInput(Page page)
requireNonNull(page, "page is null");
checkState(needsInput(), "Operator does not need input");

Block[] blocks = new Block[columnChannels.size()];
for (int outputChannel = 0; outputChannel < columnChannels.size(); outputChannel++) {
Block block = page.getBlock(columnChannels.get(outputChannel));
blocks[outputChannel] = block;
}

OperationTimer timer = new OperationTimer(statisticsCpuTimerEnabled);
statisticAggregationOperator.addInput(page);
timer.end(statisticsTiming);

page = page.getColumns(columnChannels);

ListenableFuture<Void> blockedOnAggregation = statisticAggregationOperator.isBlocked();
CompletableFuture<?> future = pageSink.appendPage(new Page(blocks));
CompletableFuture<?> future = pageSink.appendPage(page);
updateMemoryUsage();
ListenableFuture<?> blockedOnWrite = toListenableFuture(future);
blocked = asVoid(allAsList(blockedOnAggregation, blockedOnWrite));
Expand Down