Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ public class FileSystemExchangeConfig
private int exchangeSinkBuffersPerPartition = 2;
private DataSize exchangeSinkMaxFileSize = DataSize.of(1, GIGABYTE);
private int exchangeSourceConcurrentReaders = 4;
private int maxOutputPartitionCount = 50;

@NotNull
@NotEmpty(message = "At least one base directory needs to be configured")
Expand Down Expand Up @@ -147,4 +148,17 @@ public FileSystemExchangeConfig setExchangeSourceConcurrentReaders(int exchangeS
this.exchangeSourceConcurrentReaders = exchangeSourceConcurrentReaders;
return this;
}

@Min(1)
public int getMaxOutputPartitionCount()
{
return maxOutputPartitionCount;
}

@Config("exchange.max-output-partition-count")
public FileSystemExchangeConfig setMaxOutputPartitionCount(int maxOutputPartitionCount)
{
this.maxOutputPartitionCount = maxOutputPartitionCount;
return this;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.plugin.exchange.filesystem;

import io.trino.spi.ErrorCode;
import io.trino.spi.ErrorCodeSupplier;
import io.trino.spi.ErrorType;

import static io.trino.spi.ErrorType.USER_ERROR;

public enum FileSystemExchangeErrorCode
implements ErrorCodeSupplier
{
MAX_OUTPUT_PARTITION_COUNT_EXCEEDED(0, USER_ERROR),
/**/;

private final ErrorCode errorCode;

FileSystemExchangeErrorCode(int code, ErrorType type)
{
errorCode = new ErrorCode(code + 0x0510_0000, name(), type);
}

@Override
public ErrorCode toErrorCode()
{
return errorCode;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,10 @@

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.airlift.concurrent.Threads.daemonThreadsNamed;
import static io.trino.plugin.exchange.filesystem.FileSystemExchangeErrorCode.MAX_OUTPUT_PARTITION_COUNT_EXCEEDED;
import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static java.lang.Math.toIntExact;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.Executors.newCachedThreadPool;

Expand All @@ -58,6 +60,7 @@ public class FileSystemExchangeManager
private final int exchangeSinkBuffersPerPartition;
private final long exchangeSinkMaxFileSizeInBytes;
private final int exchangeSourceConcurrentReaders;
private final int maxOutputPartitionCount;
private final ExecutorService executor;

@Inject
Expand All @@ -77,12 +80,19 @@ public FileSystemExchangeManager(
this.exchangeSinkBuffersPerPartition = fileSystemExchangeConfig.getExchangeSinkBuffersPerPartition();
this.exchangeSinkMaxFileSizeInBytes = fileSystemExchangeConfig.getExchangeSinkMaxFileSize().toBytes();
this.exchangeSourceConcurrentReaders = fileSystemExchangeConfig.getExchangeSourceConcurrentReaders();
this.maxOutputPartitionCount = fileSystemExchangeConfig.getMaxOutputPartitionCount();
this.executor = newCachedThreadPool(daemonThreadsNamed("exchange-source-handles-creation-%s"));
}

@Override
public Exchange createExchange(ExchangeContext context, int outputPartitionCount)
{
if (outputPartitionCount > maxOutputPartitionCount) {
throw new TrinoException(
MAX_OUTPUT_PARTITION_COUNT_EXCEEDED,
format("Max number of output partitions exceeded for exchange '%s'. Allowed: %s. Requested: %s.", context.getExchangeId(), maxOutputPartitionCount, outputPartitionCount));
}

Optional<SecretKey> secretKey = Optional.empty();
if (exchangeEncryptionEnabled) {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,9 +41,11 @@
import static io.airlift.units.DataSize.Unit.BYTE;
import static io.airlift.units.DataSize.Unit.KILOBYTE;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static io.trino.plugin.exchange.filesystem.FileSystemExchangeErrorCode.MAX_OUTPUT_PARTITION_COUNT_EXCEEDED;
import static io.trino.spi.exchange.ExchangeId.createRandomExchangeId;
import static java.lang.Math.toIntExact;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

public abstract class AbstractTestExchangeManager
{
Expand Down Expand Up @@ -225,6 +227,14 @@ public void testLargePages()
exchange.close();
}

@Test
public void testMaxOutputPartitionCountCheck()
{
assertThatThrownBy(() -> exchangeManager.createExchange(new ExchangeContext(new QueryId("query"), createRandomExchangeId()), 51))
.hasMessageContaining("Max number of output partitions exceeded for exchange")
.hasFieldOrPropertyWithValue("errorCode", MAX_OUTPUT_PARTITION_COUNT_EXCEEDED.toErrorCode());
}

private void writeData(ExchangeSinkInstanceHandle handle, Multimap<Integer, String> data, boolean finish)
{
ExchangeSink sink = exchangeManager.createSink(handle, false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@ public void testDefaults()
.setExchangeSinkBufferPoolMinSize(10)
.setExchangeSinkBuffersPerPartition(2)
.setExchangeSinkMaxFileSize(DataSize.of(1, GIGABYTE))
.setExchangeSourceConcurrentReaders(4));
.setExchangeSourceConcurrentReaders(4)
.setMaxOutputPartitionCount(50));
}

@Test
Expand All @@ -51,6 +52,7 @@ public void testExplicitPropertyMappings()
.put("exchange.sink-buffers-per-partition", "3")
.put("exchange.sink-max-file-size", "2GB")
.put("exchange.source-concurrent-readers", "10")
.put("exchange.max-output-partition-count", "53")
.buildOrThrow();

FileSystemExchangeConfig expected = new FileSystemExchangeConfig()
Expand All @@ -60,7 +62,8 @@ public void testExplicitPropertyMappings()
.setExchangeSinkBufferPoolMinSize(20)
.setExchangeSinkBuffersPerPartition(3)
.setExchangeSinkMaxFileSize(DataSize.of(2, GIGABYTE))
.setExchangeSourceConcurrentReaders(10);
.setExchangeSourceConcurrentReaders(10)
.setMaxOutputPartitionCount(53);

assertFullMapping(properties, expected);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,13 @@
*/
package io.trino.plugin.hive;

import io.trino.Session;
import io.trino.plugin.exchange.filesystem.containers.MinioStorage;
import io.trino.testing.QueryRunner;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Test;

import static io.trino.SystemSessionProperties.HASH_PARTITION_COUNT;
import static io.trino.plugin.exchange.filesystem.containers.MinioStorage.getExchangeManagerProperties;
import static io.trino.testing.FaultTolerantExecutionConnectorTestHelper.getExtraProperties;
import static io.trino.testing.sql.TestTable.randomTableSuffix;
Expand Down Expand Up @@ -103,6 +106,15 @@ public void testOptimizeHiveSystemTable()
.hasMessageContaining("This connector does not support query retries");
}

@Test
public void testMaxOutputPartitionCountCheck()
{
Session session = Session.builder(getSession())
.setSystemProperty(HASH_PARTITION_COUNT, "51")
.build();
assertQueryFails(session, "SELECT nationkey, count(*) FROM nation GROUP BY nationkey", "Max number of output partitions exceeded for exchange.*");
}

@AfterClass(alwaysRun = true)
public void destroy()
throws Exception
Expand Down