diff --git a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java index 948fed4e6ad4..9a172e069f58 100644 --- a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java +++ b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java @@ -34,6 +34,7 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.units.DataSize.Unit.KILOBYTE; +import static io.airlift.units.DataSize.succinctBytes; import static io.trino.sql.analyzer.RegexLibrary.JONI; @DefunctConfig({ @@ -71,7 +72,7 @@ public class FeaturesConfig private boolean redistributeWrites = true; private boolean scaleWriters = true; - private DataSize writerMinSize = DataSize.of(32, DataSize.Unit.MEGABYTE); + private DataSize writerScalingMinDataProcessed = DataSize.of(120, DataSize.Unit.MEGABYTE); private DataIntegrityVerification exchangeDataIntegrityVerification = DataIntegrityVerification.ABORT; /** * default value is overwritten for fault tolerant execution in {@link #applyFaultTolerantExecutionDefaults()}} @@ -153,16 +154,25 @@ public FeaturesConfig setScaleWriters(boolean scaleWriters) } @NotNull - public DataSize getWriterMinSize() + public DataSize getWriterScalingMinDataProcessed() { - return writerMinSize; + return writerScalingMinDataProcessed; } - @Config("writer-min-size") + @Config("writer-scaling-min-data-processed") + @ConfigDescription("Minimum amount of uncompressed output data processed by writers before writer scaling can happen") + public FeaturesConfig setWriterScalingMinDataProcessed(DataSize writerScalingMinDataProcessed) + { + this.writerScalingMinDataProcessed = writerScalingMinDataProcessed; + return this; + } + + @Deprecated + @LegacyConfig(value = "writer-min-size", replacedBy = "writer-scaling-min-data-processed") @ConfigDescription("Target minimum size of writer output when scaling writers") public FeaturesConfig setWriterMinSize(DataSize writerMinSize) { - this.writerMinSize = writerMinSize; + this.writerScalingMinDataProcessed = succinctBytes(writerMinSize.toBytes() * 2); return this; } diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java index 6fd4ae6c7b5b..2f50f160ecd8 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -36,6 +36,7 @@ import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.execution.QueryManagerConfig.FAULT_TOLERANT_EXECUTION_MAX_PARTITION_COUNT_LIMIT; import static io.trino.execution.QueryManagerConfig.MAX_TASK_RETRY_ATTEMPTS; import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; @@ -84,7 +85,8 @@ public final class SystemSessionProperties public static final String TASK_SCALE_WRITERS_ENABLED = "task_scale_writers_enabled"; public static final String MAX_WRITER_TASKS_COUNT = "max_writer_tasks_count"; public static final String TASK_SCALE_WRITERS_MAX_WRITER_COUNT = "task_scale_writers_max_writer_count"; - public static final String WRITER_MIN_SIZE = "writer_min_size"; + public static final String WRITER_SCALING_MIN_DATA_PROCESSED = "writer_scaling_min_data_processed"; + public static final String SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD = "skewed_partition_min_data_processed_rebalance_threshold"; public static final String PUSH_TABLE_WRITE_THROUGH_UNION = "push_table_write_through_union"; public static final String EXECUTION_POLICY = "execution_policy"; public static final String DICTIONARY_AGGREGATION = "dictionary_aggregation"; @@ -334,10 +336,15 @@ public SystemSessionProperties( taskManagerConfig.getScaleWritersMaxWriterCount(), true), dataSizeProperty( - WRITER_MIN_SIZE, - "Target minimum size of writer output when scaling writers", - featuresConfig.getWriterMinSize(), + WRITER_SCALING_MIN_DATA_PROCESSED, + "Minimum amount of uncompressed output data processed by writers before writer scaling can happen", + featuresConfig.getWriterScalingMinDataProcessed(), false), + dataSizeProperty( + SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, + "Minimum data processed to trigger skewed partition rebalancing in local and remote exchange", + DataSize.of(50, MEGABYTE), + true), booleanProperty( PUSH_TABLE_WRITE_THROUGH_UNION, "Parallelize writes when using UNION ALL in queries that write data", @@ -1149,9 +1156,14 @@ public static int getMaxWriterTaskCount(Session session) return session.getSystemProperty(MAX_WRITER_TASKS_COUNT, Integer.class); } - public static DataSize getWriterMinSize(Session session) + public static DataSize getWriterScalingMinDataProcessed(Session session) + { + return session.getSystemProperty(WRITER_SCALING_MIN_DATA_PROCESSED, DataSize.class); + } + + public static DataSize getSkewedPartitionMinDataProcessedRebalanceThreshold(Session session) { - return session.getSystemProperty(WRITER_MIN_SIZE, DataSize.class); + return session.getSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, DataSize.class); } public static boolean isPushTableWriteThroughUnion(Session session) diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTask.java b/core/trino-main/src/main/java/io/trino/execution/SqlTask.java index 0df8b1534fee..2e6d91a581dc 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlTask.java @@ -322,6 +322,7 @@ private TaskStatus createTaskStatus(TaskHolder taskHolder) int runningPartitionedDrivers = 0; long runningPartitionedSplitsWeight = 0L; DataSize outputDataSize = DataSize.ofBytes(0); + DataSize writerInputDataSize = DataSize.ofBytes(0); DataSize physicalWrittenDataSize = DataSize.ofBytes(0); Optional writerCount = Optional.empty(); DataSize userMemoryReservation = DataSize.ofBytes(0); @@ -337,6 +338,7 @@ private TaskStatus createTaskStatus(TaskHolder taskHolder) queuedPartitionedSplitsWeight = taskStats.getQueuedPartitionedSplitsWeight(); runningPartitionedDrivers = taskStats.getRunningPartitionedDrivers(); runningPartitionedSplitsWeight = taskStats.getRunningPartitionedSplitsWeight(); + writerInputDataSize = taskStats.getWriterInputDataSize(); physicalWrittenDataSize = taskStats.getPhysicalWrittenDataSize(); writerCount = taskStats.getMaxWriterCount(); userMemoryReservation = taskStats.getUserMemoryReservation(); @@ -358,6 +360,7 @@ else if (taskHolder.getTaskExecution() != null) { runningPartitionedSplitsWeight += pipelineStatus.getRunningPartitionedSplitsWeight(); physicalWrittenBytes += pipelineContext.getPhysicalWrittenDataSize(); } + writerInputDataSize = succinctBytes(taskContext.getWriterInputDataSize()); physicalWrittenDataSize = succinctBytes(physicalWrittenBytes); writerCount = taskContext.getMaxWriterCount(); userMemoryReservation = taskContext.getMemoryReservation(); @@ -382,6 +385,7 @@ else if (taskHolder.getTaskExecution() != null) { runningPartitionedDrivers, outputBuffer.getStatus(), outputDataSize, + writerInputDataSize, physicalWrittenDataSize, writerCount, userMemoryReservation, diff --git a/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java b/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java index a1a42afc66a7..280a1c9cac4b 100644 --- a/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java +++ b/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java @@ -78,18 +78,18 @@ public class TaskManagerConfig private Duration interruptStuckSplitTasksDetectionInterval = new Duration(2, TimeUnit.MINUTES); private boolean scaleWritersEnabled = true; - // Set the value of default max writer count to the number of processors and cap it to 32. We can do this + // Set the value of default max writer count to 2 * max(the number of processors, 32). We can do this // because preferred write partitioning is always enabled for local exchange thus partitioned inserts will never // use this property. Hence, there is no risk in terms of more numbers of physical writers which can cause high // resource utilization. - private int scaleWritersMaxWriterCount = min(getAvailablePhysicalProcessorCount(), 32); + private int scaleWritersMaxWriterCount = min(getAvailablePhysicalProcessorCount(), 32) * 2; private int writerCount = 1; // Default value of partitioned task writer count should be above 1, otherwise it can create a plan // with a single gather exchange node on the coordinator due to a single available processor. Whereas, // on the worker nodes due to more available processors, the default value could be above 1. Therefore, // it can cause error due to config mismatch during execution. Additionally, cap it to 32 in order to // avoid small pages produced by local partitioning exchanges. - private int partitionedWriterCount = min(max(nextPowerOfTwo(getAvailablePhysicalProcessorCount()), 2), 32); + private int partitionedWriterCount = min(max(nextPowerOfTwo(getAvailablePhysicalProcessorCount()), 2), 32) * 2; // Default value of task concurrency should be above 1, otherwise it can create a plan with a single gather // exchange node on the coordinator due to a single available processor. Whereas, on the worker nodes due to // more available processors, the default value could be above 1. Therefore, it can cause error due to config diff --git a/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java b/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java index d89a81f3f40a..6940d6e55483 100644 --- a/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java +++ b/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java @@ -59,6 +59,7 @@ public class TaskStatus private final int runningPartitionedDrivers; private final long runningPartitionedSplitsWeight; private final OutputBufferStatus outputBufferStatus; + private final DataSize writerInputDataSize; private final DataSize outputDataSize; private final DataSize physicalWrittenDataSize; private final Optional maxWriterCount; @@ -87,6 +88,7 @@ public TaskStatus( @JsonProperty("runningPartitionedDrivers") int runningPartitionedDrivers, @JsonProperty("outputBufferStatus") OutputBufferStatus outputBufferStatus, @JsonProperty("outputDataSize") DataSize outputDataSize, + @JsonProperty("writerInputDataSize") DataSize writerInputDataSize, @JsonProperty("physicalWrittenDataSize") DataSize physicalWrittenDataSize, @JsonProperty("writerCount") Optional maxWriterCount, @JsonProperty("memoryReservation") DataSize memoryReservation, @@ -121,6 +123,7 @@ public TaskStatus( this.outputBufferStatus = requireNonNull(outputBufferStatus, "outputBufferStatus is null"); this.outputDataSize = requireNonNull(outputDataSize, "outputDataSize is null"); + this.writerInputDataSize = requireNonNull(writerInputDataSize, "writerInputDataSize is null"); this.physicalWrittenDataSize = requireNonNull(physicalWrittenDataSize, "physicalWrittenDataSize is null"); this.maxWriterCount = requireNonNull(maxWriterCount, "maxWriterCount is null"); @@ -196,6 +199,12 @@ public int getRunningPartitionedDrivers() return runningPartitionedDrivers; } + @JsonProperty + public DataSize getWriterInputDataSize() + { + return writerInputDataSize; + } + @JsonProperty public DataSize getPhysicalWrittenDataSize() { @@ -293,6 +302,7 @@ public static TaskStatus initialTaskStatus(TaskId taskId, URI location, String n OutputBufferStatus.initial(), DataSize.ofBytes(0), DataSize.ofBytes(0), + DataSize.ofBytes(0), Optional.empty(), DataSize.ofBytes(0), DataSize.ofBytes(0), @@ -319,6 +329,7 @@ public static TaskStatus failWith(TaskStatus taskStatus, TaskState state, List> writerTasksProvider; private final NodeSelector nodeSelector; private final ScheduledExecutorService executor; - private final long writerMinSizeBytes; + private final long writerScalingMinDataProcessed; private final Set scheduledNodes = new HashSet<>(); private final AtomicBoolean done = new AtomicBoolean(); private final int maxWriterNodeCount; @@ -59,7 +59,7 @@ public ScaledWriterScheduler( Supplier> writerTasksProvider, NodeSelector nodeSelector, ScheduledExecutorService executor, - DataSize writerMinSize, + DataSize writerScalingMinDataProcessed, int maxWriterNodeCount) { this.stage = requireNonNull(stage, "stage is null"); @@ -67,7 +67,7 @@ public ScaledWriterScheduler( this.writerTasksProvider = requireNonNull(writerTasksProvider, "writerTasksProvider is null"); this.nodeSelector = requireNonNull(nodeSelector, "nodeSelector is null"); this.executor = requireNonNull(executor, "executor is null"); - this.writerMinSizeBytes = writerMinSize.toBytes(); + this.writerScalingMinDataProcessed = writerScalingMinDataProcessed.toBytes(); this.maxWriterNodeCount = maxWriterNodeCount; } @@ -120,17 +120,17 @@ private boolean isSourceTasksBufferFull() private boolean isWriteThroughputSufficient() { Collection writerTasks = writerTasksProvider.get(); - long writtenBytes = writerTasks.stream() - .map(TaskStatus::getPhysicalWrittenDataSize) + long writerInputBytes = writerTasks.stream() + .map(TaskStatus::getWriterInputDataSize) .mapToLong(DataSize::toBytes) .sum(); - long minWrittenBytesToScaleUp = writerTasks.stream() + long minWriterInputBytesToScaleUp = writerTasks.stream() .map(TaskStatus::getMaxWriterCount) .map(Optional::get) - .mapToLong(writerCount -> writerMinSizeBytes * writerCount) + .mapToLong(writerCount -> writerScalingMinDataProcessed * writerCount) .sum(); - return writtenBytes >= minWrittenBytesToScaleUp; + return writerInputBytes >= minWriterInputBytesToScaleUp; } private boolean isWeightedBufferFull() diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java index 9c3b61aa6bfb..0ea5d3a5d70e 100644 --- a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java @@ -266,6 +266,16 @@ public CounterStat getOutputPositions() return new CounterStat(); } + public long getWriterInputDataSize() + { + // Avoid using stream api for performance reasons + long writerInputDataSize = 0; + for (OperatorContext context : operatorContexts) { + writerInputDataSize += context.getWriterInputDataSize(); + } + return writerInputDataSize; + } + public long getPhysicalWrittenDataSize() { // Avoid using stream api for performance reasons diff --git a/core/trino-main/src/main/java/io/trino/operator/OperatorContext.java b/core/trino-main/src/main/java/io/trino/operator/OperatorContext.java index 46b1090f73e5..baedf47f8555 100644 --- a/core/trino-main/src/main/java/io/trino/operator/OperatorContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/OperatorContext.java @@ -90,6 +90,7 @@ public class OperatorContext private final AtomicReference metrics = new AtomicReference<>(Metrics.EMPTY); // this is not incremental, but gets overwritten by the latest value. private final AtomicReference connectorMetrics = new AtomicReference<>(Metrics.EMPTY); // this is not incremental, but gets overwritten by the latest value. + private final AtomicLong writerInputDataSize = new AtomicLong(); private final AtomicLong physicalWrittenDataSize = new AtomicLong(); private final AtomicReference> memoryFuture; @@ -245,6 +246,11 @@ public void setFinishedFuture(ListenableFuture finishedFuture) checkState(this.finishedFuture.getAndSet(requireNonNull(finishedFuture, "finishedFuture is null")) == null, "finishedFuture already set"); } + public void recordWriterInputDataSize(long sizeInBytes) + { + writerInputDataSize.getAndAdd(sizeInBytes); + } + public void recordPhysicalWrittenData(long sizeInBytes) { physicalWrittenDataSize.getAndAdd(sizeInBytes); @@ -485,6 +491,11 @@ public CounterStat getOutputPositions() return outputPositions; } + public long getWriterInputDataSize() + { + return writerInputDataSize.get(); + } + public long getPhysicalWrittenDataSize() { return physicalWrittenDataSize.get(); diff --git a/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java b/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java index 8c5c7a7d59a8..605d970564e1 100644 --- a/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java @@ -317,6 +317,16 @@ public CounterStat getOutputPositions() return stat; } + public long getWriterInputDataSize() + { + // Avoid using stream api due to performance reasons + long writerInputDataSize = 0; + for (DriverContext context : drivers) { + writerInputDataSize += context.getWriterInputDataSize(); + } + return writerInputDataSize; + } + public long getPhysicalWrittenDataSize() { // Avoid using stream api due to performance reasons diff --git a/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java b/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java index bf15e46339de..37619858e1e4 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/TableWriterOperator.java @@ -258,6 +258,7 @@ public void addInput(Page page) blocked = asVoid(allAsList(blockedOnAggregation, blockedOnWrite)); rowCount += page.getPositionCount(); updateWrittenBytes(); + operatorContext.recordWriterInputDataSize(page.getSizeInBytes()); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java index 4f214e5c1828..0cffc6deccfc 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java @@ -354,6 +354,16 @@ public CounterStat getOutputPositions() return stat; } + public long getWriterInputDataSize() + { + // Avoid using stream api due to performance reasons + long writerInputDataSize = 0; + for (PipelineContext context : pipelineContexts) { + writerInputDataSize += context.getWriterInputDataSize(); + } + return writerInputDataSize; + } + public long getPhysicalWrittenDataSize() { // Avoid using stream api for performance reasons @@ -599,6 +609,7 @@ public TaskStats getTaskStats() succinctBytes(outputDataSize), outputPositions, new Duration(outputBlockedTime, NANOSECONDS).convertToMostSuccinctTimeUnit(), + succinctBytes(getWriterInputDataSize()), succinctBytes(physicalWrittenDataSize), getMaxWriterCount(), fullGcCount, diff --git a/core/trino-main/src/main/java/io/trino/operator/TaskStats.java b/core/trino-main/src/main/java/io/trino/operator/TaskStats.java index 1dc3e3d6103d..d5de839a9ea5 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TaskStats.java +++ b/core/trino-main/src/main/java/io/trino/operator/TaskStats.java @@ -83,6 +83,7 @@ public class TaskStats private final Duration outputBlockedTime; + private final DataSize writerInputDataSize; private final DataSize physicalWrittenDataSize; private final Optional maxWriterCount; @@ -133,6 +134,7 @@ public TaskStats(DateTime createTime, DateTime endTime) 0, new Duration(0, MILLISECONDS), DataSize.ofBytes(0), + DataSize.ofBytes(0), Optional.empty(), 0, new Duration(0, MILLISECONDS), @@ -191,6 +193,7 @@ public TaskStats( @JsonProperty("outputBlockedTime") Duration outputBlockedTime, + @JsonProperty("writerInputDataSize") DataSize writerInputDataSize, @JsonProperty("physicalWrittenDataSize") DataSize physicalWrittenDataSize, @JsonProperty("writerCount") Optional writerCount, @@ -266,6 +269,7 @@ public TaskStats( this.outputBlockedTime = requireNonNull(outputBlockedTime, "outputBlockedTime is null"); + this.writerInputDataSize = requireNonNull(writerInputDataSize, "writerInputDataSize is null"); this.physicalWrittenDataSize = requireNonNull(physicalWrittenDataSize, "physicalWrittenDataSize is null"); this.maxWriterCount = requireNonNull(writerCount, "writerCount is null"); @@ -491,6 +495,12 @@ public Duration getOutputBlockedTime() return outputBlockedTime; } + @JsonProperty + public DataSize getWriterInputDataSize() + { + return writerInputDataSize; + } + @JsonProperty public DataSize getPhysicalWrittenDataSize() { @@ -587,6 +597,7 @@ public TaskStats summarize() outputDataSize, outputPositions, outputBlockedTime, + writerInputDataSize, physicalWrittenDataSize, maxWriterCount, fullGcCount, @@ -636,6 +647,7 @@ public TaskStats summarizeFinal() outputDataSize, outputPositions, outputBlockedTime, + writerInputDataSize, physicalWrittenDataSize, maxWriterCount, fullGcCount, diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java index 6a6a18d365f7..3fd4dca42b64 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java @@ -26,6 +26,7 @@ import io.trino.operator.InterpretedHashGenerator; import io.trino.operator.PartitionFunction; import io.trino.operator.PrecomputedHashGenerator; +import io.trino.operator.output.SkewedPartitionRebalancer; import io.trino.spi.Page; import io.trino.spi.type.Type; import io.trino.sql.planner.MergePartitioningHandle; @@ -33,16 +34,14 @@ import io.trino.sql.planner.PartitioningHandle; import io.trino.sql.planner.SystemPartitioningHandle; import io.trino.type.BlockTypeOperators; -import it.unimi.dsi.fastutil.longs.Long2LongMap; -import it.unimi.dsi.fastutil.longs.Long2LongOpenHashMap; import java.io.Closeable; import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -51,6 +50,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.SystemSessionProperties.getSkewedPartitionMinDataProcessedRebalanceThreshold; import static io.trino.operator.exchange.LocalExchangeSink.finishedLocalExchangeSink; import static io.trino.sql.planner.PartitioningHandle.isScaledWriterHashDistribution; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; @@ -71,9 +71,6 @@ public class LocalExchange private final List sources; - // Physical written bytes for each writer in the same order as source buffers - private final List> physicalWrittenBytesSuppliers = new CopyOnWriteArrayList<>(); - @GuardedBy("this") private boolean allSourcesFinished; @@ -99,7 +96,7 @@ public LocalExchange( Optional partitionHashChannel, DataSize maxBufferedBytes, BlockTypeOperators blockTypeOperators, - DataSize writerMinSize) + DataSize writerScalingMinDataProcessed) { int bufferCount = computeBufferCount(partitioning, defaultConcurrency, partitionChannels); @@ -129,30 +126,22 @@ else if (partitioning.equals(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION)) { sources = IntStream.range(0, bufferCount) .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) .collect(toImmutableList()); + AtomicLong dataProcessed = new AtomicLong(0); exchangerSupplier = () -> new ScaleWriterExchanger( asPageConsumers(sources), memoryManager, maxBufferedBytes.toBytes(), - () -> { - // Avoid using stream api for performance reasons - long physicalWrittenBytes = 0; - for (Supplier physicalWrittenBytesSupplier : physicalWrittenBytesSuppliers) { - physicalWrittenBytes += physicalWrittenBytesSupplier.get(); - } - return physicalWrittenBytes; - }, - writerMinSize); + dataProcessed, + writerScalingMinDataProcessed); } else if (isScaledWriterHashDistribution(partitioning)) { int partitionCount = bufferCount * SCALE_WRITERS_MAX_PARTITIONS_PER_WRITER; - List> writerPartitionRowCountsSuppliers = new CopyOnWriteArrayList<>(); - UniformPartitionRebalancer uniformPartitionRebalancer = new UniformPartitionRebalancer( - physicalWrittenBytesSuppliers, - () -> computeAggregatedPartitionRowCounts(writerPartitionRowCountsSuppliers), + SkewedPartitionRebalancer skewedPartitionRebalancer = new SkewedPartitionRebalancer( partitionCount, bufferCount, - writerMinSize.toBytes()); - + 1, + writerScalingMinDataProcessed.toBytes(), + getSkewedPartitionMinDataProcessedRebalanceThreshold(session).toBytes()); LocalExchangeMemoryManager memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); sources = IntStream.range(0, bufferCount) .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) @@ -168,16 +157,14 @@ else if (isScaledWriterHashDistribution(partitioning)) { partitionChannels, partitionChannelTypes, partitionHashChannel); - ScaleWriterPartitioningExchanger exchanger = new ScaleWriterPartitioningExchanger( + return new ScaleWriterPartitioningExchanger( asPageConsumers(sources), memoryManager, maxBufferedBytes.toBytes(), createPartitionPagePreparer(partitioning, partitionChannels), partitionFunction, partitionCount, - uniformPartitionRebalancer); - writerPartitionRowCountsSuppliers.add(exchanger::getAndResetPartitionRowCounts); - return exchanger; + skewedPartitionRebalancer); }; } else if (partitioning.equals(FIXED_HASH_DISTRIBUTION) || partitioning.getCatalogHandle().isPresent() || @@ -221,29 +208,14 @@ public synchronized LocalExchangeSinkFactory createSinkFactory() return newFactory; } - public synchronized LocalExchangeSource getNextSource(Supplier physicalWrittenBytesSupplier) + public synchronized LocalExchangeSource getNextSource() { checkState(nextSourceIndex < sources.size(), "All operators already created"); LocalExchangeSource result = sources.get(nextSourceIndex); - physicalWrittenBytesSuppliers.add(physicalWrittenBytesSupplier); nextSourceIndex++; return result; } - private Long2LongMap computeAggregatedPartitionRowCounts(List> writerPartitionRowCountsSuppliers) - { - Long2LongMap aggregatedPartitionRowCounts = new Long2LongOpenHashMap(); - List writerPartitionRowCounts = writerPartitionRowCountsSuppliers.stream() - .map(Supplier::get) - .collect(toImmutableList()); - - writerPartitionRowCounts.forEach(partitionRowCounts -> - partitionRowCounts.forEach((writerPartitionId, rowCount) -> - aggregatedPartitionRowCounts.merge(writerPartitionId.longValue(), rowCount.longValue(), Long::sum))); - - return aggregatedPartitionRowCounts; - } - private static Function createPartitionPagePreparer(PartitioningHandle partitioning, List partitionChannels) { Function partitionPagePreparer; diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java index 3a78a8deb96c..57b6153d3b47 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java @@ -48,7 +48,7 @@ public Operator createOperator(DriverContext driverContext) checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, LocalExchangeSourceOperator.class.getSimpleName()); - return new LocalExchangeSourceOperator(operatorContext, localExchange.getNextSource(driverContext::getPhysicalWrittenDataSize)); + return new LocalExchangeSourceOperator(operatorContext, localExchange.getNextSource()); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java index 19172faf1744..ccf07a621a6f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java @@ -77,7 +77,7 @@ public Operator createOperator(DriverContext driverContext) PageWithPositionComparator comparator = orderingCompiler.compilePageWithPositionComparator(types, sortChannels, orderings); List sources = IntStream.range(0, localExchange.getBufferCount()) .boxed() - .map(index -> localExchange.getNextSource(driverContext::getPhysicalWrittenDataSize)) + .map(index -> localExchange.getNextSource()) .collect(toImmutableList()); return new LocalMergeSourceOperator(operatorContext, sources, types, comparator); } diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java index 2b217790e7e7..dcc359c03a26 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java @@ -20,13 +20,13 @@ import io.trino.spi.Page; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import java.util.function.Supplier; import static java.util.Objects.requireNonNull; /** - * Scale up local writers based on throughput and physical written bytes. + * Scale up local writers based on throughput and data processed by writers. * Input pages are distributed across different writers in a round-robin fashion. */ public class ScaleWriterExchanger @@ -37,32 +37,32 @@ public class ScaleWriterExchanger private final List> buffers; private final LocalExchangeMemoryManager memoryManager; private final long maxBufferedBytes; - private final Supplier physicalWrittenBytesSupplier; - private final long writerMinSize; + private final AtomicLong dataProcessed; + private final long writerScalingMinDataProcessed; // Start with single writer and increase the writer count based on - // physical written bytes and buffer utilization. + // data processed by writers and buffer utilization. private int writerCount = 1; - private long lastScaleUpPhysicalWrittenBytes; private int nextWriterIndex = -1; public ScaleWriterExchanger( List> buffers, LocalExchangeMemoryManager memoryManager, long maxBufferedBytes, - Supplier physicalWrittenBytesSupplier, - DataSize writerMinSize) + AtomicLong dataProcessed, + DataSize writerScalingMinDataProcessed) { this.buffers = requireNonNull(buffers, "buffers is null"); this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); this.maxBufferedBytes = maxBufferedBytes; - this.physicalWrittenBytesSupplier = requireNonNull(physicalWrittenBytesSupplier, "physicalWrittenBytesSupplier is null"); - this.writerMinSize = writerMinSize.toBytes(); + this.dataProcessed = requireNonNull(dataProcessed, "dataProcessed is null"); + this.writerScalingMinDataProcessed = writerScalingMinDataProcessed.toBytes(); } @Override public void accept(Page page) { + dataProcessed.addAndGet(page.getSizeInBytes()); Consumer buffer = buffers.get(getNextWriterIndex()); memoryManager.updateMemoryUsage(page.getRetainedSizeInBytes()); buffer.accept(page); @@ -71,14 +71,11 @@ public void accept(Page page) private int getNextWriterIndex() { // Scale up writers when current buffer memory utilization is more than 50% of the - // maximum and physical written bytes by the last scaled up writer is greater than - // writerMinSize. + // maximum and data processed is greater than current writer count * writerScalingMinOutputSize. // This also mean that we won't scale local writers if the writing speed can cope up // with incoming data. In another word, buffer utilization is below 50%. if (writerCount < buffers.size() && memoryManager.getBufferedBytes() >= maxBufferedBytes / 2) { - long physicalWrittenBytes = physicalWrittenBytesSupplier.get(); - if ((physicalWrittenBytes - lastScaleUpPhysicalWrittenBytes) >= writerCount * writerMinSize) { - lastScaleUpPhysicalWrittenBytes = physicalWrittenBytes; + if (dataProcessed.get() >= writerCount * writerScalingMinDataProcessed) { writerCount++; log.debug("Increased task writer count: %d", writerCount); } diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java index 0532f4cd09c9..77c038278c2c 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java @@ -15,21 +15,15 @@ package io.trino.operator.exchange; import com.google.common.util.concurrent.ListenableFuture; -import com.google.errorprone.annotations.concurrent.GuardedBy; import io.trino.operator.PartitionFunction; +import io.trino.operator.output.SkewedPartitionRebalancer; import io.trino.spi.Page; import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.longs.Long2IntMap; -import it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap; -import it.unimi.dsi.fastutil.longs.Long2LongMap; -import it.unimi.dsi.fastutil.longs.Long2LongOpenHashMap; import java.util.List; import java.util.function.Consumer; import java.util.function.Function; -import static io.trino.operator.exchange.UniformPartitionRebalancer.WriterPartitionId; -import static io.trino.operator.exchange.UniformPartitionRebalancer.WriterPartitionId.serialize; import static java.util.Arrays.fill; import static java.util.Objects.requireNonNull; @@ -41,24 +35,13 @@ public class ScaleWriterPartitioningExchanger private final long maxBufferedBytes; private final Function partitionedPagePreparer; private final PartitionFunction partitionFunction; - private final UniformPartitionRebalancer partitionRebalancer; + private final SkewedPartitionRebalancer partitionRebalancer; private final IntArrayList[] writerAssignments; private final int[] partitionRowCounts; private final int[] partitionWriterIds; private final int[] partitionWriterIndexes; - private final IntArrayList usedPartitions = new IntArrayList(); - - // Use Long2IntMap instead of Map which helps to save memory in the worst case scenario. - // Here first 32 bit of long key contains writerId whereas last 32 bit contains partitionId. - private final Long2IntMap pageWriterPartitionRowCounts = new Long2IntOpenHashMap(); - - // Use Long2LongMap instead of Map which helps to save memory in the worst case scenario. - // Here first 32 bit of long key contains writerId whereas last 32 bit contains partitionId. - @GuardedBy("this") - private final Long2LongMap writerPartitionRowCounts = new Long2LongOpenHashMap(); - public ScaleWriterPartitioningExchanger( List> buffers, LocalExchangeMemoryManager memoryManager, @@ -66,7 +49,7 @@ public ScaleWriterPartitioningExchanger( Function partitionedPagePreparer, PartitionFunction partitionFunction, int partitionCount, - UniformPartitionRebalancer partitionRebalancer) + SkewedPartitionRebalancer partitionRebalancer) { this.buffers = requireNonNull(buffers, "buffers is null"); this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); @@ -95,7 +78,7 @@ public void accept(Page page) { // Scale up writers when current buffer memory utilization is more than 50% of the maximum if (memoryManager.getBufferedBytes() > maxBufferedBytes * 0.5) { - partitionRebalancer.rebalancePartitions(); + partitionRebalancer.rebalance(); } Page partitionPage = partitionedPagePreparer.apply(page); @@ -114,27 +97,17 @@ public void accept(Page page) if (writerId == -1) { writerId = getNextWriterId(partitionId); partitionWriterIds[partitionId] = writerId; - usedPartitions.add(partitionId); } writerAssignments[writerId].add(position); } - for (int partitionId : usedPartitions) { - int writerId = partitionWriterIds[partitionId]; - pageWriterPartitionRowCounts.put(serialize(new WriterPartitionId(writerId, partitionId)), partitionRowCounts[partitionId]); - - // Reset the value of partition row count and writer id for the next page processing cycle + for (int partitionId = 0; partitionId < partitionRowCounts.length; partitionId++) { + partitionRebalancer.addPartitionRowCount(partitionId, partitionRowCounts[partitionId]); + // Reset the value of partition row count partitionRowCounts[partitionId] = 0; partitionWriterIds[partitionId] = -1; } - // Update partitions row count state which will help with scaling partitions across writers - updatePartitionRowCounts(pageWriterPartitionRowCounts); - - // Reset pageWriterPartitionRowCounts and usedPartitions for the next page processing cycle - pageWriterPartitionRowCounts.clear(); - usedPartitions.clear(); - // build a page for each writer for (int bucket = 0; bucket < writerAssignments.length; bucket++) { IntArrayList positionsList = writerAssignments[bucket]; @@ -167,27 +140,16 @@ public ListenableFuture waitForWriting() return memoryManager.getNotFullFuture(); } - public synchronized Long2LongMap getAndResetPartitionRowCounts() - { - Long2LongMap result = new Long2LongOpenHashMap(writerPartitionRowCounts); - writerPartitionRowCounts.clear(); - return result; - } - - private synchronized void updatePartitionRowCounts(Long2IntMap pagePartitionRowCounts) - { - pagePartitionRowCounts.forEach((writerPartitionId, rowCount) -> - writerPartitionRowCounts.merge(writerPartitionId, rowCount, Long::sum)); - } - private int getNextWriterId(int partitionId) { - return partitionRebalancer.getWriterId(partitionId, partitionWriterIndexes[partitionId]++); + return partitionRebalancer.getTaskId(partitionId, partitionWriterIndexes[partitionId]++); } private void sendPageToPartition(Consumer buffer, Page pageSplit) { - memoryManager.updateMemoryUsage(pageSplit.getRetainedSizeInBytes()); + long retainedSizeInBytes = pageSplit.getRetainedSizeInBytes(); + partitionRebalancer.addDataProcessed(retainedSizeInBytes); + memoryManager.updateMemoryUsage(retainedSizeInBytes); buffer.accept(pageSplit); } } diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/UniformPartitionRebalancer.java b/core/trino-main/src/main/java/io/trino/operator/exchange/UniformPartitionRebalancer.java deleted file mode 100644 index 2a0802f27b82..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/UniformPartitionRebalancer.java +++ /dev/null @@ -1,446 +0,0 @@ -/* - * 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 io.trino.operator.exchange; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.errorprone.annotations.ThreadSafe; -import io.airlift.log.Logger; -import io.airlift.units.DataSize; -import io.trino.execution.resourcegroups.IndexedPriorityQueue; -import it.unimi.dsi.fastutil.longs.Long2LongMap; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicLongArray; -import java.util.function.Supplier; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static java.lang.Double.isNaN; -import static java.lang.Math.floorMod; -import static java.lang.Math.max; -import static java.util.Objects.requireNonNull; - -/** - * Help in finding the skewness across writers when writing partitioned data using preferred partitioning. - * It then tries to uniformly distribute the biggest partitions from skewed writers to all the available writers. - *

- * Example: - *

- * Before: For three writers with skewed partitions - * Writer 1 -> No partition assigned -> 0 bytes - * Writer 2 -> No partition assigned -> 0 bytes - * Writer 3 -> Partition 1 (100MB) + Partition 2 (100MB) + Partition 3 (100MB) -> 300 MB - *

- * After scaling: - * Writer 1 -> Partition 1 (50MB) + Partition 3 (50MB) -> 100 MB - * Writer 2 -> Partition 2 (50MB) -> 50 MB - * Writer 3 -> Partition 1 (150MB) + Partition 2 (150MB) + Partition 3 (150MB) -> 450 MB - */ -@ThreadSafe -public class UniformPartitionRebalancer -{ - private static final Logger log = Logger.get(UniformPartitionRebalancer.class); - // If the percentage difference between the two writers with maximum and minimum physical written bytes - // since last rebalance is above 0.7 (or 70%), then we consider them skewed. - private static final double SKEWNESS_THRESHOLD = 0.7; - - private final List> writerPhysicalWrittenBytesSuppliers; - // Use Long2LongMap instead of Map which helps to save memory in the worst case scenario. - // Here first 32 bit of Long key contains writerId whereas last 32 bit contains partitionId. - private final Supplier partitionRowCountsSupplier; - private final long writerMinSize; - private final int numberOfWriters; - private final long rebalanceThresholdMinPhysicalWrittenBytes; - - private final AtomicLongArray writerPhysicalWrittenBytesAtLastRebalance; - - private final PartitionInfo[] partitionInfos; - - public UniformPartitionRebalancer( - List> writerPhysicalWrittenBytesSuppliers, - Supplier partitionRowCountsSupplier, - int partitionCount, - int numberOfWriters, - long writerMinSize) - { - this.writerPhysicalWrittenBytesSuppliers = requireNonNull(writerPhysicalWrittenBytesSuppliers, "writerPhysicalWrittenBytesSuppliers is null"); - this.partitionRowCountsSupplier = requireNonNull(partitionRowCountsSupplier, "partitionRowCountsSupplier is null"); - this.writerMinSize = writerMinSize; - this.numberOfWriters = numberOfWriters; - this.rebalanceThresholdMinPhysicalWrittenBytes = max(DataSize.of(50, MEGABYTE).toBytes(), writerMinSize); - - this.writerPhysicalWrittenBytesAtLastRebalance = new AtomicLongArray(numberOfWriters); - - partitionInfos = new PartitionInfo[partitionCount]; - for (int i = 0; i < partitionCount; i++) { - partitionInfos[i] = new PartitionInfo(i % numberOfWriters); - } - } - - public int getWriterId(int partitionId, int index) - { - return partitionInfos[partitionId].getWriterId(index); - } - - @VisibleForTesting - List getWriterIds(int partitionId) - { - return partitionInfos[partitionId].getWriterIds(); - } - - public void rebalancePartitions() - { - List writerPhysicalWrittenBytes = writerPhysicalWrittenBytesSuppliers.stream() - .map(Supplier::get) - .collect(toImmutableList()); - - // Rebalance only when total bytes written since last rebalance is greater than rebalance threshold - if (getPhysicalWrittenBytesSinceLastRebalance(writerPhysicalWrittenBytes) > rebalanceThresholdMinPhysicalWrittenBytes) { - rebalancePartitions(writerPhysicalWrittenBytes); - } - } - - private int getPhysicalWrittenBytesSinceLastRebalance(List writerPhysicalWrittenBytes) - { - int physicalWrittenBytesSinceLastRebalance = 0; - for (int writerId = 0; writerId < writerPhysicalWrittenBytes.size(); writerId++) { - physicalWrittenBytesSinceLastRebalance += - writerPhysicalWrittenBytes.get(writerId) - writerPhysicalWrittenBytesAtLastRebalance.get(writerId); - } - - return physicalWrittenBytesSinceLastRebalance; - } - - private synchronized void rebalancePartitions(List writerPhysicalWrittenBytes) - { - Long2LongMap partitionRowCounts = partitionRowCountsSupplier.get(); - RebalanceContext context = new RebalanceContext(writerPhysicalWrittenBytes, partitionRowCounts); - - IndexedPriorityQueue maxWriters = new IndexedPriorityQueue<>(); - IndexedPriorityQueue minWriters = new IndexedPriorityQueue<>(); - for (int writerId = 0; writerId < numberOfWriters; writerId++) { - WriterId writer = new WriterId(writerId); - maxWriters.addOrUpdate(writer, context.getWriterEstimatedWrittenBytes(writer)); - minWriters.addOrUpdate(writer, Long.MAX_VALUE - context.getWriterEstimatedWrittenBytes(writer)); - } - - // Find skewed partitions and scale them across multiple writers - while (true) { - // Find the writer with maximum physical written bytes since last rebalance - WriterId maxWriter = maxWriters.poll(); - - if (maxWriter == null) { - break; - } - - // Find the skewness against writer with max physical written bytes since last rebalance - List minSkewedWriters = findSkewedMinWriters(context, maxWriter, minWriters); - if (minSkewedWriters.isEmpty()) { - break; - } - - for (WriterId minSkewedWriter : minSkewedWriters) { - // There's no need to add the maxWriter back to priority queues if no partition rebalancing happened - List affectedWriters = context.rebalancePartition(maxWriter, minSkewedWriter); - if (!affectedWriters.isEmpty()) { - for (WriterId affectedWriter : affectedWriters) { - maxWriters.addOrUpdate(affectedWriter, context.getWriterEstimatedWrittenBytes(maxWriter)); - minWriters.addOrUpdate(affectedWriter, Long.MAX_VALUE - context.getWriterEstimatedWrittenBytes(maxWriter)); - } - break; - } - } - - // Add all the min skewed writers back to the minWriters queue with updated priorities - for (WriterId minSkewedWriter : minSkewedWriters) { - maxWriters.addOrUpdate(minSkewedWriter, context.getWriterEstimatedWrittenBytes(minSkewedWriter)); - minWriters.addOrUpdate(minSkewedWriter, Long.MAX_VALUE - context.getWriterEstimatedWrittenBytes(minSkewedWriter)); - } - } - - resetStateForNextRebalance(context, writerPhysicalWrittenBytes, partitionRowCounts); - } - - private List findSkewedMinWriters(RebalanceContext context, WriterId maxWriter, IndexedPriorityQueue minWriters) - { - ImmutableList.Builder minSkewedWriters = ImmutableList.builder(); - long maxWriterWrittenBytes = context.getWriterEstimatedWrittenBytes(maxWriter); - while (true) { - // Find the writer with minimum written bytes since last rebalance - WriterId minWriter = minWriters.poll(); - if (minWriter == null) { - break; - } - - long minWriterWrittenBytes = context.getWriterEstimatedWrittenBytes(minWriter); - - // find the skewness against writer with max written bytes since last rebalance - double skewness = ((double) (maxWriterWrittenBytes - minWriterWrittenBytes)) / maxWriterWrittenBytes; - if (skewness <= SKEWNESS_THRESHOLD || isNaN(skewness)) { - break; - } - - minSkewedWriters.add(minWriter); - } - return minSkewedWriters.build(); - } - - private void resetStateForNextRebalance(RebalanceContext context, List writerPhysicalWrittenBytes, Long2LongMap partitionRowCounts) - { - partitionRowCounts.forEach((serializedKey, rowCount) -> { - WriterPartitionId writerPartitionId = WriterPartitionId.deserialize(serializedKey); - PartitionInfo partitionInfo = partitionInfos[writerPartitionId.partitionId]; - if (context.isPartitionRebalanced(writerPartitionId.partitionId)) { - // Reset physical written bytes for rebalanced partitions - partitionInfo.resetPhysicalWrittenBytesAtLastRebalance(); - } - else { - long writtenBytes = context.estimatePartitionWrittenBytesSinceLastRebalance(new WriterId(writerPartitionId.writerId), rowCount); - partitionInfo.addToPhysicalWrittenBytesAtLastRebalance(writtenBytes); - } - }); - - for (int i = 0; i < numberOfWriters; i++) { - writerPhysicalWrittenBytesAtLastRebalance.set(i, writerPhysicalWrittenBytes.get(i)); - } - } - - private class RebalanceContext - { - private final Set rebalancedPartitions = new HashSet<>(); - private final long[] writerPhysicalWrittenBytesSinceLastRebalance; - private final long[] writerRowCountSinceLastRebalance; - private final long[] writerEstimatedWrittenBytes; - private final List> writerMaxPartitions; - - private RebalanceContext(List writerPhysicalWrittenBytes, Long2LongMap partitionRowCounts) - { - writerPhysicalWrittenBytesSinceLastRebalance = new long[numberOfWriters]; - writerEstimatedWrittenBytes = new long[numberOfWriters]; - for (int writerId = 0; writerId < writerPhysicalWrittenBytes.size(); writerId++) { - long physicalWrittenBytesSinceLastRebalance = - writerPhysicalWrittenBytes.get(writerId) - writerPhysicalWrittenBytesAtLastRebalance.get(writerId); - writerPhysicalWrittenBytesSinceLastRebalance[writerId] = physicalWrittenBytesSinceLastRebalance; - writerEstimatedWrittenBytes[writerId] = physicalWrittenBytesSinceLastRebalance; - } - - writerRowCountSinceLastRebalance = new long[numberOfWriters]; - writerMaxPartitions = new ArrayList<>(numberOfWriters); - for (int writerId = 0; writerId < numberOfWriters; writerId++) { - writerMaxPartitions.add(new IndexedPriorityQueue<>()); - } - - partitionRowCounts.forEach((serializedKey, rowCount) -> { - WriterPartitionId writerPartitionId = WriterPartitionId.deserialize(serializedKey); - writerRowCountSinceLastRebalance[writerPartitionId.writerId] += rowCount; - writerMaxPartitions - .get(writerPartitionId.writerId) - .addOrUpdate(new PartitionIdWithRowCount(writerPartitionId.partitionId, rowCount), rowCount); - }); - } - - private List rebalancePartition(WriterId from, WriterId to) - { - IndexedPriorityQueue maxPartitions = writerMaxPartitions.get(from.id); - ImmutableList.Builder affectedWriters = ImmutableList.builder(); - - for (PartitionIdWithRowCount partitionToRebalance : maxPartitions) { - // Find the partition with maximum written bytes since last rebalance - PartitionInfo partitionInfo = partitionInfos[partitionToRebalance.id]; - - // If a partition is already rebalanced or min skewed writer is already writing to that partition, then skip - // this partition and move on to the other partition inside max writer. Also, we don't rebalance same partition - // twice because we want to make sure that every writer wrote writerMinSize for a given partition - if (!isPartitionRebalanced(partitionToRebalance.id) && !partitionInfo.containsWriter(to.id)) { - // First remove the partition from the priority queue since there's no need go over it again. As in the next - // section we will check whether it can be scaled or not. - maxPartitions.remove(partitionToRebalance); - - long estimatedPartitionWrittenBytesSinceLastRebalance = estimatePartitionWrittenBytesSinceLastRebalance(from, partitionToRebalance.rowCount); - long estimatedPartitionWrittenBytes = - estimatedPartitionWrittenBytesSinceLastRebalance + partitionInfo.getPhysicalWrittenBytesAtLastRebalancePerWriter(); - - // Scale the partition when estimated physicalWrittenBytes is greater than writerMinSize. - if (partitionInfo.getWriterCount() <= numberOfWriters && estimatedPartitionWrittenBytes >= writerMinSize) { - partitionInfo.addWriter(to.id); - rebalancedPartitions.add(partitionToRebalance.id); - updateWriterEstimatedWrittenBytes(to, estimatedPartitionWrittenBytesSinceLastRebalance, partitionInfo); - for (int writer : partitionInfo.getWriterIds()) { - affectedWriters.add(new WriterId(writer)); - } - log.debug("Scaled partition (%s) to writer %s with writer count %s", partitionToRebalance.id, to.id, partitionInfo.getWriterCount()); - } - - break; - } - } - - return affectedWriters.build(); - } - - private void updateWriterEstimatedWrittenBytes(WriterId to, long estimatedPartitionWrittenBytesSinceLastRebalance, PartitionInfo partitionInfo) - { - // Since a partition is rebalanced from max to min skewed writer, decrease the priority of max - // writer as well as increase the priority of min writer. - int newWriterCount = partitionInfo.getWriterCount(); - int oldWriterCount = newWriterCount - 1; - for (int writer : partitionInfo.getWriterIds()) { - if (writer != to.id) { - writerEstimatedWrittenBytes[writer] -= estimatedPartitionWrittenBytesSinceLastRebalance / newWriterCount; - } - } - - writerEstimatedWrittenBytes[to.id] += estimatedPartitionWrittenBytesSinceLastRebalance * oldWriterCount / newWriterCount; - } - - private long getWriterEstimatedWrittenBytes(WriterId writer) - { - return writerEstimatedWrittenBytes[writer.id]; - } - - private boolean isPartitionRebalanced(int partitionId) - { - return rebalancedPartitions.contains(partitionId); - } - - private long estimatePartitionWrittenBytesSinceLastRebalance(WriterId writer, long partitionRowCount) - { - if (writerRowCountSinceLastRebalance[writer.id] == 0) { - return 0L; - } - return (writerPhysicalWrittenBytesSinceLastRebalance[writer.id] * partitionRowCount) / writerRowCountSinceLastRebalance[writer.id]; - } - } - - @ThreadSafe - private static class PartitionInfo - { - private final List writerAssignments; - // Partition estimated physical written bytes at the end of last rebalance cycle - private final AtomicLong physicalWrittenBytesAtLastRebalance = new AtomicLong(0); - - private PartitionInfo(int initialWriterId) - { - this.writerAssignments = new CopyOnWriteArrayList<>(ImmutableList.of(initialWriterId)); - } - - private boolean containsWriter(int writerId) - { - return writerAssignments.contains(writerId); - } - - private void addWriter(int writerId) - { - writerAssignments.add(writerId); - } - - private int getWriterId(int index) - { - return writerAssignments.get(floorMod(index, getWriterCount())); - } - - private List getWriterIds() - { - return ImmutableList.copyOf(writerAssignments); - } - - private int getWriterCount() - { - return writerAssignments.size(); - } - - private void resetPhysicalWrittenBytesAtLastRebalance() - { - physicalWrittenBytesAtLastRebalance.set(0); - } - - private void addToPhysicalWrittenBytesAtLastRebalance(long writtenBytes) - { - physicalWrittenBytesAtLastRebalance.addAndGet(writtenBytes); - } - - private long getPhysicalWrittenBytesAtLastRebalancePerWriter() - { - return physicalWrittenBytesAtLastRebalance.get() / writerAssignments.size(); - } - } - - public record WriterPartitionId(int writerId, int partitionId) - { - public static WriterPartitionId deserialize(long value) - { - int writerId = (int) (value >> 32); - int partitionId = (int) value; - - return new WriterPartitionId(writerId, partitionId); - } - - public static long serialize(WriterPartitionId writerPartitionId) - { - // Serialize to long to save memory where first 32 bit contains writerId whereas last 32 bit - // contains partitionId. - return ((long) writerPartitionId.writerId << 32 | writerPartitionId.partitionId & 0xFFFFFFFFL); - } - - public WriterPartitionId(int writerId, int partitionId) - { - this.writerId = writerId; - this.partitionId = partitionId; - } - } - - private record WriterId(int id) - { - private WriterId(int id) - { - this.id = id; - } - } - - private record PartitionIdWithRowCount(int id, long rowCount) - { - private PartitionIdWithRowCount(int id, long rowCount) - { - this.id = id; - this.rowCount = rowCount; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - PartitionIdWithRowCount that = (PartitionIdWithRowCount) o; - return id == that.id; - } - - @Override - public int hashCode() - { - return Objects.hashCode(id); - } - } -} diff --git a/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java b/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java index 2fd12d002818..f050341a1760 100644 --- a/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java +++ b/core/trino-main/src/main/java/io/trino/operator/output/SkewedPartitionRebalancer.java @@ -18,7 +18,6 @@ import com.google.errorprone.annotations.ThreadSafe; import com.google.errorprone.annotations.concurrent.GuardedBy; import io.airlift.log.Logger; -import io.airlift.units.DataSize; import io.trino.Session; import io.trino.execution.resourcegroups.IndexedPriorityQueue; import io.trino.operator.PartitionFunction; @@ -38,7 +37,6 @@ import java.util.stream.IntStream; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.sql.planner.PartitioningHandle.isScaledWriterHashDistribution; import static java.lang.Double.isNaN; import static java.lang.Math.ceil; @@ -79,7 +77,6 @@ public class SkewedPartitionRebalancer // If the percentage difference between the two different task buckets with maximum and minimum processed bytes // since last rebalance is above 0.7 (or 70%), then we consider them skewed. private static final double TASK_BUCKET_SKEWNESS_THRESHOLD = 0.7; - private static final long MIN_DATA_PROCESSED_REBALANCE_THRESHOLD = DataSize.of(50, MEGABYTE).toBytes(); private final int partitionCount; private final int taskCount; @@ -91,6 +88,9 @@ public class SkewedPartitionRebalancer private final AtomicLong dataProcessed; private final AtomicLong dataProcessedAtLastRebalance; + @GuardedBy("this") + private final long[] partitionDataSize; + @GuardedBy("this") private final long[] partitionDataSizeAtLastRebalance; @@ -148,11 +148,16 @@ public static PartitionFunction createPartitionFunction( IntStream.range(0, bucketCount).toArray()); } - public static SkewedPartitionRebalancer createSkewedPartitionRebalancer(int partitionCount, int taskCount, int taskPartitionedWriterCount, long minPartitionDataProcessedRebalanceThreshold) + public static SkewedPartitionRebalancer createSkewedPartitionRebalancer( + int partitionCount, + int taskCount, + int taskPartitionedWriterCount, + long minPartitionDataProcessedRebalanceThreshold, + long maxDataProcessedRebalanceThreshold) { // Keep the task bucket count to 50% of total local writers int taskBucketCount = (int) ceil(0.5 * taskPartitionedWriterCount); - return new SkewedPartitionRebalancer(partitionCount, taskCount, taskBucketCount, minPartitionDataProcessedRebalanceThreshold); + return new SkewedPartitionRebalancer(partitionCount, taskCount, taskBucketCount, minPartitionDataProcessedRebalanceThreshold, maxDataProcessedRebalanceThreshold); } public static int getTaskCount(PartitioningScheme partitioningScheme) @@ -165,22 +170,24 @@ public static int getTaskCount(PartitioningScheme partitioningScheme) return IntStream.of(bucketToPartition).max().getAsInt() + 1; } - private SkewedPartitionRebalancer( + public SkewedPartitionRebalancer( int partitionCount, int taskCount, int taskBucketCount, - long minPartitionDataProcessedRebalanceThreshold) + long minPartitionDataProcessedRebalanceThreshold, + long maxDataProcessedRebalanceThreshold) { this.partitionCount = partitionCount; this.taskCount = taskCount; this.taskBucketCount = taskBucketCount; this.minPartitionDataProcessedRebalanceThreshold = minPartitionDataProcessedRebalanceThreshold; - this.minDataProcessedRebalanceThreshold = max(minPartitionDataProcessedRebalanceThreshold, MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); + this.minDataProcessedRebalanceThreshold = max(minPartitionDataProcessedRebalanceThreshold, maxDataProcessedRebalanceThreshold); this.partitionRowCount = new AtomicLongArray(partitionCount); this.dataProcessed = new AtomicLong(); this.dataProcessedAtLastRebalance = new AtomicLong(); + this.partitionDataSize = new long[partitionCount]; this.partitionDataSizeAtLastRebalance = new long[partitionCount]; this.partitionDataSizeSinceLastRebalancePerTask = new long[partitionCount]; this.estimatedTaskBucketDataSizeSinceLastRebalance = new long[taskCount * taskBucketCount]; @@ -249,7 +256,7 @@ private synchronized void rebalancePartitions(long dataProcessed) return; } - long[] partitionDataSize = calculatePartitionDataSize(dataProcessed); + calculatePartitionDataSize(dataProcessed); // initialize partitionDataSizeSinceLastRebalancePerTask for (int partition = 0; partition < partitionCount; partition++) { @@ -287,23 +294,20 @@ private synchronized void rebalancePartitions(long dataProcessed) } } - rebalanceBasedOnTaskBucketSkewness(maxTaskBuckets, minTaskBuckets, taskBucketMaxPartitions, partitionDataSize); + rebalanceBasedOnTaskBucketSkewness(maxTaskBuckets, minTaskBuckets, taskBucketMaxPartitions); dataProcessedAtLastRebalance.set(dataProcessed); } - private long[] calculatePartitionDataSize(long dataProcessed) + private void calculatePartitionDataSize(long dataProcessed) { long totalPartitionRowCount = 0; for (int partition = 0; partition < partitionCount; partition++) { totalPartitionRowCount += partitionRowCount.get(partition); } - long[] partitionDataSize = new long[partitionCount]; for (int partition = 0; partition < partitionCount; partition++) { partitionDataSize[partition] = (partitionRowCount.get(partition) * dataProcessed) / totalPartitionRowCount; } - - return partitionDataSize; } private long calculateTaskBucketDataSizeSinceLastRebalance(IndexedPriorityQueue maxPartitions) @@ -318,9 +322,9 @@ private long calculateTaskBucketDataSizeSinceLastRebalance(IndexedPriorityQueue< private void rebalanceBasedOnTaskBucketSkewness( IndexedPriorityQueue maxTaskBuckets, IndexedPriorityQueue minTaskBuckets, - List> taskBucketMaxPartitions, - long[] partitionDataSize) + List> taskBucketMaxPartitions) { + List scaledPartitions = new ArrayList<>(); while (true) { TaskBucket maxTaskBucket = maxTaskBuckets.poll(); if (maxTaskBucket == null) { @@ -343,9 +347,19 @@ private void rebalanceBasedOnTaskBucketSkewness( break; } - if (partitionDataSizeSinceLastRebalancePerTask[maxPartition] >= minPartitionDataProcessedRebalanceThreshold) { + // Rebalance partition only once in a single cycle. Otherwise, rebalancing will happen quite + // aggressively in the early stage of write, while it is not required. Thus, it can have an impact on + // output file sizes and resource usage such that produced files can be small and memory usage + // might be higher. + if (scaledPartitions.contains(maxPartition)) { + continue; + } + + int totalAssignedTasks = partitionAssignments.get(maxPartition).size(); + if (partitionDataSize[maxPartition] >= (minPartitionDataProcessedRebalanceThreshold * totalAssignedTasks)) { for (TaskBucket minTaskBucket : minSkewedTaskBuckets) { if (rebalancePartition(maxPartition, minTaskBucket, maxTaskBuckets, minTaskBuckets, partitionDataSize[maxPartition])) { + scaledPartitions.add(maxPartition); break; } } @@ -411,7 +425,7 @@ private boolean rebalancePartition( minTasks.addOrUpdate(taskBucket, Long.MAX_VALUE - estimatedTaskBucketDataSizeSinceLastRebalance[taskBucket.id]); } - log.debug("Rebalanced partition %s to task %s with taskCount %s", partitionId, toTaskBucket.taskId, assignments.size()); + log.warn("Rebalanced partition %s to task %s with taskCount %s", partitionId, toTaskBucket.taskId, assignments.size()); return true; } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index 741e923b7693..7e3ab212668b 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -300,11 +300,12 @@ import static io.trino.SystemSessionProperties.getFilterAndProjectMinOutputPageRowCount; import static io.trino.SystemSessionProperties.getFilterAndProjectMinOutputPageSize; import static io.trino.SystemSessionProperties.getPagePartitioningBufferPoolSize; +import static io.trino.SystemSessionProperties.getSkewedPartitionMinDataProcessedRebalanceThreshold; import static io.trino.SystemSessionProperties.getTaskConcurrency; import static io.trino.SystemSessionProperties.getTaskPartitionedWriterCount; import static io.trino.SystemSessionProperties.getTaskScaleWritersMaxWriterCount; import static io.trino.SystemSessionProperties.getTaskWriterCount; -import static io.trino.SystemSessionProperties.getWriterMinSize; +import static io.trino.SystemSessionProperties.getWriterScalingMinDataProcessed; import static io.trino.SystemSessionProperties.isAdaptivePartialAggregationEnabled; import static io.trino.SystemSessionProperties.isEnableCoordinatorDynamicFiltersDistribution; import static io.trino.SystemSessionProperties.isEnableLargeDynamicFilters; @@ -579,7 +580,8 @@ public LocalExecutionPlan plan( partitionFunction.getPartitionCount(), taskCount, getTaskPartitionedWriterCount(taskContext.getSession()), - getWriterMinSize(taskContext.getSession()).toBytes())); + getWriterScalingMinDataProcessed(taskContext.getSession()).toBytes(), + getSkewedPartitionMinDataProcessedRebalanceThreshold(taskContext.getSession()).toBytes())); } else { partitionFunction = nodePartitioningManager.getPartitionFunction(taskContext.getSession(), partitioningScheme, partitionChannelTypes); @@ -3628,7 +3630,7 @@ private PhysicalOperation createLocalMerge(ExchangeNode node, LocalExecutionPlan Optional.empty(), maxLocalExchangeBufferSize, blockTypeOperators, - getWriterMinSize(session)); + getWriterScalingMinDataProcessed(session)); List expectedLayout = node.getInputs().get(0); Function pagePreprocessor = enforceLoadedLayoutProcessor(expectedLayout, source.getLayout()); @@ -3705,7 +3707,7 @@ else if (context.getDriverInstanceCount().isPresent()) { hashChannel, maxLocalExchangeBufferSize, blockTypeOperators, - getWriterMinSize(session)); + getWriterScalingMinDataProcessed(session)); for (int i = 0; i < node.getSources().size(); i++) { DriverFactoryParameters driverFactoryParameters = driverFactoryParametersList.get(i); PhysicalOperation source = driverFactoryParameters.getSource(); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java index e784579effef..fc04b1cb320a 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LogicalPlanner.java @@ -944,8 +944,7 @@ private RelationPlan createTableExecutePlan(Analysis analysis, TableExecute stat .map(ColumnMetadata::getName) .collect(toImmutableList()); - boolean supportsReportingWrittenBytes = metadata.supportsReportingWrittenBytes(session, tableHandle); - TableWriterNode.TableExecuteTarget tableExecuteTarget = new TableWriterNode.TableExecuteTarget(executeHandle, Optional.empty(), tableName.asSchemaTableName(), supportsReportingWrittenBytes); + TableWriterNode.TableExecuteTarget tableExecuteTarget = new TableWriterNode.TableExecuteTarget(executeHandle, Optional.empty(), tableName.asSchemaTableName()); Optional layout = metadata.getLayoutForTableExecute(session, executeHandle); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java index f250a8cefe63..454472b6adde 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java @@ -766,7 +766,7 @@ private PlanWithProperties getWriterPlanWithProperties(Optional maxWritersNodesCount = getRetryPolicy(session) != RetryPolicy.TASK ? Optional.of(Math.min(maxWriterTasks, getMaxWriterTaskCount(session))) : Optional.empty(); - if (scaleWriters && writerTarget.supportsReportingWrittenBytes(plannerContext.getMetadata(), session)) { + if (scaleWriters) { partitioningScheme = Optional.of(new PartitioningScheme(Partitioning.create(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION, ImmutableList.of()), newSource.getNode().getOutputSymbols(), Optional.empty(), false, Optional.empty(), maxWritersNodesCount)); } else if (redistributeWrites) { @@ -774,7 +774,6 @@ else if (redistributeWrites) { } } else if (scaleWriters - && writerTarget.supportsReportingWrittenBytes(plannerContext.getMetadata(), session) && writerTarget.supportsMultipleWritersPerPartition(plannerContext.getMetadata(), session) // do not insert an exchange if partitioning is compatible && !newSource.getProperties().isCompatibleTablePartitioningWith(partitioningScheme.get().getPartitioning(), false, plannerContext.getMetadata(), session)) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java index be475d120c7a..518eb5ce5d46 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java @@ -713,7 +713,6 @@ private PlanWithProperties visitTableWriter( WriterTarget writerTarget) { if (isTaskScaleWritersEnabled(session) - && writerTarget.supportsReportingWrittenBytes(plannerContext.getMetadata(), session) && writerTarget.supportsMultipleWritersPerPartition(plannerContext.getMetadata(), session) && partitioningScheme.isPresent()) { return visitScalePartitionedWriter(node, partitioningScheme.get(), source); @@ -721,12 +720,12 @@ private PlanWithProperties visitTableWriter( return partitioningScheme .map(scheme -> visitPartitionedWriter(node, scheme, source, parentPreferences)) - .orElseGet(() -> visitUnpartitionedWriter(node, source, writerTarget)); + .orElseGet(() -> visitUnpartitionedWriter(node, source)); } - private PlanWithProperties visitUnpartitionedWriter(PlanNode node, PlanNode source, WriterTarget writerTarget) + private PlanWithProperties visitUnpartitionedWriter(PlanNode node, PlanNode source) { - if (isTaskScaleWritersEnabled(session) && writerTarget.supportsReportingWrittenBytes(plannerContext.getMetadata(), session)) { + if (isTaskScaleWritersEnabled(session)) { PlanWithProperties newSource = source.accept(this, defaultParallelism(session)); PlanWithProperties exchange = deriveProperties( partitionedExchange( diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java index 7e0e964a60e5..ff2d94831fb6 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/BeginTableWrite.java @@ -208,8 +208,7 @@ public WriterTarget getWriterTarget(PlanNode node) return new TableExecuteTarget( target.getExecuteHandle(), findTableScanHandleForTableExecute(((TableExecuteNode) node).getSource()), - target.getSchemaTableName(), - target.isReportingWrittenBytesSupported()); + target.getSchemaTableName()); } if (node instanceof MergeWriterNode mergeWriterNode) { @@ -244,7 +243,6 @@ private WriterTarget createWriterTarget(WriterTarget target) return new CreateTarget( metadata.beginCreateTable(session, create.getCatalog(), create.getTableMetadata(), create.getLayout()), create.getTableMetadata().getTable(), - target.supportsReportingWrittenBytes(metadata, session), target.supportsMultipleWritersPerPartition(metadata, session), target.getMaxWriterTasks(metadata, session)); } @@ -252,7 +250,6 @@ private WriterTarget createWriterTarget(WriterTarget target) return new InsertTarget( metadata.beginInsert(session, insert.getHandle(), insert.getColumns()), metadata.getTableName(session, insert.getHandle()).getSchemaTableName(), - target.supportsReportingWrittenBytes(metadata, session), target.supportsMultipleWritersPerPartition(metadata, session), target.getMaxWriterTasks(metadata, session)); } @@ -273,7 +270,7 @@ private WriterTarget createWriterTarget(WriterTarget target) } if (target instanceof TableExecuteTarget tableExecute) { BeginTableExecuteResult result = metadata.beginTableExecute(session, tableExecute.getExecuteHandle(), tableExecute.getMandatorySourceHandle()); - return new TableExecuteTarget(result.getTableExecuteHandle(), Optional.of(result.getSourceHandle()), tableExecute.getSchemaTableName(), tableExecute.isReportingWrittenBytesSupported()); + return new TableExecuteTarget(result.getTableExecuteHandle(), Optional.of(result.getSourceHandle()), tableExecute.getSchemaTableName()); } throw new IllegalArgumentException("Unhandled target type: " + target.getClass().getSimpleName()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java index f881464f36cb..c8cf83aebbd7 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/TableWriterNode.java @@ -26,7 +26,6 @@ import io.trino.metadata.MergeHandle; import io.trino.metadata.Metadata; import io.trino.metadata.OutputTableHandle; -import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.TableExecuteHandle; import io.trino.metadata.TableHandle; import io.trino.metadata.TableLayout; @@ -193,8 +192,6 @@ public abstract static class WriterTarget @Override public abstract String toString(); - public abstract boolean supportsReportingWrittenBytes(Metadata metadata, Session session); - public abstract boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session); public abstract OptionalInt getMaxWriterTasks(Metadata metadata, Session session); @@ -220,16 +217,6 @@ public String getCatalog() return catalog; } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - QualifiedObjectName fullTableName = new QualifiedObjectName( - catalog, - tableMetadata.getTableSchema().getTable().getSchemaName(), - tableMetadata.getTableSchema().getTable().getTableName()); - return metadata.supportsReportingWrittenBytes(session, fullTableName, tableMetadata.getProperties()); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -264,7 +251,6 @@ public static class CreateTarget { private final OutputTableHandle handle; private final SchemaTableName schemaTableName; - private final boolean reportingWrittenBytesSupported; private final boolean multipleWritersPerPartitionSupported; private final OptionalInt maxWriterTasks; @@ -272,13 +258,11 @@ public static class CreateTarget public CreateTarget( @JsonProperty("handle") OutputTableHandle handle, @JsonProperty("schemaTableName") SchemaTableName schemaTableName, - @JsonProperty("reportingWrittenBytesSupported") boolean reportingWrittenBytesSupported, @JsonProperty("multipleWritersPerPartitionSupported") boolean multipleWritersPerPartitionSupported, @JsonProperty("maxWriterTasks") OptionalInt maxWriterTasks) { this.handle = requireNonNull(handle, "handle is null"); this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); - this.reportingWrittenBytesSupported = reportingWrittenBytesSupported; this.multipleWritersPerPartitionSupported = multipleWritersPerPartitionSupported; this.maxWriterTasks = requireNonNull(maxWriterTasks, "maxWriterTasks is null"); } @@ -295,12 +279,6 @@ public SchemaTableName getSchemaTableName() return schemaTableName; } - @JsonProperty - public boolean getReportingWrittenBytesSupported() - { - return reportingWrittenBytesSupported; - } - @JsonProperty public boolean isMultipleWritersPerPartitionSupported() { @@ -313,12 +291,6 @@ public String toString() return handle.toString(); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return reportingWrittenBytesSupported; - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -361,12 +333,6 @@ public String toString() return handle.toString(); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return metadata.supportsReportingWrittenBytes(session, handle); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -387,7 +353,6 @@ public static class InsertTarget { private final InsertTableHandle handle; private final SchemaTableName schemaTableName; - private final boolean reportingWrittenBytesSupported; private final boolean multipleWritersPerPartitionSupported; private final OptionalInt maxWriterTasks; @@ -395,13 +360,11 @@ public static class InsertTarget public InsertTarget( @JsonProperty("handle") InsertTableHandle handle, @JsonProperty("schemaTableName") SchemaTableName schemaTableName, - @JsonProperty("reportingWrittenBytesSupported") boolean reportingWrittenBytesSupported, @JsonProperty("multipleWritersPerPartitionSupported") boolean multipleWritersPerPartitionSupported, @JsonProperty("maxWriterTasks") OptionalInt maxWriterTasks) { this.handle = requireNonNull(handle, "handle is null"); this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); - this.reportingWrittenBytesSupported = reportingWrittenBytesSupported; this.multipleWritersPerPartitionSupported = multipleWritersPerPartitionSupported; this.maxWriterTasks = requireNonNull(maxWriterTasks, "maxWriterTasks is null"); } @@ -418,12 +381,6 @@ public SchemaTableName getSchemaTableName() return schemaTableName; } - @JsonProperty - public boolean getReportingWrittenBytesSupported() - { - return reportingWrittenBytesSupported; - } - @JsonProperty public boolean isMultipleWritersPerPartitionSupported() { @@ -436,12 +393,6 @@ public String toString() return handle.toString(); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return reportingWrittenBytesSupported; - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -485,12 +436,6 @@ public String toString() return table; } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return metadata.supportsReportingWrittenBytes(session, storageTableHandle); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -557,12 +502,6 @@ public String toString() return insertHandle.toString(); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return metadata.supportsReportingWrittenBytes(session, tableHandle); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -617,12 +556,6 @@ public String toString() return handle.map(Object::toString).orElse("[]"); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - throw new UnsupportedOperationException(); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -694,12 +627,6 @@ public String toString() return handle.map(Object::toString).orElse("[]"); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - throw new UnsupportedOperationException(); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -719,19 +646,16 @@ public static class TableExecuteTarget private final TableExecuteHandle executeHandle; private final Optional sourceHandle; private final SchemaTableName schemaTableName; - private final boolean reportingWrittenBytesSupported; @JsonCreator public TableExecuteTarget( @JsonProperty("executeHandle") TableExecuteHandle executeHandle, @JsonProperty("sourceHandle") Optional sourceHandle, - @JsonProperty("schemaTableName") SchemaTableName schemaTableName, - @JsonProperty("reportingWrittenBytesSupported") boolean reportingWrittenBytesSupported) + @JsonProperty("schemaTableName") SchemaTableName schemaTableName) { this.executeHandle = requireNonNull(executeHandle, "handle is null"); this.sourceHandle = requireNonNull(sourceHandle, "sourceHandle is null"); this.schemaTableName = requireNonNull(schemaTableName, "schemaTableName is null"); - this.reportingWrittenBytesSupported = reportingWrittenBytesSupported; } @JsonProperty @@ -757,24 +681,12 @@ public SchemaTableName getSchemaTableName() return schemaTableName; } - @JsonProperty - public boolean isReportingWrittenBytesSupported() - { - return reportingWrittenBytesSupported; - } - @Override public String toString() { return executeHandle.toString(); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return sourceHandle.map(tableHandle -> metadata.supportsReportingWrittenBytes(session, tableHandle)).orElse(reportingWrittenBytesSupported); - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { @@ -841,12 +753,6 @@ public String toString() return handle.toString(); } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return false; - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/sanity/ValidateScaledWritersUsage.java b/core/trino-main/src/main/java/io/trino/sql/planner/sanity/ValidateScaledWritersUsage.java index 373941855e9a..3aaf0879ac93 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/sanity/ValidateScaledWritersUsage.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/sanity/ValidateScaledWritersUsage.java @@ -82,12 +82,9 @@ public List visitTableWriter(TableWriterNode node, Void cont TableWriterNode.WriterTarget target = node.getTarget(); scaleWriterPartitioningHandle.forEach(partitioningHandle -> { - checkState(target.supportsReportingWrittenBytes(plannerContext.getMetadata(), session), - "The scaled writer partitioning scheme is set but writer target %s doesn't support reporting physical written bytes", target); - if (isScaledWriterHashDistribution(partitioningHandle)) { checkState(target.supportsMultipleWritersPerPartition(plannerContext.getMetadata(), session), - "The scaled writer partitioning scheme is set for the partitioned write but writer target %s doesn't support multiple writers per partition", target); + "The hash scaled writer partitioning scheme is set for the partitioned write but writer target %s doesn't support multiple writers per partition", target); } }); return children; diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java index fc4227592062..d27ef50b619e 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java @@ -165,7 +165,6 @@ public class MockConnector private final Set tableProcedures; private final Set tableFunctions; private final Optional functionProvider; - private final boolean supportsReportingWrittenBytes; private final boolean allowMissingColumnsOnInsert; private final Supplier>> analyzeProperties; private final Supplier>> schemaProperties; @@ -217,7 +216,6 @@ public class MockConnector Supplier>> schemaProperties, Supplier>> tableProperties, Supplier>> columnProperties, - boolean supportsReportingWrittenBytes, Function tableFunctionSplitsSources, OptionalInt maxWriterTasks, BiFunction> getLayoutForTableExecute) @@ -257,7 +255,6 @@ public class MockConnector this.tableProcedures = requireNonNull(tableProcedures, "tableProcedures is null"); this.tableFunctions = requireNonNull(tableFunctions, "tableFunctions is null"); this.functionProvider = requireNonNull(functionProvider, "functionProvider is null"); - this.supportsReportingWrittenBytes = supportsReportingWrittenBytes; this.allowMissingColumnsOnInsert = allowMissingColumnsOnInsert; this.analyzeProperties = requireNonNull(analyzeProperties, "analyzeProperties is null"); this.schemaProperties = requireNonNull(schemaProperties, "schemaProperties is null"); @@ -834,18 +831,6 @@ public void revokeTablePrivileges(ConnectorSession session, SchemaTableName tabl getMockAccessControl().revokeTablePrivileges(tableName, privileges, revokee, grantOption); } - @Override - public boolean supportsReportingWrittenBytes(ConnectorSession session, SchemaTableName schemaTableName, Map tableProperties) - { - return supportsReportingWrittenBytes; - } - - @Override - public boolean supportsReportingWrittenBytes(ConnectorSession session, ConnectorTableHandle tableHandle) - { - return supportsReportingWrittenBytes; - } - @Override public OptionalInt getMaxWriterTasks(ConnectorSession session) { diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java b/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java index 4a9b07a0c3f4..84724f1d9be5 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnectorFactory.java @@ -129,7 +129,6 @@ public class MockConnectorFactory // access control private final ListRoleGrants roleGrants; private final Optional accessControl; - private final boolean supportsReportingWrittenBytes; private final OptionalInt maxWriterTasks; private final BiFunction> getLayoutForTableExecute; @@ -173,7 +172,6 @@ private MockConnectorFactory( Supplier>> columnProperties, Optional partitioningProvider, ListRoleGrants roleGrants, - boolean supportsReportingWrittenBytes, Optional accessControl, boolean allowMissingColumnsOnInsert, Function tableFunctionSplitsSources, @@ -221,7 +219,6 @@ private MockConnectorFactory( this.tableFunctions = requireNonNull(tableFunctions, "tableFunctions is null"); this.functionProvider = requireNonNull(functionProvider, "functionProvider is null"); this.allowMissingColumnsOnInsert = allowMissingColumnsOnInsert; - this.supportsReportingWrittenBytes = supportsReportingWrittenBytes; this.tableFunctionSplitsSources = requireNonNull(tableFunctionSplitsSources, "tableFunctionSplitsSources is null"); this.maxWriterTasks = maxWriterTasks; this.getLayoutForTableExecute = requireNonNull(getLayoutForTableExecute, "getLayoutForTableExecute is null"); @@ -277,7 +274,6 @@ public Connector create(String catalogName, Map config, Connecto schemaProperties, tableProperties, columnProperties, - supportsReportingWrittenBytes, tableFunctionSplitsSources, maxWriterTasks, getLayoutForTableExecute); @@ -416,7 +412,6 @@ public static final class Builder private Grants tableGrants = new AllowAllGrants<>(); private Function rowFilter = tableName -> null; private BiFunction columnMask = (tableName, columnName) -> null; - private boolean supportsReportingWrittenBytes; private boolean allowMissingColumnsOnInsert; private OptionalInt maxWriterTasks = OptionalInt.empty(); private BiFunction> getLayoutForTableExecute = (session, handle) -> Optional.empty(); @@ -717,12 +712,6 @@ public Builder withColumnMask(BiFunction properties = ImmutableMap.builder() .put("task.initial-splits-per-node", "1") .put("task.split-concurrency-adjustment-interval", "1s") @@ -110,7 +112,7 @@ public void testExplicitPropertyMappings() .put("task.scale-writers.enabled", "false") .put("task.scale-writers.max-writer-count", Integer.toString(maxWriterCount)) .put("task.writer-count", "4") - .put("task.partitioned-writer-count", Integer.toString(processorCount)) + .put("task.partitioned-writer-count", Integer.toString(partitionedWriterCount)) .put("task.concurrency", Integer.toString(processorCount)) .put("task.http-response-threads", "4") .put("task.http-timeout-threads", "10") @@ -150,7 +152,7 @@ public void testExplicitPropertyMappings() .setScaleWritersEnabled(false) .setScaleWritersMaxWriterCount(maxWriterCount) .setWriterCount(4) - .setPartitionedWriterCount(processorCount) + .setPartitionedWriterCount(partitionedWriterCount) .setTaskConcurrency(processorCount) .setHttpResponseThreads(4) .setHttpTimeoutThreads(10) diff --git a/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java b/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java index a497c84e1786..5f8125c66403 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java @@ -183,6 +183,7 @@ public TaskStatus getTaskStatus() OutputBufferStatus.initial(), DataSize.of(0, BYTE), DataSize.of(0, BYTE), + DataSize.of(0, BYTE), Optional.empty(), DataSize.of(0, BYTE), DataSize.of(0, BYTE), diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java index 1fa3e489b7ae..52d366b18c99 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestScaledWriterScheduler.java @@ -113,7 +113,7 @@ public void testGetNewTaskCountWithUnderutilizedSkewedTaskAndOverutilizedNonSkew } @Test - public void testGetNewTaskCountWhenWrittenBytesIsGreaterThanMinWrittenBytesForScaleUp() + public void testGetNewTaskCountWhenWriterDataProcessedIsGreaterThanMinForScaleUp() { TaskStatus taskStatus1 = buildTaskStatus(1, DataSize.of(32, DataSize.Unit.MEGABYTE)); TaskStatus taskStatus2 = buildTaskStatus(1, DataSize.of(32, DataSize.Unit.MEGABYTE)); @@ -125,7 +125,7 @@ public void testGetNewTaskCountWhenWrittenBytesIsGreaterThanMinWrittenBytesForSc } @Test - public void testGetNewTaskCountWhenWrittenBytesIsLessThanMinWrittenBytesForScaleUp() + public void testGetNewTaskCountWhenWriterDataProcessedIsLessThanMinForScaleUp() { TaskStatus taskStatus1 = buildTaskStatus(1, DataSize.of(32, DataSize.Unit.MEGABYTE)); TaskStatus taskStatus2 = buildTaskStatus(1, DataSize.of(32, DataSize.Unit.MEGABYTE)); @@ -133,7 +133,7 @@ public void testGetNewTaskCountWhenWrittenBytesIsLessThanMinWrittenBytesForScale ScaledWriterScheduler scaledWriterScheduler = buildScaleWriterSchedulerWithInitialTasks(taskStatus1, taskStatus2, taskStatus3); // Scale up will not happen because for one of the task there are two local writers which makes the - // minWrittenBytes for scaling up to (2 * writerMinSizeBytes) that is greater than physicalWrittenBytes. + // minWrittenBytes for scaling up to (2 * writerScalingMinDataProcessed) that is greater than writerInputDataSize. assertEquals(scaledWriterScheduler.schedule().getNewTasks().size(), 0); } @@ -208,12 +208,12 @@ private static TaskStatus buildTaskStatus(boolean isOutputBufferOverUtilized, lo return buildTaskStatus(isOutputBufferOverUtilized, outputDataSize, Optional.of(1), DataSize.of(32, DataSize.Unit.MEGABYTE)); } - private static TaskStatus buildTaskStatus(int maxWriterCount, DataSize physicalWrittenDataSize) + private static TaskStatus buildTaskStatus(int maxWriterCount, DataSize writerInputDataSize) { - return buildTaskStatus(true, 12345L, Optional.of(maxWriterCount), physicalWrittenDataSize); + return buildTaskStatus(true, 12345L, Optional.of(maxWriterCount), writerInputDataSize); } - private static TaskStatus buildTaskStatus(boolean isOutputBufferOverUtilized, long outputDataSize, Optional maxWriterCount, DataSize physicalWrittenDataSize) + private static TaskStatus buildTaskStatus(boolean isOutputBufferOverUtilized, long outputDataSize, Optional maxWriterCount, DataSize writerInputDataSize) { return new TaskStatus( TaskId.valueOf("taskId"), @@ -228,7 +228,8 @@ private static TaskStatus buildTaskStatus(boolean isOutputBufferOverUtilized, lo 0, new OutputBufferStatus(OptionalLong.empty(), isOutputBufferOverUtilized, false), DataSize.ofBytes(outputDataSize), - physicalWrittenDataSize, + writerInputDataSize, + DataSize.of(1, DataSize.Unit.MEGABYTE), maxWriterCount, DataSize.of(1, DataSize.Unit.MEGABYTE), DataSize.of(1, DataSize.Unit.MEGABYTE), diff --git a/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java b/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java index 0fc7397eb04c..f63be46d8baa 100644 --- a/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java +++ b/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java @@ -273,6 +273,7 @@ private static TaskInfo buildTaskInfo(TaskId taskId, TaskState state, Duration s OutputBufferStatus.initial(), DataSize.of(0, DataSize.Unit.MEGABYTE), DataSize.of(1, DataSize.Unit.MEGABYTE), + DataSize.of(1, DataSize.Unit.MEGABYTE), Optional.of(1), DataSize.of(1, DataSize.Unit.MEGABYTE), DataSize.of(1, DataSize.Unit.MEGABYTE), @@ -336,6 +337,7 @@ private static TaskInfo buildTaskInfo(TaskId taskId, TaskState state, Duration s 0, new Duration(0, MILLISECONDS), DataSize.ofBytes(0), + DataSize.ofBytes(0), Optional.empty(), 0, new Duration(0, MILLISECONDS), diff --git a/core/trino-main/src/test/java/io/trino/memory/TestTotalReservationOnBlockedNodesTaskLowMemoryKiller.java b/core/trino-main/src/test/java/io/trino/memory/TestTotalReservationOnBlockedNodesTaskLowMemoryKiller.java index acb18df48018..e272cead88ca 100644 --- a/core/trino-main/src/test/java/io/trino/memory/TestTotalReservationOnBlockedNodesTaskLowMemoryKiller.java +++ b/core/trino-main/src/test/java/io/trino/memory/TestTotalReservationOnBlockedNodesTaskLowMemoryKiller.java @@ -248,6 +248,7 @@ private static TaskInfo buildTaskInfo(TaskId taskId, boolean speculative) OutputBufferStatus.initial(), DataSize.of(0, DataSize.Unit.MEGABYTE), DataSize.of(1, DataSize.Unit.MEGABYTE), + DataSize.of(1, DataSize.Unit.MEGABYTE), Optional.of(1), DataSize.of(1, DataSize.Unit.MEGABYTE), DataSize.of(1, DataSize.Unit.MEGABYTE), @@ -311,6 +312,7 @@ private static TaskInfo buildTaskInfo(TaskId taskId, boolean speculative) 0, new Duration(0, MILLISECONDS), DataSize.ofBytes(0), + DataSize.ofBytes(0), Optional.empty(), 0, new Duration(0, MILLISECONDS), diff --git a/core/trino-main/src/test/java/io/trino/operator/TestTableWriterOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestTableWriterOperator.java index c85401ea25a8..8c581dca4e83 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestTableWriterOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestTableWriterOperator.java @@ -298,7 +298,6 @@ private Operator createTableWriterOperator( new ConnectorOutputTableHandle() {}), schemaTableName, false, - false, OptionalInt.empty()), ImmutableList.of(0), session, diff --git a/core/trino-main/src/test/java/io/trino/operator/TestTaskStats.java b/core/trino-main/src/test/java/io/trino/operator/TestTaskStats.java index 1399e0ba7161..453a664b6ce0 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestTaskStats.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestTaskStats.java @@ -80,6 +80,7 @@ public class TestTaskStats new Duration(272, NANOSECONDS), + DataSize.ofBytes(25), DataSize.ofBytes(25), Optional.of(2), diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index c35bde218531..4fc80d29467f 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -56,7 +56,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.airlift.units.DataSize.Unit.KILOBYTE; import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static io.trino.SystemSessionProperties.SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD; import static io.trino.spi.connector.ConnectorBucketNodeMap.createBucketNodeMap; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -80,10 +82,11 @@ public class TestLocalExchange { private static final List TYPES = ImmutableList.of(BIGINT); private static final DataSize RETAINED_PAGE_SIZE = DataSize.ofBytes(createPage(42).getRetainedSizeInBytes()); + private static final DataSize PAGE_SIZE = DataSize.ofBytes(createPage(42).getSizeInBytes()); private static final DataSize LOCAL_EXCHANGE_MAX_BUFFERED_BYTES = DataSize.of(32, MEGABYTE); private static final BlockTypeOperators TYPE_OPERATOR_FACTORY = new BlockTypeOperators(new TypeOperators()); private static final Session SESSION = testSessionBuilder().build(); - private static final DataSize WRITER_MIN_SIZE = DataSize.of(32, MEGABYTE); + private static final DataSize WRITER_SCALING_MIN_DATA_PROCESSED = DataSize.of(32, MEGABYTE); private final ConcurrentMap partitionManagers = new ConcurrentHashMap<>(); private NodePartitioningManager nodePartitioningManager; @@ -118,7 +121,7 @@ public void testGatherSingleWriter() Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(99)), TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 1); @@ -127,7 +130,7 @@ public void testGatherSingleWriter() LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); sinkFactory.noMoreSinkFactories(); - LocalExchangeSource source = getNextSource(exchange); + LocalExchangeSource source = exchange.getNextSource(); assertSource(source, 0); LocalExchangeSink sink = sinkFactory.createSink(); @@ -191,7 +194,7 @@ public void testRandom() Optional.empty(), LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -203,10 +206,10 @@ public void testRandom() assertSinkCanWrite(sink); sinkFactory.close(); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); for (int i = 0; i < 100; i++) { @@ -240,7 +243,7 @@ public void testScaleWriter() Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(4)), TYPE_OPERATOR_FACTORY, - DataSize.ofBytes(retainedSizeOfPages(2))); + DataSize.ofBytes(sizeOfPages(2))); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 3); @@ -252,16 +255,13 @@ public void testScaleWriter() assertSinkCanWrite(sink); sinkFactory.close(); - AtomicLong physicalWrittenBytesA = new AtomicLong(0); - LocalExchangeSource sourceA = exchange.getNextSource(physicalWrittenBytesA::get); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - AtomicLong physicalWrittenBytesB = new AtomicLong(0); - LocalExchangeSource sourceB = exchange.getNextSource(physicalWrittenBytesB::get); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - AtomicLong physicalWrittenBytesC = new AtomicLong(0); - LocalExchangeSource sourceC = exchange.getNextSource(physicalWrittenBytesC::get); + LocalExchangeSource sourceC = exchange.getNextSource(); assertSource(sourceC, 0); sink.addPage(createPage(0)); @@ -270,8 +270,7 @@ public void testScaleWriter() assertEquals(sourceB.getBufferInfo().getBufferedPages(), 0); assertEquals(sourceC.getBufferInfo().getBufferedPages(), 0); - // writer min file and buffered data size limits are exceeded, so we should see pages in sourceB - physicalWrittenBytesA.set(retainedSizeOfPages(2)); + // writer min output size and buffered data size limits are exceeded, so we should see pages in sourceB sink.addPage(createPage(0)); assertEquals(sourceA.getBufferInfo().getBufferedPages(), 2); assertEquals(sourceB.getBufferInfo().getBufferedPages(), 1); @@ -280,32 +279,12 @@ public void testScaleWriter() assertRemovePage(sourceA, createPage(0)); assertRemovePage(sourceA, createPage(0)); - // no limit is breached, so we should see round-robin distribution across sourceA and sourceB - physicalWrittenBytesB.set(retainedSizeOfPages(1)); + // writer min output size and buffered data size limits are exceeded again, sink.addPage(createPage(0)); sink.addPage(createPage(0)); sink.addPage(createPage(0)); - assertEquals(sourceA.getBufferInfo().getBufferedPages(), 2); + assertEquals(sourceA.getBufferInfo().getBufferedPages(), 1); assertEquals(sourceB.getBufferInfo().getBufferedPages(), 2); - assertEquals(sourceC.getBufferInfo().getBufferedPages(), 0); - - // writer min file and buffered data size limits are exceeded again, but according to - // round-robin sourceB should receive a page - physicalWrittenBytesA.set(retainedSizeOfPages(4)); - physicalWrittenBytesB.set(retainedSizeOfPages(2)); - sink.addPage(createPage(0)); - assertEquals(sourceA.getBufferInfo().getBufferedPages(), 2); - assertEquals(sourceB.getBufferInfo().getBufferedPages(), 3); - assertEquals(sourceC.getBufferInfo().getBufferedPages(), 0); - - assertSinkWriteBlocked(sink); - assertRemoveAllPages(sourceA, createPage(0)); - - // sourceC should receive a page - physicalWrittenBytesB.set(retainedSizeOfPages(3)); - sink.addPage(createPage(0)); - assertEquals(sourceA.getBufferInfo().getBufferedPages(), 0); - assertEquals(sourceB.getBufferInfo().getBufferedPages(), 3); assertEquals(sourceC.getBufferInfo().getBufferedPages(), 1); }); } @@ -323,7 +302,7 @@ public void testNoWriterScalingWhenOnlyBufferSizeLimitIsExceeded() Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(4)), TYPE_OPERATOR_FACTORY, - DataSize.ofBytes(retainedSizeOfPages(2))); + DataSize.ofBytes(sizeOfPages(10))); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 3); @@ -335,13 +314,13 @@ public void testNoWriterScalingWhenOnlyBufferSizeLimitIsExceeded() assertSinkCanWrite(sink); sinkFactory.close(); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - LocalExchangeSource sourceC = getNextSource(exchange); + LocalExchangeSource sourceC = exchange.getNextSource(); assertSource(sourceC, 0); range(0, 6).forEach(i -> sink.addPage(createPage(0))); @@ -352,7 +331,7 @@ public void testNoWriterScalingWhenOnlyBufferSizeLimitIsExceeded() } @Test - public void testNoWriterScalingWhenOnlyWriterMinSizeLimitIsExceeded() + public void testNoWriterScalingWhenOnlyWriterScalingMinDataProcessedLimitIsExceeded() { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, @@ -364,7 +343,7 @@ public void testNoWriterScalingWhenOnlyWriterMinSizeLimitIsExceeded() Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(20)), TYPE_OPERATOR_FACTORY, - DataSize.ofBytes(retainedSizeOfPages(2))); + DataSize.ofBytes(sizeOfPages(2))); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 3); @@ -377,13 +356,13 @@ public void testNoWriterScalingWhenOnlyWriterMinSizeLimitIsExceeded() sinkFactory.close(); AtomicLong physicalWrittenBytesA = new AtomicLong(0); - LocalExchangeSource sourceA = exchange.getNextSource(physicalWrittenBytesA::get); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - LocalExchangeSource sourceC = getNextSource(exchange); + LocalExchangeSource sourceC = exchange.getNextSource(); assertSource(sourceC, 0); range(0, 8).forEach(i -> sink.addPage(createPage(0))); @@ -400,7 +379,9 @@ public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, - SESSION, + testSessionBuilder() + .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") + .build(), 4, partitioningHandle, ImmutableList.of(0), @@ -408,7 +389,7 @@ public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(2)), TYPE_OPERATOR_FACTORY, - DataSize.of(50, MEGABYTE)); + DataSize.of(10, KILOBYTE)); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 4); @@ -420,20 +401,16 @@ public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) assertSinkCanWrite(sink); sinkFactory.close(); - AtomicLong physicalWrittenBytesA = new AtomicLong(0); - LocalExchangeSource sourceA = exchange.getNextSource(physicalWrittenBytesA::get); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - AtomicLong physicalWrittenBytesB = new AtomicLong(0); - LocalExchangeSource sourceB = exchange.getNextSource(physicalWrittenBytesB::get); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - AtomicLong physicalWrittenBytesC = new AtomicLong(0); - LocalExchangeSource sourceC = exchange.getNextSource(physicalWrittenBytesC::get); + LocalExchangeSource sourceC = exchange.getNextSource(); assertSource(sourceC, 0); - AtomicLong physicalWrittenBytesD = new AtomicLong(0); - LocalExchangeSource sourceD = exchange.getNextSource(physicalWrittenBytesD::get); + LocalExchangeSource sourceD = exchange.getNextSource(); assertSource(sourceD, 0); sink.addPage(createSingleValuePage(0, 1000)); @@ -447,9 +424,6 @@ public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) assertSource(sourceC, 0); assertSource(sourceD, 2); - physicalWrittenBytesA.set(DataSize.of(2, MEGABYTE).toBytes()); - physicalWrittenBytesD.set(DataSize.of(150, MEGABYTE).toBytes()); - sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); @@ -461,35 +435,31 @@ public void testScalingForSkewedWriters(PartitioningHandle partitioningHandle) assertSource(sourceC, 0); assertSource(sourceD, 4); - physicalWrittenBytesB.set(DataSize.of(100, MEGABYTE).toBytes()); - physicalWrittenBytesD.set(DataSize.of(250, MEGABYTE).toBytes()); - sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); // Still there is a skewness across writers since writerA and writerC aren't writing any data. - // Hence, scaling will happen for partition in writerD and writerB to writerA. - assertSource(sourceA, 3); - assertSource(sourceB, 3); - assertSource(sourceC, 0); - assertSource(sourceD, 6); - - physicalWrittenBytesA.set(DataSize.of(52, MEGABYTE).toBytes()); - physicalWrittenBytesB.set(DataSize.of(150, MEGABYTE).toBytes()); - physicalWrittenBytesD.set(DataSize.of(300, MEGABYTE).toBytes()); + // Hence, scaling will happen for partition in writerD and writerB to writerC. + assertSource(sourceA, 2); + assertSource(sourceB, 4); + assertSource(sourceC, 1); + assertSource(sourceD, 5); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(0, 1000)); - // Now only writerC is unused. So, scaling will happen to all the available writers. + // Still there is a skewness across writers since writerA isn't writing any data. + // Hence, scaling will happen for partition in writerD and writerB to writerA. assertSource(sourceA, 4); - assertSource(sourceB, 4); - assertSource(sourceC, 1); - assertSource(sourceD, 7); + assertSource(sourceB, 5); + assertSource(sourceC, 3); + assertSource(sourceD, 6); }); } @@ -498,7 +468,9 @@ public void testNoScalingWhenDataWrittenIsLessThanMinFileSize(PartitioningHandle { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, - SESSION, + testSessionBuilder() + .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") + .build(), 4, partitioningHandle, ImmutableList.of(0), @@ -518,20 +490,16 @@ public void testNoScalingWhenDataWrittenIsLessThanMinFileSize(PartitioningHandle assertSinkCanWrite(sink); sinkFactory.close(); - AtomicLong physicalWrittenBytesA = new AtomicLong(0); - LocalExchangeSource sourceA = exchange.getNextSource(physicalWrittenBytesA::get); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - AtomicLong physicalWrittenBytesB = new AtomicLong(0); - LocalExchangeSource sourceB = exchange.getNextSource(physicalWrittenBytesB::get); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - AtomicLong physicalWrittenBytesC = new AtomicLong(0); - LocalExchangeSource sourceC = exchange.getNextSource(physicalWrittenBytesC::get); + LocalExchangeSource sourceC = exchange.getNextSource(); assertSource(sourceC, 0); - AtomicLong physicalWrittenBytesD = new AtomicLong(0); - LocalExchangeSource sourceD = exchange.getNextSource(physicalWrittenBytesD::get); + LocalExchangeSource sourceD = exchange.getNextSource(); assertSource(sourceD, 0); sink.addPage(createSingleValuePage(0, 1000)); @@ -545,9 +513,6 @@ public void testNoScalingWhenDataWrittenIsLessThanMinFileSize(PartitioningHandle assertSource(sourceC, 0); assertSource(sourceD, 2); - physicalWrittenBytesA.set(DataSize.of(2, MEGABYTE).toBytes()); - physicalWrittenBytesD.set(DataSize.of(40, MEGABYTE).toBytes()); - sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); @@ -566,7 +531,9 @@ public void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, - SESSION, + testSessionBuilder() + .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") + .build(), 4, partitioningHandle, ImmutableList.of(0), @@ -574,7 +541,7 @@ public void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle Optional.empty(), DataSize.of(50, MEGABYTE), TYPE_OPERATOR_FACTORY, - DataSize.of(10, MEGABYTE)); + DataSize.of(10, KILOBYTE)); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 4); @@ -586,20 +553,16 @@ public void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle assertSinkCanWrite(sink); sinkFactory.close(); - AtomicLong physicalWrittenBytesA = new AtomicLong(0); - LocalExchangeSource sourceA = exchange.getNextSource(physicalWrittenBytesA::get); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - AtomicLong physicalWrittenBytesB = new AtomicLong(0); - LocalExchangeSource sourceB = exchange.getNextSource(physicalWrittenBytesB::get); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - AtomicLong physicalWrittenBytesC = new AtomicLong(0); - LocalExchangeSource sourceC = exchange.getNextSource(physicalWrittenBytesC::get); + LocalExchangeSource sourceC = exchange.getNextSource(); assertSource(sourceC, 0); - AtomicLong physicalWrittenBytesD = new AtomicLong(0); - LocalExchangeSource sourceD = exchange.getNextSource(physicalWrittenBytesD::get); + LocalExchangeSource sourceD = exchange.getNextSource(); assertSource(sourceD, 0); sink.addPage(createSingleValuePage(0, 1000)); @@ -613,9 +576,6 @@ public void testNoScalingWhenBufferUtilizationIsLessThanLimit(PartitioningHandle assertSource(sourceC, 0); assertSource(sourceD, 2); - physicalWrittenBytesA.set(DataSize.of(2, MEGABYTE).toBytes()); - physicalWrittenBytesD.set(DataSize.of(50, MEGABYTE).toBytes()); - sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(0, 1000)); @@ -634,7 +594,9 @@ public void testNoScalingWhenNoWriterSkewness() { LocalExchange localExchange = new LocalExchange( nodePartitioningManager, - SESSION, + testSessionBuilder() + .setSystemProperty(SKEWED_PARTITION_MIN_DATA_PROCESSED_REBALANCE_THRESHOLD, "20kB") + .build(), 2, SCALED_WRITER_HASH_DISTRIBUTION, ImmutableList.of(0), @@ -642,7 +604,7 @@ public void testNoScalingWhenNoWriterSkewness() Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(2)), TYPE_OPERATOR_FACTORY, - DataSize.of(50, MEGABYTE)); + DataSize.of(50, KILOBYTE)); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -654,24 +616,19 @@ public void testNoScalingWhenNoWriterSkewness() assertSinkCanWrite(sink); sinkFactory.close(); - AtomicLong physicalWrittenBytesA = new AtomicLong(0); - LocalExchangeSource sourceA = exchange.getNextSource(physicalWrittenBytesA::get); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - AtomicLong physicalWrittenBytesB = new AtomicLong(0); - LocalExchangeSource sourceB = exchange.getNextSource(physicalWrittenBytesB::get); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - sink.addPage(createSingleValuePage(0, 100)); - sink.addPage(createSingleValuePage(1, 100)); + sink.addPage(createSingleValuePage(0, 1000)); + sink.addPage(createSingleValuePage(1, 1000)); // Two partitions are assigned to two different writers assertSource(sourceA, 1); assertSource(sourceB, 1); - physicalWrittenBytesA.set(DataSize.of(50, MEGABYTE).toBytes()); - physicalWrittenBytesB.set(DataSize.of(50, MEGABYTE).toBytes()); - sink.addPage(createSingleValuePage(0, 1000)); sink.addPage(createSingleValuePage(1, 1000)); @@ -694,7 +651,7 @@ public void testPassthrough() Optional.empty(), DataSize.ofBytes(retainedSizeOfPages(1)), TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -708,10 +665,10 @@ public void testPassthrough() assertSinkCanWrite(sinkB); sinkFactory.close(); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); sinkA.addPage(createPage(0)); @@ -761,7 +718,7 @@ public void testPartition() Optional.empty(), LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -773,10 +730,10 @@ public void testPartition() assertSinkCanWrite(sink); sinkFactory.close(); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); sink.addPage(createPage(0)); @@ -857,7 +814,7 @@ public BucketFunction getBucketFunction(ConnectorTransactionHandle transactionHa Optional.empty(), LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -869,10 +826,10 @@ public BucketFunction getBucketFunction(ConnectorTransactionHandle transactionHa assertSinkCanWrite(sink); sinkFactory.close(); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); Page pageA = SequencePageBuilder.createSequencePage(types, 1, 100, 42); @@ -908,7 +865,7 @@ public void writeUnblockWhenAllReadersFinish() Optional.empty(), LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -922,10 +879,10 @@ public void writeUnblockWhenAllReadersFinish() assertSinkCanWrite(sinkB); sinkFactory.close(); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); sourceA.finish(); @@ -955,7 +912,7 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed() Optional.empty(), DataSize.ofBytes(2), TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); + WRITER_SCALING_MIN_DATA_PROCESSED); run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); @@ -975,10 +932,10 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed() sinkFactory.close(); - LocalExchangeSource sourceA = getNextSource(exchange); + LocalExchangeSource sourceA = exchange.getNextSource(); assertSource(sourceA, 0); - LocalExchangeSource sourceB = getNextSource(exchange); + LocalExchangeSource sourceB = exchange.getNextSource(); assertSource(sourceB, 0); sinkA.addPage(createPage(0)); @@ -1059,11 +1016,6 @@ private void run(LocalExchange localExchange, Consumer test) test.accept(localExchange); } - private LocalExchangeSource getNextSource(LocalExchange exchange) - { - return exchange.getNextSource(() -> DataSize.of(0, MEGABYTE).toBytes()); - } - private static void assertSource(LocalExchangeSource source, int pageCount) { LocalExchangeBufferInfo bufferInfo = source.getBufferInfo(); @@ -1096,11 +1048,6 @@ private static void assertSourceFinished(LocalExchangeSource source) assertTrue(source.isFinished()); } - private static void assertRemoveAllPages(LocalExchangeSource source, Page expectedPage) - { - range(0, source.getBufferInfo().getBufferedPages()).forEach(i -> assertRemovePage(source, expectedPage)); - } - private static void assertRemovePage(LocalExchangeSource source, Page expectedPage) { assertRemovePage(TYPES, source, expectedPage); @@ -1174,6 +1121,11 @@ private static Page createSingleValuePage(int value, int length) return new Page(block); } + private static long sizeOfPages(int count) + { + return PAGE_SIZE.toBytes() * count; + } + public static long retainedSizeOfPages(int count) { return RETAINED_PAGE_SIZE.toBytes() * count; diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestUniformPartitionRebalancer.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestUniformPartitionRebalancer.java deleted file mode 100644 index 819e8473c704..000000000000 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestUniformPartitionRebalancer.java +++ /dev/null @@ -1,508 +0,0 @@ -/* - * 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 io.trino.operator.exchange; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.airlift.units.DataSize; -import it.unimi.dsi.fastutil.longs.Long2LongMap; -import it.unimi.dsi.fastutil.longs.Long2LongOpenHashMap; -import org.testng.annotations.Test; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; -import java.util.stream.IntStream; - -import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static io.trino.operator.exchange.UniformPartitionRebalancer.WriterPartitionId; -import static io.trino.operator.exchange.UniformPartitionRebalancer.WriterPartitionId.serialize; -import static org.assertj.core.api.Assertions.assertThat; - -public class TestUniformPartitionRebalancer -{ - @Test - public void testRebalanceWithWriterSkewness() - { - AtomicLong physicalWrittenBytesForWriter0 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter1 = new AtomicLong(0); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 4, - 2, - DataSize.of(4, MEGABYTE).toBytes()); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 2L, - new WriterPartitionId(1, 1), 20000L, - new WriterPartitionId(0, 2), 2L, - new WriterPartitionId(1, 3), 20000L))); - - physicalWrittenBytesForWriter1.set(DataSize.of(200, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(0), - ImmutableList.of(1, 0)); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 3), 10000L, - new WriterPartitionId(1, 3), 10000L, - new WriterPartitionId(1, 1), 40000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(50, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(500, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1, 0), - ImmutableList.of(0), - ImmutableList.of(1, 0)); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 1), 10000L, - new WriterPartitionId(1, 1), 10000L, - new WriterPartitionId(0, 3), 10000L, - new WriterPartitionId(1, 3), 20000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(100, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1, 0), - ImmutableList.of(0), - ImmutableList.of(1, 0)); - } - - @Test - public void testComputeRebalanceThroughputWithAllWritersOfTheSamePartition() - { - AtomicLong physicalWrittenBytesForWriter0 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter1 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter2 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter3 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter4 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter5 = new AtomicLong(0); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get, - physicalWrittenBytesForWriter2::get, - physicalWrittenBytesForWriter3::get, - physicalWrittenBytesForWriter4::get, - physicalWrittenBytesForWriter5::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 2, - 6, - DataSize.of(4, MEGABYTE).toBytes()); - - // init 6 writers and 2 partitions, so partition0 -> writer0 and partition1 -> writer1 - assertThat(getWriterIdsForPartitions(partitionRebalancer, 2)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1)); - - // new data, partition0 -> writer0 has 100M, and partition1 -> writer1 has 1M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 10000L, - new WriterPartitionId(1, 1), 100L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(100, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(1, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition0 rebalanced, partition0 -> writer[0, 2] - // partition1's data is less than threshold. - assertThat(getWriterIdsForPartitions(partitionRebalancer, 2)) - .containsExactly( - ImmutableList.of(0, 2), - ImmutableList.of(1)); - - // new data, partition0 -> writer[0, 2] each has 100M, and partition1 -> writer1 has 1M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 20000L, - new WriterPartitionId(1, 1), 200L, - new WriterPartitionId(2, 0), 10000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(200, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(2, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter2.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition0 rebalanced, partition0 -> writer[0, 2, 3] - // partition1's data is less than threshold. - assertThat(getWriterIdsForPartitions(partitionRebalancer, 2)) - .containsExactly( - ImmutableList.of(0, 2, 3), - ImmutableList.of(1)); - - // new data, partition0 -> writer[0, 2, 3] each has 100M, and partition1 -> writer1 has 1M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 30000L, - new WriterPartitionId(1, 1), 300L, - new WriterPartitionId(2, 0), 20000L, - new WriterPartitionId(3, 0), 10000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(300, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(3, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter2.set(DataSize.of(200, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter3.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition0 rebalanced, partition0 -> writer[0, 2, 3, 4] - // partition1's data is less than threshold. - assertThat(getWriterIdsForPartitions(partitionRebalancer, 2)) - .containsExactly( - ImmutableList.of(0, 2, 3, 4), - ImmutableList.of(1)); - - // new data, partition0 -> writer[0, 2, 3, 4] each has 100M, and partition1 -> writer1 has 90M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 40000L, - new WriterPartitionId(1, 1), 9300L, - new WriterPartitionId(2, 0), 30000L, - new WriterPartitionId(3, 0), 20000L, - new WriterPartitionId(4, 0), 10000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(400, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(93, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter2.set(DataSize.of(300, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter3.set(DataSize.of(200, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter4.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition0 be rebalanced, partition0 -> writer[0, 2, 3, 4, 5] - // only partition0 rebalanced, because after rebalanced partition0 - // we estimate 6 writers' throughput are [80, 90, 80, 80, 80, 80], - // and data skew is less than threshold. - assertThat(getWriterIdsForPartitions(partitionRebalancer, 2)) - .containsExactly( - ImmutableList.of(0, 2, 3, 4, 5), - ImmutableList.of(1)); - } - - @Test - public void testRebalanceAffectAllWritersOfTheSamePartition() - { - AtomicLong physicalWrittenBytesForWriter0 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter1 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter2 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter3 = new AtomicLong(0); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get, - physicalWrittenBytesForWriter2::get, - physicalWrittenBytesForWriter3::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 3, - 4, - DataSize.of(4, MEGABYTE).toBytes()); - - // init 4 writers and 3 partitions, so partition0 -> writer0, partition1 -> writer1 and - // partition2 -> writer2 - assertThat(getWriterIdsForPartitions(partitionRebalancer, 3)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(2)); - - // new data, partition0 -> writer0 has 100M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 10000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition0 rebalanced, partition0 -> writer[0, 1] - assertThat(getWriterIdsForPartitions(partitionRebalancer, 3)) - .containsExactly( - ImmutableList.of(0, 1), - ImmutableList.of(1), - ImmutableList.of(2)); - - // new data, partition1 -> writer1 has 100M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 10000L, - new WriterPartitionId(1, 1), 10000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(100, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition1 rebalanced, partition0 -> writer[0, 1], partition1 -> writer[1, 0] - assertThat(getWriterIdsForPartitions(partitionRebalancer, 3)) - .containsExactly( - ImmutableList.of(0, 1), - ImmutableList.of(1, 0), - ImmutableList.of(2)); - - // new data, partition0 -> wrter0 31M, partition0 -> writer1 30M - // partition1 -> writer0 10M, partition1 -> writer1 10M - // partition2 -> writer2 10M - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 13000L, - new WriterPartitionId(0, 1), 3000L, - new WriterPartitionId(1, 0), 1000L, - new WriterPartitionId(1, 1), 11000L, - new WriterPartitionId(2, 2), 1000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(141, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(140, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter2.set(DataSize.of(10, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - // check partition0 rebalanced, partition0 -> writer[0, 1, 3] - // this affect the writer1 and writer3's throughput, - // now all writers' throughput is [30, 30, 10, 20] and the skew is less than threshold, - // no more rebalance needed. - assertThat(getWriterIdsForPartitions(partitionRebalancer, 3)) - .containsExactly( - ImmutableList.of(0, 1, 3), - ImmutableList.of(1, 0), - ImmutableList.of(2)); - } - - @Test - public void testNoRebalanceWhenDataWrittenIsLessThanTheRebalanceLimit() - { - AtomicLong physicalWrittenBytesForWriter0 = new AtomicLong(0); - AtomicLong physicalWrittenBytesForWriter1 = new AtomicLong(0); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 4, - 2, - DataSize.of(4, MEGABYTE).toBytes()); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 2L, - new WriterPartitionId(1, 1), 20000L, - new WriterPartitionId(0, 2), 2L, - new WriterPartitionId(1, 3), 20000L))); - - physicalWrittenBytesForWriter1.set(DataSize.of(30, MEGABYTE).toBytes()); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(0), - ImmutableList.of(1)); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(0), - ImmutableList.of(1)); - } - - @Test - public void testNoRebalanceWithoutWriterSkewness() - { - AtomicReference physicalWrittenBytesForWriter0 = new AtomicReference<>(0L); - AtomicReference physicalWrittenBytesForWriter1 = new AtomicReference<>(0L); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 4, - 2, - DataSize.of(4, MEGABYTE).toBytes()); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 20000L, - new WriterPartitionId(1, 1), 20000L, - new WriterPartitionId(0, 2), 20000L, - new WriterPartitionId(1, 3), 20000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(50, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter1.set(DataSize.of(100, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(0), - ImmutableList.of(1)); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(0), - ImmutableList.of(1)); - } - - @Test - public void testNoRebalanceWhenDataWrittenByThePartitionIsLessThanWriterMinSize() - { - AtomicReference physicalWrittenBytesForWriter0 = new AtomicReference<>(0L); - AtomicReference physicalWrittenBytesForWriter1 = new AtomicReference<>(0L); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 4, - 2, - DataSize.of(500, MEGABYTE).toBytes()); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 2L, - new WriterPartitionId(1, 1), 20000L, - new WriterPartitionId(0, 2), 2L, - new WriterPartitionId(1, 3), 20000L))); - - physicalWrittenBytesForWriter1.set(DataSize.of(200, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 4)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(0), - ImmutableList.of(1)); - } - - @Test - public void testPartitionShouldNotScaledTwiceInTheSameRebalanceCall() - { - AtomicReference physicalWrittenBytesForWriter0 = new AtomicReference<>(0L); - AtomicReference physicalWrittenBytesForWriter1 = new AtomicReference<>(0L); - AtomicReference physicalWrittenBytesForWriter2 = new AtomicReference<>(0L); - List> writerPhysicalWrittenBytes = ImmutableList.of( - physicalWrittenBytesForWriter0::get, - physicalWrittenBytesForWriter1::get, - physicalWrittenBytesForWriter2::get); - AtomicReference partitionRowCounts = new AtomicReference<>(new Long2LongOpenHashMap()); - - UniformPartitionRebalancer partitionRebalancer = new UniformPartitionRebalancer( - writerPhysicalWrittenBytes, - partitionRowCounts::get, - 6, - 3, - DataSize.of(32, MEGABYTE).toBytes()); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 0), 2L, - new WriterPartitionId(1, 1), 2L, - new WriterPartitionId(2, 2), 2L, - new WriterPartitionId(0, 3), 2L, - new WriterPartitionId(1, 4), 2L, - new WriterPartitionId(2, 5), 20000L))); - - physicalWrittenBytesForWriter2.set(DataSize.of(200, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 6)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(2), - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(2, 0)); - - partitionRowCounts.set(serializeToLong2LongMap(ImmutableMap.of( - new WriterPartitionId(0, 5), 10000L, - new WriterPartitionId(2, 5), 10000L))); - - physicalWrittenBytesForWriter0.set(DataSize.of(100, MEGABYTE).toBytes()); - physicalWrittenBytesForWriter2.set(DataSize.of(300, MEGABYTE).toBytes()); - - partitionRebalancer.rebalancePartitions(); - - assertThat(getWriterIdsForPartitions(partitionRebalancer, 6)) - .containsExactly( - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(2), - ImmutableList.of(0), - ImmutableList.of(1), - ImmutableList.of(2, 0, 1)); - } - - private Long2LongMap serializeToLong2LongMap(Map input) - { - return new Long2LongOpenHashMap( - input.entrySet().stream() - .collect(toImmutableMap( - entry -> serialize(entry.getKey()), - Map.Entry::getValue))); - } - - private List> getWriterIdsForPartitions(UniformPartitionRebalancer partitionRebalancer, int partitionCount) - { - return IntStream.range(0, partitionCount) - .mapToObj(partitionRebalancer::getWriterIds) - .collect(toImmutableList()); - } -} diff --git a/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java b/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java index 176bf7f0e2f2..4698a161903c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java +++ b/core/trino-main/src/test/java/io/trino/operator/output/TestSkewedPartitionRebalancer.java @@ -32,12 +32,13 @@ public class TestSkewedPartitionRebalancer { private static final long MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD = DataSize.of(1, MEGABYTE).toBytes(); + private static final long MIN_DATA_PROCESSED_REBALANCE_THRESHOLD = DataSize.of(50, MEGABYTE).toBytes(); @Test public void testRebalanceWithSkewness() { int partitionCount = 3; - SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD); + SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -92,7 +93,7 @@ public void testRebalanceWithSkewness() public void testRebalanceWithoutSkewness() { int partitionCount = 6; - SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 4, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD); + SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 4, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -118,7 +119,7 @@ public void testRebalanceWithoutSkewness() public void testNoRebalanceWhenDataWrittenIsLessThanTheRebalanceLimit() { int partitionCount = 3; - SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD); + SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); @@ -138,18 +139,18 @@ public void testNoRebalanceWhenDataWrittenIsLessThanTheRebalanceLimit() } @Test - public void testNoRebalanceWhenDataWrittenByThePartitionIsLessThanWriterMinSize() + public void testNoRebalanceWhenDataWrittenByThePartitionIsLessThanWriterScalingMinDataProcessed() { int partitionCount = 3; long minPartitionDataProcessedRebalanceThreshold = DataSize.of(50, MEGABYTE).toBytes(); - SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, minPartitionDataProcessedRebalanceThreshold); + SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, minPartitionDataProcessedRebalanceThreshold, MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); rebalancer.addPartitionRowCount(0, 1000); rebalancer.addPartitionRowCount(1, 600); rebalancer.addPartitionRowCount(2, 0); rebalancer.addDataProcessed(DataSize.of(60, MEGABYTE).toBytes()); - // No rebalancing will happen since no partition has crossed the writerMinSize limit of 50MB + // No rebalancing will happen since no partition has crossed the writerScalingMinDataProcessed limit of 50MB rebalancer.rebalance(); assertThat(getPartitionPositions(function, 6)) @@ -161,6 +162,45 @@ public void testNoRebalanceWhenDataWrittenByThePartitionIsLessThanWriterMinSize( .containsExactly(ImmutableList.of(0), ImmutableList.of(1), ImmutableList.of(2)); } + @Test + public void testRebalancePartitionToSingleTaskInARebalancingLoop() + { + int partitionCount = 3; + SkewedPartitionRebalancer rebalancer = createSkewedPartitionRebalancer(partitionCount, 3, 6, MIN_PARTITION_DATA_PROCESSED_REBALANCE_THRESHOLD, MIN_DATA_PROCESSED_REBALANCE_THRESHOLD); + SkewedPartitionFunction function = new SkewedPartitionFunction(new TestPartitionFunction(partitionCount), rebalancer); + + rebalancer.addPartitionRowCount(0, 1000); + rebalancer.addPartitionRowCount(1, 0); + rebalancer.addPartitionRowCount(2, 0); + + rebalancer.addDataProcessed(DataSize.of(60, MEGABYTE).toBytes()); + // rebalancing will only happen to single task even though two tasks are available + rebalancer.rebalance(); + + assertThat(getPartitionPositions(function, 17)) + .containsExactly( + new IntArrayList(ImmutableList.of(0, 6, 12)), + new IntArrayList(ImmutableList.of(1, 3, 4, 7, 9, 10, 13, 15, 16)), + new IntArrayList(ImmutableList.of(2, 5, 8, 11, 14))); + assertThat(rebalancer.getPartitionAssignments()) + .containsExactly(ImmutableList.of(0, 1), ImmutableList.of(1), ImmutableList.of(2)); + + rebalancer.addPartitionRowCount(0, 1000); + rebalancer.addPartitionRowCount(1, 0); + rebalancer.addPartitionRowCount(2, 0); + + rebalancer.addDataProcessed(DataSize.of(60, MEGABYTE).toBytes()); + rebalancer.rebalance(); + + assertThat(getPartitionPositions(function, 17)) + .containsExactly( + new IntArrayList(ImmutableList.of(0, 9)), + new IntArrayList(ImmutableList.of(1, 3, 4, 7, 10, 12, 13, 16)), + new IntArrayList(ImmutableList.of(2, 5, 6, 8, 11, 14, 15))); + assertThat(rebalancer.getPartitionAssignments()) + .containsExactly(ImmutableList.of(0, 1, 2), ImmutableList.of(1), ImmutableList.of(2)); + } + private List> getPartitionPositions(PartitionFunction function, int maxPosition) { List> partitionPositions = new ArrayList<>(); diff --git a/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java b/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java index f3ab22373d25..801e36460b2c 100644 --- a/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java +++ b/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java @@ -879,6 +879,7 @@ private TaskStatus buildTaskStatus() initialTaskStatus.getRunningPartitionedDrivers(), initialTaskStatus.getOutputBufferStatus(), initialTaskStatus.getOutputDataSize(), + initialTaskStatus.getWriterInputDataSize(), initialTaskStatus.getPhysicalWrittenDataSize(), initialTaskStatus.getMaxWriterCount(), initialTaskStatus.getMemoryReservation(), diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java index e9edac5aaf0d..2226d98ac87f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java @@ -39,7 +39,7 @@ public void testDefaults() .setLegacyCatalogRoles(false) .setRedistributeWrites(true) .setScaleWriters(true) - .setWriterMinSize(DataSize.of(32, MEGABYTE)) + .setWriterScalingMinDataProcessed(DataSize.of(120, MEGABYTE)) .setRegexLibrary(JONI) .setRe2JDfaStatesLimit(Integer.MAX_VALUE) .setRe2JDfaRetries(5) @@ -74,7 +74,7 @@ public void testExplicitPropertyMappings() Map properties = ImmutableMap.builder() .put("redistribute-writes", "false") .put("scale-writers", "false") - .put("writer-min-size", "42GB") + .put("writer-scaling-min-data-processed", "4GB") .put("regex-library", "RE2J") .put("re2j.dfa-states-limit", "42") .put("re2j.dfa-retries", "42") @@ -106,7 +106,7 @@ public void testExplicitPropertyMappings() FeaturesConfig expected = new FeaturesConfig() .setRedistributeWrites(false) .setScaleWriters(false) - .setWriterMinSize(DataSize.of(42, GIGABYTE)) + .setWriterScalingMinDataProcessed(DataSize.of(4, GIGABYTE)) .setRegexLibrary(RE2J) .setRe2JDfaStatesLimit(42) .setRe2JDfaRetries(42) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestingWriterTarget.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestingWriterTarget.java index f20e07eecd05..a7ea3662a885 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestingWriterTarget.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestingWriterTarget.java @@ -29,12 +29,6 @@ public String toString() return "testing handle"; } - @Override - public boolean supportsReportingWrittenBytes(Metadata metadata, Session session) - { - return false; - } - @Override public boolean supportsMultipleWritersPerPartition(Metadata metadata, Session session) { diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveEmptyMergeWriterRuleSet.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveEmptyMergeWriterRuleSet.java index 730b8f6f842e..fa70ef647aff 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveEmptyMergeWriterRuleSet.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveEmptyMergeWriterRuleSet.java @@ -89,7 +89,7 @@ private void testRemoveEmptyMergeRewrite(Rule rule, boolean pla List.of(rowCount)); return p.tableFinish( planWithExchange ? withExchange(p, merge, rowCount) : merge, - p.createTarget(catalogHandle, schemaTableName, true, true), + p.createTarget(catalogHandle, schemaTableName, true), rowCount); }) .matches(values("A")); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java index 41e2cd994b25..1ead492f4af8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java @@ -712,7 +712,7 @@ public TableFinishNode tableWithExchangeCreate(WriterTarget target, PlanNode sou rowCountSymbol); } - public CreateTarget createTarget(CatalogHandle catalogHandle, SchemaTableName schemaTableName, boolean reportingWrittenBytesSupported, boolean multipleWritersPerPartitionSupported, OptionalInt maxWriterTasks) + public CreateTarget createTarget(CatalogHandle catalogHandle, SchemaTableName schemaTableName, boolean multipleWritersPerPartitionSupported, OptionalInt maxWriterTasks) { OutputTableHandle tableHandle = new OutputTableHandle( catalogHandle, @@ -722,14 +722,13 @@ public CreateTarget createTarget(CatalogHandle catalogHandle, SchemaTableName sc return new CreateTarget( tableHandle, schemaTableName, - reportingWrittenBytesSupported, multipleWritersPerPartitionSupported, maxWriterTasks); } - public CreateTarget createTarget(CatalogHandle catalogHandle, SchemaTableName schemaTableName, boolean reportingWrittenBytesSupported, boolean multipleWritersPerPartitionSupported) + public CreateTarget createTarget(CatalogHandle catalogHandle, SchemaTableName schemaTableName, boolean multipleWritersPerPartitionSupported) { - return createTarget(catalogHandle, schemaTableName, reportingWrittenBytesSupported, multipleWritersPerPartitionSupported, OptionalInt.empty()); + return createTarget(catalogHandle, schemaTableName, multipleWritersPerPartitionSupported, OptionalInt.empty()); } public MergeWriterNode merge(SchemaTableName schemaTableName, PlanNode mergeSource, Symbol mergeRow, Symbol rowId, List outputs) @@ -1203,8 +1202,7 @@ public TableExecuteNode tableExecute( TestingTransactionHandle.create(), new TestingTableExecuteHandle()), Optional.empty(), - new SchemaTableName("schemaName", "tableName"), - false), + new SchemaTableName("schemaName", "tableName")), symbol("partialrows", BIGINT), symbol("fragment", VARBINARY), columns, diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesScaledWriters.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesScaledWriters.java index fe93273a64ed..7ac201fd02b9 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesScaledWriters.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesScaledWriters.java @@ -42,19 +42,16 @@ protected LocalQueryRunner createLocalQueryRunner() .build(); LocalQueryRunner queryRunner = LocalQueryRunner.create(session); queryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); - queryRunner.createCatalog("mock_dont_report_written_bytes", createConnectorFactorySupportingReportingBytesWritten(false, "mock_dont_report_written_bytes"), ImmutableMap.of()); - queryRunner.createCatalog("mock_report_written_bytes", createConnectorFactorySupportingReportingBytesWritten(true, "mock_report_written_bytes"), ImmutableMap.of()); + queryRunner.createCatalog("catalog", createConnectorFactory("catalog"), ImmutableMap.of()); return queryRunner; } - private MockConnectorFactory createConnectorFactorySupportingReportingBytesWritten(boolean supportsWrittenBytes, String name) + private MockConnectorFactory createConnectorFactory(String name) { - MockConnectorFactory connectorFactory = MockConnectorFactory.builder() - .withSupportsReportingWrittenBytes(supportsWrittenBytes) + return MockConnectorFactory.builder() .withGetTableHandle(((session, schemaTableName) -> null)) .withName(name) .build(); - return connectorFactory; } @DataProvider(name = "scale_writers") @@ -64,14 +61,14 @@ public Object[][] prepareScaledWritersOption() } @Test(dataProvider = "scale_writers") - public void testScaledWritersEnabled(boolean isScaleWritersEnabled) + public void testScaledWriters(boolean isScaleWritersEnabled) { Session session = testSessionBuilder() .setSystemProperty("scale_writers", Boolean.toString(isScaleWritersEnabled)) .build(); @Language("SQL") - String query = "CREATE TABLE mock_report_written_bytes.mock.test AS SELECT * FROM tpch.tiny.nation"; + String query = "CREATE TABLE catalog.mock.test AS SELECT * FROM tpch.tiny.nation"; SubPlan subPlan = subplan(query, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED, false, session); if (isScaleWritersEnabled) { assertThat(subPlan.getAllFragments().get(1).getPartitioning().getConnectorHandle()).isEqualTo(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION.getConnectorHandle()); @@ -81,18 +78,4 @@ public void testScaledWritersEnabled(boolean isScaleWritersEnabled) fragment -> assertThat(fragment.getPartitioning().getConnectorHandle()).isNotEqualTo(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION.getConnectorHandle())); } } - - @Test(dataProvider = "scale_writers") - public void testScaledWritersDisabled(boolean isScaleWritersEnabled) - { - Session session = testSessionBuilder() - .setSystemProperty("scale_writers", Boolean.toString(isScaleWritersEnabled)) - .build(); - - @Language("SQL") - String query = "CREATE TABLE mock_dont_report_written_bytes.mock.test AS SELECT * FROM tpch.tiny.nation"; - SubPlan subPlan = subplan(query, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED, false, session); - subPlan.getAllFragments().forEach( - fragment -> assertThat(fragment.getPartitioning().getConnectorHandle()).isNotEqualTo(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION.getConnectorHandle())); - } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForPartitionedInsertAndMerge.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForPartitionedInsertAndMerge.java index 4544b8fcc1af..d578e7d26008 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForPartitionedInsertAndMerge.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForPartitionedInsertAndMerge.java @@ -86,7 +86,6 @@ protected LocalQueryRunner createLocalQueryRunner() private MockConnectorFactory createMergeConnectorFactory() { return MockConnectorFactory.builder() - .withSupportsReportingWrittenBytes(true) .withGetTableHandle(((session, schemaTableName) -> { if (schemaTableName.getTableName().equals("source_table")) { return new MockConnectorTableHandle(schemaTableName); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java index 8433806e43c0..25c534ce1500 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java @@ -65,27 +65,21 @@ protected LocalQueryRunner createLocalQueryRunner() { LocalQueryRunner queryRunner = LocalQueryRunner.create(testSessionBuilder().build()); queryRunner.createCatalog( - "mock_dont_report_written_bytes", - createConnectorFactory("mock_dont_report_written_bytes", false, true), + "mock_without_multiple_writer_per_partition", + createConnectorFactory("mock_without_multiple_writer_per_partition", false), ImmutableMap.of()); queryRunner.createCatalog( - "mock_report_written_bytes_without_multiple_writer_per_partition", - createConnectorFactory("mock_report_written_bytes", true, false), - ImmutableMap.of()); - queryRunner.createCatalog( - "mock_report_written_bytes_with_multiple_writer_per_partition", - createConnectorFactory("mock_report_written_bytes_with_multiple_writer_per_partition", true, true), + "mock_with_multiple_writer_per_partition", + createConnectorFactory("mock_with_multiple_writer_per_partition", true), ImmutableMap.of()); return queryRunner; } private MockConnectorFactory createConnectorFactory( String catalogHandle, - boolean supportsWrittenBytes, boolean supportsMultipleWritersPerPartition) { return MockConnectorFactory.builder() - .withSupportsReportingWrittenBytes(supportsWrittenBytes) .withGetTableHandle(((session, tableName) -> { if (tableName.getTableName().equals("source_table") || tableName.getTableName().equals("system_partitioned_table") @@ -125,12 +119,12 @@ private MockConnectorFactory createConnectorFactory( } @Test - public void testLocalScaledUnpartitionedWriterDistributionWithSupportsReportingWrittenBytes() + public void testLocalScaledUnpartitionedWriterDistribution() { assertDistributedPlan( "INSERT INTO unpartitioned_table SELECT * FROM source_table", testSessionBuilder() - .setCatalog("mock_report_written_bytes_without_multiple_writer_per_partition") + .setCatalog("mock_without_multiple_writer_per_partition") .setSchema("mock") .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "true") .setSystemProperty(SCALE_WRITERS, "false") @@ -146,7 +140,7 @@ public void testLocalScaledUnpartitionedWriterDistributionWithSupportsReportingW assertDistributedPlan( "INSERT INTO unpartitioned_table SELECT * FROM source_table", testSessionBuilder() - .setCatalog("mock_report_written_bytes_without_multiple_writer_per_partition") + .setCatalog("mock_without_multiple_writer_per_partition") .setSchema("mock") .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "false") .setSystemProperty(SCALE_WRITERS, "false") @@ -163,122 +157,12 @@ public void testLocalScaledUnpartitionedWriterDistributionWithSupportsReportingW @Test(dataProvider = "taskScaleWritersOption") public void testLocalScaledPartitionedWriterWithoutSupportForMultipleWritersPerPartition(boolean taskScaleWritersEnabled) { - String catalogName = "mock_report_written_bytes_without_multiple_writer_per_partition"; - PartitioningHandle partitioningHandle = new PartitioningHandle( - Optional.of(getCatalogHandle(catalogName)), - Optional.of(MockConnectorTransactionHandle.INSTANCE), - CONNECTOR_PARTITIONING_HANDLE); - - assertDistributedPlan( - "INSERT INTO connector_partitioned_table SELECT * FROM source_table", - testSessionBuilder() - .setCatalog(catalogName) - .setSchema("mock") - .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, String.valueOf(taskScaleWritersEnabled)) - .setSystemProperty(SCALE_WRITERS, "false") - .build(), - anyTree( - tableWriter( - ImmutableList.of("customer", "year"), - ImmutableList.of("customer", "year"), - exchange(LOCAL, REPARTITION, partitioningHandle, - exchange(REMOTE, REPARTITION, partitioningHandle, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); - } - - @Test(dataProvider = "taskScaleWritersOption") - public void testLocalScaledUnpartitionedWriterDistributionWithoutSupportsReportingWrittenBytes(boolean taskScaleWritersEnabled) - { - assertDistributedPlan( - "INSERT INTO unpartitioned_table SELECT * FROM source_table", - testSessionBuilder() - .setCatalog("mock_dont_report_written_bytes") - .setSchema("mock") - .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, String.valueOf(taskScaleWritersEnabled)) - .setSystemProperty(SCALE_WRITERS, "false") - .build(), - anyTree( - tableWriter( - ImmutableList.of("customer", "year"), - ImmutableList.of("customer", "year"), - exchange(LOCAL, GATHER, SINGLE_DISTRIBUTION, - exchange(REMOTE, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); - } - - @Test(dataProvider = "taskScaleWritersOption") - public void testLocalScaledPartitionedWriterWithoutSupportsForReportingWrittenBytes(boolean taskScaleWritersEnabled) - { - String catalogName = "mock_dont_report_written_bytes"; - PartitioningHandle partitioningHandle = new PartitioningHandle( - Optional.of(getCatalogHandle(catalogName)), - Optional.of(MockConnectorTransactionHandle.INSTANCE), - CONNECTOR_PARTITIONING_HANDLE); - - assertDistributedPlan( - "INSERT INTO system_partitioned_table SELECT * FROM source_table", - testSessionBuilder() - .setCatalog(catalogName) - .setSchema("mock") - .setSystemProperty(USE_PREFERRED_WRITE_PARTITIONING, "true") - .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, String.valueOf(taskScaleWritersEnabled)) - .setSystemProperty(SCALE_WRITERS, "false") - .build(), - anyTree( - tableWriter( - ImmutableList.of("customer", "year"), - ImmutableList.of("customer", "year"), - project( - exchange(LOCAL, REPARTITION, FIXED_HASH_DISTRIBUTION, - exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); - - assertDistributedPlan( - "INSERT INTO connector_partitioned_table SELECT * FROM source_table", - testSessionBuilder() - .setCatalog(catalogName) - .setSchema("mock") - .setSystemProperty(USE_PREFERRED_WRITE_PARTITIONING, "true") - .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, String.valueOf(taskScaleWritersEnabled)) - .setSystemProperty(SCALE_WRITERS, "false") - .build(), - anyTree( - tableWriter( - ImmutableList.of("customer", "year"), - ImmutableList.of("customer", "year"), - exchange(LOCAL, REPARTITION, partitioningHandle, - exchange(REMOTE, REPARTITION, partitioningHandle, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); - } - - @Test(dataProvider = "taskScaleWritersOption") - public void testLocalScaledPartitionedWriterWithoutSupportForReportingWrittenBytesAndPreferredPartitioning(boolean taskScaleWritersEnabled) - { - String catalogName = "mock_dont_report_written_bytes"; + String catalogName = "mock_without_multiple_writer_per_partition"; PartitioningHandle partitioningHandle = new PartitioningHandle( Optional.of(getCatalogHandle(catalogName)), Optional.of(MockConnectorTransactionHandle.INSTANCE), CONNECTOR_PARTITIONING_HANDLE); - assertDistributedPlan( - "INSERT INTO system_partitioned_table SELECT * FROM source_table", - testSessionBuilder() - .setCatalog(catalogName) - .setSchema("mock") - .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, String.valueOf(taskScaleWritersEnabled)) - .setSystemProperty(SCALE_WRITERS, "false") - .build(), - anyTree( - tableWriter( - ImmutableList.of("customer", "year"), - ImmutableList.of("customer", "year"), - project( - exchange(LOCAL, REPARTITION, FIXED_HASH_DISTRIBUTION, - exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); - assertDistributedPlan( "INSERT INTO connector_partitioned_table SELECT * FROM source_table", testSessionBuilder() @@ -308,7 +192,7 @@ public void testLocalScaledPartitionedWriterForSystemPartitioningWithEnforcedPre assertDistributedPlan( "INSERT INTO system_partitioned_table SELECT * FROM source_table", testSessionBuilder() - .setCatalog("mock_report_written_bytes_with_multiple_writer_per_partition") + .setCatalog("mock_with_multiple_writer_per_partition") .setSchema("mock") // Enforce preferred partitioning .setSystemProperty(USE_PREFERRED_WRITE_PARTITIONING, "true") @@ -328,7 +212,7 @@ public void testLocalScaledPartitionedWriterForSystemPartitioningWithEnforcedPre assertDistributedPlan( "INSERT INTO system_partitioned_table SELECT * FROM source_table", testSessionBuilder() - .setCatalog("mock_report_written_bytes_with_multiple_writer_per_partition") + .setCatalog("mock_with_multiple_writer_per_partition") .setSchema("mock") // Enforce preferred partitioning .setSystemProperty(USE_PREFERRED_WRITE_PARTITIONING, "true") @@ -349,7 +233,7 @@ public void testLocalScaledPartitionedWriterForSystemPartitioningWithEnforcedPre @Test public void testLocalScaledPartitionedWriterForConnectorPartitioning() { - String catalogName = "mock_report_written_bytes_with_multiple_writer_per_partition"; + String catalogName = "mock_with_multiple_writer_per_partition"; PartitioningHandle partitioningHandle = new PartitioningHandle( Optional.of(getCatalogHandle(catalogName)), Optional.of(MockConnectorTransactionHandle.INSTANCE), @@ -399,7 +283,7 @@ public void testLocalScaledPartitionedWriterWithEnforcedLocalPreferredPartitioni assertDistributedPlan( "INSERT INTO system_partitioned_table SELECT * FROM source_table", testSessionBuilder() - .setCatalog("mock_report_written_bytes_with_multiple_writer_per_partition") + .setCatalog("mock_with_multiple_writer_per_partition") .setSchema("mock") .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "true") .setSystemProperty(SCALE_WRITERS, "false") @@ -417,7 +301,7 @@ public void testLocalScaledPartitionedWriterWithEnforcedLocalPreferredPartitioni assertDistributedPlan( "INSERT INTO system_partitioned_table SELECT * FROM source_table", testSessionBuilder() - .setCatalog("mock_report_written_bytes_with_multiple_writer_per_partition") + .setCatalog("mock_with_multiple_writer_per_partition") .setSchema("mock") .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "false") .setSystemProperty(SCALE_WRITERS, "false") diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java index 1cd9ee45903f..6a3b137d6d0d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestLimitMaxWriterNodesCount.java @@ -129,7 +129,6 @@ private MockConnectorFactory prepareConnectorFactory(String catalogName, Optiona distributedWithFilteringAndRepartitioning(), ImmutableList.of(PropertyMetadata.stringProperty("file_size_threshold", "file_size_threshold", "10GB", false))))) .withPartitionProvider(new TestTableScanNodePartitioning.TestPartitioningProvider(new InMemoryNodeManager())) - .withSupportsReportingWrittenBytes(true) .withMaxWriterTasks(maxWriterTasks) .withGetColumns(schemaTableName -> ImmutableList.of( new ColumnMetadata("column_a", VARCHAR), diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java index 2958c1e363aa..19a3e4931da0 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestValidateScaledWritersUsage.java @@ -62,23 +62,20 @@ public class TestValidateScaledWritersUsage private PlanBuilder planBuilder; private Symbol symbol; private TableScanNode tableScanNode; - private CatalogHandle catalogSupportingScaledWriters; - private CatalogHandle catalogNotSupportingScaledWriters; + private CatalogHandle catalog; private SchemaTableName schemaTableName; @BeforeClass public void setup() { schemaTableName = new SchemaTableName("any", "any"); - catalogSupportingScaledWriters = createTestCatalogHandle("bytes_written_reported"); - catalogNotSupportingScaledWriters = createTestCatalogHandle("no_bytes_written_reported"); + catalog = createTestCatalogHandle("catalog"); queryRunner = LocalQueryRunner.create(TEST_SESSION); - queryRunner.createCatalog(catalogSupportingScaledWriters.getCatalogName(), createConnectorFactorySupportingReportingBytesWritten(true, catalogSupportingScaledWriters.getCatalogName()), ImmutableMap.of()); - queryRunner.createCatalog(catalogNotSupportingScaledWriters.getCatalogName(), createConnectorFactorySupportingReportingBytesWritten(false, catalogNotSupportingScaledWriters.getCatalogName()), ImmutableMap.of()); + queryRunner.createCatalog(catalog.getCatalogName(), createConnectorFactory(catalog.getCatalogName()), ImmutableMap.of()); plannerContext = queryRunner.getPlannerContext(); planBuilder = new PlanBuilder(new PlanNodeIdAllocator(), plannerContext.getMetadata(), TEST_SESSION); TableHandle nationTableHandle = new TableHandle( - catalogSupportingScaledWriters, + catalog, new TpchTableHandle("sf1", "nation", 1.0), TestingTransactionHandle.create()); TpchColumnHandle nationkeyColumnHandle = new TpchColumnHandle("nationkey", BIGINT); @@ -94,14 +91,12 @@ public void tearDown() plannerContext = null; planBuilder = null; tableScanNode = null; - catalogSupportingScaledWriters = null; - catalogNotSupportingScaledWriters = null; + catalog = null; } - private MockConnectorFactory createConnectorFactorySupportingReportingBytesWritten(boolean supportsWrittenBytes, String name) + private MockConnectorFactory createConnectorFactory(String name) { return MockConnectorFactory.builder() - .withSupportsReportingWrittenBytes(supportsWrittenBytes) .withGetTableHandle(((session, schemaTableName) -> null)) .withName(name) .build(); @@ -112,7 +107,7 @@ public void testScaledWritersUsedAndTargetSupportsIt(PartitioningHandle scaledWr { PlanNode tableWriterSource = planBuilder.exchange(ex -> ex - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) + .partitioningScheme(new PartitioningScheme(Partitioning.create(scaledWriterPartitionHandle, ImmutableList.of()), ImmutableList.of(symbol))) .addInputsSet(symbol) .addSource(planBuilder.exchange(innerExchange -> innerExchange @@ -122,141 +117,12 @@ public void testScaledWritersUsedAndTargetSupportsIt(PartitioningHandle scaledWr PlanNode root = planBuilder.output( outputBuilder -> outputBuilder .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogSupportingScaledWriters, schemaTableName, true, true), + planBuilder.createTarget(catalog, schemaTableName, true), tableWriterSource, symbol))); validatePlan(root); } - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersUsedAndTargetDoesNotSupportReportingWrittenBytes(PartitioningHandle scaledWriterPartitionHandle) - { - PlanNode tableWriterSource = planBuilder.exchange(ex -> - ex - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(scaledWriterPartitionHandle, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode)))); - PlanNode root = planBuilder.output( - outputBuilder -> outputBuilder - .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogNotSupportingScaledWriters, schemaTableName, false, true), - tableWriterSource, - symbol))); - assertThatThrownBy(() -> validatePlan(root)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("The scaled writer partitioning scheme is set but writer target no_bytes_written_reported:INSTANCE doesn't support reporting physical written bytes"); - } - - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersWithMultipleSourceExchangesAndTargetDoesNotSupportReportingWrittenBytes(PartitioningHandle scaledWriterPartitionHandle) - { - PlanNode tableWriterSource = planBuilder.exchange(ex -> - ex - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol, symbol))) - .addInputsSet(symbol, symbol) - .addInputsSet(symbol, symbol) - .addSource(planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(scaledWriterPartitionHandle, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode))) - .addSource(planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode)))); - PlanNode root = planBuilder.output( - outputBuilder -> outputBuilder - .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogNotSupportingScaledWriters, schemaTableName, false, true), - tableWriterSource, - symbol))); - assertThatThrownBy(() -> validatePlan(root)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("The scaled writer partitioning scheme is set but writer target no_bytes_written_reported:INSTANCE doesn't support reporting physical written bytes"); - } - - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersWithMultipleSourceExchangesAndTargetSupportIt(PartitioningHandle scaledWriterPartitionHandle) - { - PlanNode tableWriterSource = planBuilder.exchange(ex -> - ex - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol, symbol))) - .addInputsSet(symbol, symbol) - .addInputsSet(symbol, symbol) - .addSource(planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(scaledWriterPartitionHandle, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode))) - .addSource(planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode)))); - PlanNode root = planBuilder.output( - outputBuilder -> outputBuilder - .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogSupportingScaledWriters, schemaTableName, true, true), - tableWriterSource, - symbol))); - validatePlan(root); - } - - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersUsedAboveTableWriterInThePlanTree(PartitioningHandle scaledWriterPartitionHandle) - { - PlanNode tableWriterSource = planBuilder.exchange(ex -> - ex - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode)))); - PlanNode root = planBuilder.output( - outputBuilder -> outputBuilder - .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogNotSupportingScaledWriters, schemaTableName, false, true), - tableWriterSource, - symbol))); - validatePlan(root); - } - - @Test(dataProvider = "scaledWriterPartitioningHandles") - public void testScaledWritersTwoTableWritersNodes(PartitioningHandle scaledWriterPartitionHandle) - { - PlanNode tableWriterSource = planBuilder.exchange(ex -> - ex - .partitioningScheme(new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(planBuilder.tableWriter( - ImmutableList.of(symbol), - ImmutableList.of("column_a"), - Optional.empty(), - planBuilder.createTarget(catalogSupportingScaledWriters, schemaTableName, true, true), - planBuilder.exchange(innerExchange -> - innerExchange - .partitioningScheme(new PartitioningScheme(Partitioning.create(scaledWriterPartitionHandle, ImmutableList.of()), ImmutableList.of(symbol))) - .addInputsSet(symbol) - .addSource(tableScanNode)), - symbol))); - PlanNode root = planBuilder.output( - outputBuilder -> outputBuilder - .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogNotSupportingScaledWriters, schemaTableName, false, true), - tableWriterSource, - symbol))); - assertThatThrownBy(() -> validatePlan(root)) - .isInstanceOf(IllegalStateException.class) - .hasMessage("The scaled writer partitioning scheme is set but writer target no_bytes_written_reported:INSTANCE doesn't support reporting physical written bytes"); - } - @Test(dataProvider = "scaledWriterPartitioningHandles") public void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartition(PartitioningHandle scaledWriterPartitionHandle) { @@ -272,7 +138,7 @@ public void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartiti PlanNode root = planBuilder.output( outputBuilder -> outputBuilder .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogNotSupportingScaledWriters, schemaTableName, true, false), + planBuilder.createTarget(catalog, schemaTableName, false), tableWriterSource, symbol))); @@ -282,7 +148,7 @@ public void testScaledWriterUsedAndTargetDoesNotSupportMultipleWritersPerPartiti else { assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(IllegalStateException.class) - .hasMessage("The scaled writer partitioning scheme is set for the partitioned write but writer target no_bytes_written_reported:INSTANCE doesn't support multiple writers per partition"); + .hasMessage("The hash scaled writer partitioning scheme is set for the partitioned write but writer target catalog:INSTANCE doesn't support multiple writers per partition"); } } @@ -307,7 +173,7 @@ public void testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMu PlanNode root = planBuilder.output( outputBuilder -> outputBuilder .source(planBuilder.tableWithExchangeCreate( - planBuilder.createTarget(catalogNotSupportingScaledWriters, schemaTableName, true, false), + planBuilder.createTarget(catalog, schemaTableName, false), tableWriterSource, symbol))); @@ -317,7 +183,7 @@ public void testScaledWriterWithMultipleSourceExchangesAndTargetDoesNotSupportMu else { assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(IllegalStateException.class) - .hasMessage("The scaled writer partitioning scheme is set for the partitioned write but writer target no_bytes_written_reported:INSTANCE doesn't support multiple writers per partition"); + .hasMessage("The hash scaled writer partitioning scheme is set for the partitioned write but writer target catalog:INSTANCE doesn't support multiple writers per partition"); } } @@ -339,8 +205,7 @@ private void validatePlan(PlanNode root) { queryRunner.inTransaction(session -> { // metadata.getCatalogHandle() registers the catalog for the transaction - plannerContext.getMetadata().getCatalogHandle(session, catalogSupportingScaledWriters.getCatalogName()); - plannerContext.getMetadata().getCatalogHandle(session, catalogNotSupportingScaledWriters.getCatalogName()); + plannerContext.getMetadata().getCatalogHandle(session, catalog.getCatalogName()); new ValidateScaledWritersUsage().validate( root, session, diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java index 433a6254294a..81d5651ee22f 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorMetadata.java @@ -1470,11 +1470,15 @@ default Optional redirectTable(ConnectorSession session, return Optional.empty(); } + // TODO - Remove this method since now it is only used in test BaseConnectorTest#testWrittenDataSize() + @Deprecated default boolean supportsReportingWrittenBytes(ConnectorSession session, SchemaTableName schemaTableName, Map tableProperties) { return false; } + // TODO - Remove this method since now it is only used in test BaseConnectorTest#testWrittenDataSize() + @Deprecated default boolean supportsReportingWrittenBytes(ConnectorSession session, ConnectorTableHandle connectorTableHandle) { return false; diff --git a/docs/src/main/sphinx/admin/properties-writer-scaling.rst b/docs/src/main/sphinx/admin/properties-writer-scaling.rst index aa093b65cb68..6c21eef13219 100644 --- a/docs/src/main/sphinx/admin/properties-writer-scaling.rst +++ b/docs/src/main/sphinx/admin/properties-writer-scaling.rst @@ -33,8 +33,8 @@ session property. Enable scaling the number of concurrent writers within a task. The maximum writer count per task for scaling is ``task.scale-writers.max-writer-count``. Additional -writers are added only when the average amount of physical data written per writer -is above the minimum threshold of ``writer-min-size`` and query is bottlenecked on +writers are added only when the average amount of uncompressed data processed per writer +is above the minimum threshold of ``writer-scaling-min-data-processed`` and query is bottlenecked on writing. This can be specified on a per-query basis using the ``task_scale_writers_enabled`` session property. @@ -61,3 +61,15 @@ another writer is eligible to be added. Each writer task may have multiple writers, controlled by ``task.writer-count``, thus this value is effectively divided by the number of writers per task. This can be specified on a per-query basis using the ``writer_min_size`` session property. + +.. warning:: + This property is deprecated now. Please use ``writer-scaling-min-data-processed``. + +``writer-scaling-min-data-processed`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +* **Type:** :ref:`prop-type-data-size` +* **Default value:** ``100MB`` + +The minimum amount of uncompressed data that must be processed by a writer +before another writer can be added. This can be specified on a +per-query basis using the ``writer_scaling_min_data_processed`` session property. diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index 23db1f13f420..860d83b6b0fc 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -119,7 +119,7 @@ import static io.trino.SystemSessionProperties.TASK_SCALE_WRITERS_MAX_WRITER_COUNT; import static io.trino.SystemSessionProperties.TASK_WRITER_COUNT; import static io.trino.SystemSessionProperties.USE_TABLE_SCAN_NODE_PARTITIONING; -import static io.trino.SystemSessionProperties.WRITER_MIN_SIZE; +import static io.trino.SystemSessionProperties.WRITER_SCALING_MIN_DATA_PROCESSED; import static io.trino.plugin.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; import static io.trino.plugin.hive.HiveColumnHandle.FILE_MODIFIED_TIME_COLUMN_NAME; import static io.trino.plugin.hive.HiveColumnHandle.FILE_SIZE_COLUMN_NAME; @@ -4055,7 +4055,7 @@ protected void testSingleWriter(Session session, HiveStorageFormat storageFormat .setSystemProperty("task_writer_count", "1") .setSystemProperty("scale_writers", "true") .setSystemProperty("task_scale_writers_enabled", "false") - .setSystemProperty("writer_min_size", "32MB") + .setSystemProperty("writer_scaling_min_data_processed", "32MB") .build(), createTableSql, (long) computeActual("SELECT count(*) FROM tpch.tiny.orders").getOnlyValue()); @@ -4080,7 +4080,7 @@ private void testMultipleWriters(Session session, HiveStorageFormat storageForma .setSystemProperty("task_writer_count", "1") .setSystemProperty("scale_writers", "true") .setSystemProperty("task_scale_writers_enabled", "false") - .setSystemProperty("writer_min_size", "1MB") + .setSystemProperty("writer_scaling_min_data_processed", "1MB") .setCatalogSessionProperty(catalog, "parquet_writer_block_size", "4MB") .build(), createTableSql, @@ -4110,7 +4110,7 @@ private void testMultipleWritersWithSkewedData(Session session, HiveStorageForma .setSystemProperty("task_writer_count", "1") .setSystemProperty("scale_writers", "true") .setSystemProperty("task_scale_writers_enabled", "false") - .setSystemProperty("writer_min_size", "1MB") + .setSystemProperty("writer_scaling_min_data_processed", "1MB") .setSystemProperty("join_distribution_type", "PARTITIONED") .setCatalogSessionProperty(catalog, "parquet_writer_block_size", "4MB") .build(), @@ -4182,14 +4182,14 @@ public void testWriterTasksCountLimitPartitionedScaleWritersEnabled() testLimitWriterTasks(2, 2, true, true, true, DataSize.of(32, MEGABYTE)); } - private void testLimitWriterTasks(int maxWriterTasks, int expectedFilesCount, boolean scaleWritersEnabled, boolean redistributeWrites, boolean partitioned, DataSize writerMinSize) + private void testLimitWriterTasks(int maxWriterTasks, int expectedFilesCount, boolean scaleWritersEnabled, boolean redistributeWrites, boolean partitioned, DataSize writerScalingMinDataProcessed) { Session session = Session.builder(getSession()) .setSystemProperty(SCALE_WRITERS, Boolean.toString(scaleWritersEnabled)) .setSystemProperty(MAX_WRITER_TASKS_COUNT, Integer.toString(maxWriterTasks)) .setSystemProperty(REDISTRIBUTE_WRITES, Boolean.toString(redistributeWrites)) .setSystemProperty(TASK_WRITER_COUNT, "1") - .setSystemProperty(WRITER_MIN_SIZE, writerMinSize.toString()) + .setSystemProperty(WRITER_SCALING_MIN_DATA_PROCESSED, writerScalingMinDataProcessed.toString()) .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "false") .build(); String tableName = "writing_tasks_limit_%s".formatted(randomNameSuffix()); @@ -4208,7 +4208,7 @@ private void testLimitWriterTasks(int maxWriterTasks, int expectedFilesCount, bo protected AbstractLongAssert testTaskScaleWriters( Session session, - DataSize writerMinSize, + DataSize writerScalingMinDataProcessed, int taskMaxScaleWriterCount, boolean scaleWriters) { @@ -4221,7 +4221,7 @@ protected AbstractLongAssert testTaskScaleWriters( Session.builder(session) .setSystemProperty(SCALE_WRITERS, String.valueOf(scaleWriters)) .setSystemProperty(TASK_SCALE_WRITERS_ENABLED, "true") - .setSystemProperty(WRITER_MIN_SIZE, writerMinSize.toString()) + .setSystemProperty(WRITER_SCALING_MIN_DATA_PROCESSED, writerScalingMinDataProcessed.toString()) .setSystemProperty(TASK_SCALE_WRITERS_MAX_WRITER_COUNT, String.valueOf(taskMaxScaleWriterCount)) // Set the value higher than sf1 input data size such that fault-tolerant scheduler // shouldn't add new task and scaling only happens through the local scaling exchange. @@ -8067,7 +8067,7 @@ public void testOptimizeWithWriterScaling() Session writerScalingSession = Session.builder(optimizeEnabledSession()) .setSystemProperty("scale_writers", "true") - .setSystemProperty("writer_min_size", "100GB") + .setSystemProperty("writer_scaling_min_data_processed", "100GB") .build(); assertUpdate(writerScalingSession, "ALTER TABLE " + tableName + " EXECUTE optimize(file_size_threshold => '10kB')"); @@ -8104,7 +8104,7 @@ public void testOptimizeWithPartitioning() Session optimizeEnabledSession = optimizeEnabledSession(); Session writerScalingSession = Session.builder(optimizeEnabledSession) .setSystemProperty("scale_writers", "true") - .setSystemProperty("writer_min_size", "100GB") + .setSystemProperty("writer_scaling_min_data_processed", "100GB") .build(); // optimize with unsupported WHERE diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java index 6a1718f23343..dbd77ca15869 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/hive/TestHiveTransactionalTable.java @@ -1902,7 +1902,7 @@ public void testDeleteWithOriginalFiles() withTemporaryTable("test_delete_with_original_files", true, false, NONE, tableName -> { // these 3 properties are necessary to make sure there is more than 1 original file created onTrino().executeQuery("SET SESSION scale_writers = true"); - onTrino().executeQuery("SET SESSION writer_min_size = '4kB'"); + onTrino().executeQuery("SET SESSION writer_scaling_min_data_processed = '4kB'"); onTrino().executeQuery("SET SESSION task_scale_writers_enabled = false"); onTrino().executeQuery("SET SESSION task_writer_count = 2"); onTrino().executeQuery(format( @@ -1923,7 +1923,7 @@ public void testDeleteWithOriginalFilesWithWhereClause() withTemporaryTable("test_delete_with_original_files_with_where_clause", true, false, NONE, tableName -> { // these 3 properties are necessary to make sure there is more than 1 original file created onTrino().executeQuery("SET SESSION scale_writers = true"); - onTrino().executeQuery("SET SESSION writer_min_size = '4kB'"); + onTrino().executeQuery("SET SESSION writer_scaling_min_data_processed = '4kB'"); onTrino().executeQuery("SET SESSION task_scale_writers_enabled = false"); onTrino().executeQuery("SET SESSION task_writer_count = 2"); onTrino().executeQuery(format("CREATE TABLE %s WITH (transactional = true) AS SELECT * FROM tpch.sf1000.orders LIMIT 100000", tableName)); @@ -1989,7 +1989,7 @@ private void unbucketedTransactionalTableWithTaskWriterCountGreaterThanOne(boole ") AS SELECT orderkey, orderstatus, totalprice, orderdate, clerk, shippriority, \"comment\", custkey, orderpriority " + "FROM tpch.sf1000.orders LIMIT 0", tableName, isPartitioned ? ", partitioned_by = ARRAY['orderpriority']" : "")); onTrino().executeQuery("SET SESSION scale_writers = true"); - onTrino().executeQuery("SET SESSION writer_min_size = '4kB'"); + onTrino().executeQuery("SET SESSION writer_scaling_min_data_processed = '4kB'"); onTrino().executeQuery("SET SESSION task_scale_writers_enabled = false"); onTrino().executeQuery("SET SESSION task_writer_count = 4"); onTrino().executeQuery("SET SESSION task_partitioned_writer_count = 4");