/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.facebook.presto.operator;
import com.facebook.presto.execution.buffer.OutputBuffer;
import com.facebook.presto.execution.buffer.PagesSerde;
import com.facebook.presto.execution.buffer.PagesSerdeFactory;
import com.facebook.presto.execution.buffer.SerializedPage;
import com.facebook.presto.spi.Page;
import com.facebook.presto.spi.PageBuilder;
import com.facebook.presto.spi.block.Block;
import com.facebook.presto.spi.block.PageBuilderStatus;
import com.facebook.presto.spi.block.RunLengthEncodedBlock;
import com.facebook.presto.spi.predicate.NullableValue;
import com.facebook.presto.spi.type.Type;
import com.facebook.presto.sql.planner.plan.PlanNodeId;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.units.DataSize;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import static com.facebook.presto.execution.buffer.PageSplitterUtil.splitPage;
import static com.facebook.presto.spi.block.PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES;
import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.util.Objects.requireNonNull;
public class PartitionedOutputOperator
implements Operator
{
public static class PartitionedOutputFactory
implements OutputFactory
{
private final PartitionFunction partitionFunction;
private final List<Integer> partitionChannels;
private final List<Optional<NullableValue>> partitionConstants;
private final OutputBuffer outputBuffer;
private final OptionalInt nullChannel;
private final DataSize maxMemory;
public PartitionedOutputFactory(
PartitionFunction partitionFunction,
List<Integer> partitionChannels,
List<Optional<NullableValue>> partitionConstants,
OptionalInt nullChannel,
OutputBuffer outputBuffer,
DataSize maxMemory)
{
this.partitionFunction = requireNonNull(partitionFunction, "partitionFunction is null");
this.partitionChannels = requireNonNull(partitionChannels, "partitionChannels is null");
this.partitionConstants = requireNonNull(partitionConstants, "partitionConstants is null");
this.nullChannel = requireNonNull(nullChannel, "nullChannel is null");
this.outputBuffer = requireNonNull(outputBuffer, "outputBuffer is null");
this.maxMemory = requireNonNull(maxMemory, "maxMemory is null");
}
@Override
public OperatorFactory createOutputOperator(
int operatorId,
PlanNodeId planNodeId,
List<Type> types,
Function<Page, Page> pagePreprocessor,
PagesSerdeFactory serdeFactory)
{
return new PartitionedOutputOperatorFactory(
operatorId,
planNodeId,
types,
pagePreprocessor,
partitionFunction,
partitionChannels,
partitionConstants,
nullChannel,
outputBuffer,
serdeFactory,
maxMemory);
}
}
public static class PartitionedOutputOperatorFactory
implements OperatorFactory
{
private final int operatorId;
private final PlanNodeId planNodeId;
private final List<Type> sourceTypes;
private final Function<Page, Page> pagePreprocessor;
private final PartitionFunction partitionFunction;
private final List<Integer> partitionChannels;
private final List<Optional<NullableValue>> partitionConstants;
private final OptionalInt nullChannel;
private final OutputBuffer outputBuffer;
private final PagesSerdeFactory serdeFactory;
private final DataSize maxMemory;
public PartitionedOutputOperatorFactory(
int operatorId,
PlanNodeId planNodeId,
List<Type> sourceTypes,
Function<Page, Page> pagePreprocessor,
PartitionFunction partitionFunction,
List<Integer> partitionChannels,
List<Optional<NullableValue>> partitionConstants,
OptionalInt nullChannel,
OutputBuffer outputBuffer,
PagesSerdeFactory serdeFactory,
DataSize maxMemory)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
this.sourceTypes = requireNonNull(sourceTypes, "sourceTypes is null");
this.pagePreprocessor = requireNonNull(pagePreprocessor, "pagePreprocessor is null");
this.partitionFunction = requireNonNull(partitionFunction, "partitionFunction is null");
this.partitionChannels = requireNonNull(partitionChannels, "partitionChannels is null");
this.partitionConstants = requireNonNull(partitionConstants, "partitionConstants is null");
this.nullChannel = requireNonNull(nullChannel, "nullChannel is null");
this.outputBuffer = requireNonNull(outputBuffer, "outputBuffer is null");
this.serdeFactory = requireNonNull(serdeFactory, "serdeFactory is null");
this.maxMemory = requireNonNull(maxMemory, "maxMemory is null");
}
@Override
public List<Type> getTypes()
{
return ImmutableList.of();
}
@Override
public Operator createOperator(DriverContext driverContext)
{
OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, PartitionedOutputOperator.class.getSimpleName());
return new PartitionedOutputOperator(
operatorContext,
sourceTypes,
pagePreprocessor,
partitionFunction,
partitionChannels,
partitionConstants,
nullChannel,
outputBuffer,
serdeFactory,
maxMemory);
}
@Override
public void close()
{
}
@Override
public OperatorFactory duplicate()
{
return new PartitionedOutputOperatorFactory(
operatorId,
planNodeId,
sourceTypes,
pagePreprocessor,
partitionFunction,
partitionChannels,
partitionConstants,
nullChannel,
outputBuffer,
serdeFactory,
maxMemory);
}
}
private final OperatorContext operatorContext;
private final Function<Page, Page> pagePreprocessor;
private final PagePartitioner partitionFunction;
private ListenableFuture<?> blocked = NOT_BLOCKED;
private boolean finished;
public PartitionedOutputOperator(
OperatorContext operatorContext,
List<Type> sourceTypes,
Function<Page, Page> pagePreprocessor,
PartitionFunction partitionFunction,
List<Integer> partitionChannels,
List<Optional<NullableValue>> partitionConstants,
OptionalInt nullChannel,
OutputBuffer outputBuffer,
PagesSerdeFactory serdeFactory,
DataSize maxMemory)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.pagePreprocessor = requireNonNull(pagePreprocessor, "pagePreprocessor is null");
this.partitionFunction = new PagePartitioner(
partitionFunction,
partitionChannels,
partitionConstants,
nullChannel,
outputBuffer,
serdeFactory,
sourceTypes,
maxMemory);
operatorContext.setInfoSupplier(this::getInfo);
// TODO: We should try to make this more accurate
// Recalculating the retained size of all the PageBuilders is somewhat expensive,
// so we only do it once here rather than in addInput(), and assume that the size will be constant.
operatorContext.getSystemMemoryContext().newLocalMemoryContext().setBytes(this.partitionFunction.getRetainedSizeInBytes());
}
@Override
public OperatorContext getOperatorContext()
{
return operatorContext;
}
public PartitionedOutputInfo getInfo()
{
return partitionFunction.getInfo();
}
@Override
public List<Type> getTypes()
{
return ImmutableList.of();
}
@Override
public void finish()
{
finished = true;
blocked = partitionFunction.flush(true);
}
@Override
public boolean isFinished()
{
return finished && isBlocked().isDone();
}
@Override
public ListenableFuture<?> isBlocked()
{
if (blocked != NOT_BLOCKED && blocked.isDone()) {
blocked = NOT_BLOCKED;
}
return blocked;
}
@Override
public boolean needsInput()
{
return !finished && isBlocked().isDone();
}
@Override
public void addInput(Page page)
{
requireNonNull(page, "page is null");
checkState(isBlocked().isDone(), "output is already blocked");
if (page.getPositionCount() == 0) {
return;
}
page = pagePreprocessor.apply(page);
blocked = partitionFunction.partitionPage(page);
operatorContext.recordGeneratedOutput(page.getSizeInBytes(), page.getPositionCount());
}
@Override
public Page getOutput()
{
return null;
}
private static class PagePartitioner
{
private final OutputBuffer outputBuffer;
private final List<Type> sourceTypes;
private final PartitionFunction partitionFunction;
private final List<Integer> partitionChannels;
private final List<Optional<Block>> partitionConstants;
private final PagesSerde serde;
private final List<PageBuilder> pageBuilders;
private final OptionalInt nullChannel; // when present, send the position to every partition if this channel is null.
private final AtomicLong rowsAdded = new AtomicLong();
private final AtomicLong pagesAdded = new AtomicLong();
public PagePartitioner(
PartitionFunction partitionFunction,
List<Integer> partitionChannels,
List<Optional<NullableValue>> partitionConstants,
OptionalInt nullChannel,
OutputBuffer outputBuffer,
PagesSerdeFactory serdeFactory,
List<Type> sourceTypes,
DataSize maxMemory)
{
this.partitionFunction = requireNonNull(partitionFunction, "partitionFunction is null");
this.partitionChannels = requireNonNull(partitionChannels, "partitionChannels is null");
this.partitionConstants = requireNonNull(partitionConstants, "partitionConstants is null").stream()
.map(constant -> constant.map(NullableValue::asBlock))
.collect(toImmutableList());
this.nullChannel = requireNonNull(nullChannel, "nullChannel is null");
this.outputBuffer = requireNonNull(outputBuffer, "outputBuffer is null");
this.sourceTypes = requireNonNull(sourceTypes, "sourceTypes is null");
this.serde = requireNonNull(serdeFactory, "serdeFactory is null").createPagesSerde();
int pageSize = Math.min(PageBuilderStatus.DEFAULT_MAX_PAGE_SIZE_IN_BYTES, ((int) maxMemory.toBytes()) / partitionFunction.getPartitionCount());
pageSize = Math.max(1, pageSize);
ImmutableList.Builder<PageBuilder> pageBuilders = ImmutableList.builder();
for (int i = 0; i < partitionFunction.getPartitionCount(); i++) {
pageBuilders.add(PageBuilder.withMaxPageSize(pageSize, sourceTypes));
}
this.pageBuilders = pageBuilders.build();
}
// Does not include size of SharedBuffer
public long getRetainedSizeInBytes()
{
return pageBuilders.stream()
.mapToLong(PageBuilder::getRetainedSizeInBytes)
.sum();
}
public PartitionedOutputInfo getInfo()
{
return new PartitionedOutputInfo(rowsAdded.get(), pagesAdded.get());
}
public ListenableFuture<?> partitionPage(Page page)
{
requireNonNull(page, "page is null");
Page partitionFunctionArgs = getPartitionFunctionArguments(page);
for (int position = 0; position < page.getPositionCount(); position++) {
if (nullChannel.isPresent() && page.getBlock(nullChannel.getAsInt()).isNull(position)) {
for (PageBuilder pageBuilder : pageBuilders) {
pageBuilder.declarePosition();
for (int channel = 0; channel < sourceTypes.size(); channel++) {
Type type = sourceTypes.get(channel);
type.appendTo(page.getBlock(channel), position, pageBuilder.getBlockBuilder(channel));
}
}
}
else {
int partition = partitionFunction.getPartition(partitionFunctionArgs, position);
PageBuilder pageBuilder = pageBuilders.get(partition);
pageBuilder.declarePosition();
for (int channel = 0; channel < sourceTypes.size(); channel++) {
Type type = sourceTypes.get(channel);
type.appendTo(page.getBlock(channel), position, pageBuilder.getBlockBuilder(channel));
}
}
}
return flush(false);
}
private Page getPartitionFunctionArguments(Page page)
{
Block[] blocks = new Block[partitionChannels.size()];
for (int i = 0; i < blocks.length; i++) {
Optional<Block> partitionConstant = partitionConstants.get(i);
if (partitionConstant.isPresent()) {
blocks[i] = new RunLengthEncodedBlock(partitionConstant.get(), page.getPositionCount());
}
else {
blocks[i] = page.getBlock(partitionChannels.get(i));
}
}
return new Page(page.getPositionCount(), blocks);
}
public ListenableFuture<?> flush(boolean force)
{
// add all full pages to output buffer
List<ListenableFuture<?>> blockedFutures = new ArrayList<>();
for (int partition = 0; partition < pageBuilders.size(); partition++) {
PageBuilder partitionPageBuilder = pageBuilders.get(partition);
if (!partitionPageBuilder.isEmpty() && (force || partitionPageBuilder.isFull())) {
Page pagePartition = partitionPageBuilder.build();
partitionPageBuilder.reset();
List<SerializedPage> serializedPages = splitPage(pagePartition, DEFAULT_MAX_PAGE_SIZE_IN_BYTES).stream()
.map(serde::serialize)
.collect(toImmutableList());
blockedFutures.add(outputBuffer.enqueue(partition, serializedPages));
pagesAdded.incrementAndGet();
rowsAdded.addAndGet(pagePartition.getPositionCount());
}
}
ListenableFuture<?> future = Futures.allAsList(blockedFutures);
if (future.isDone()) {
return NOT_BLOCKED;
}
return future;
}
}
public static class PartitionedOutputInfo
implements Mergeable<PartitionedOutputInfo>, OperatorInfo
{
private final long rowsAdded;
private final long pagesAdded;
@JsonCreator
public PartitionedOutputInfo(
@JsonProperty("rowsAdded") long rowsAdded,
@JsonProperty("pagesAdded") long pagesAdded)
{
this.rowsAdded = rowsAdded;
this.pagesAdded = pagesAdded;
}
@JsonProperty
public long getRowsAdded()
{
return rowsAdded;
}
@JsonProperty
public long getPagesAdded()
{
return pagesAdded;
}
@Override
public PartitionedOutputInfo mergeWith(PartitionedOutputInfo other)
{
return new PartitionedOutputInfo(rowsAdded + other.rowsAdded, pagesAdded + other.pagesAdded);
}
@Override
public String toString()
{
return toStringHelper(this)
.add("rowsAdded", rowsAdded)
.add("pagesAdded", pagesAdded)
.toString();
}
}
}