mirror of https://github.com/apache/druid.git
Improve doc and exception message for invalid user configurations (#10598)
* improve doc and exception message * add spelling check rules and remove unused import * add a test to improve test coverage
This commit is contained in:
parent
31740b3b29
commit
fe693a4f01
|
@ -46,7 +46,7 @@ public class HumanReadableBytes
|
|||
public int getBytesInInt()
|
||||
{
|
||||
if (bytes > Integer.MAX_VALUE) {
|
||||
throw new ISE("Number overflow");
|
||||
throw new ISE("Number [%d] exceeds range of Integer.MAX_VALUE", bytes);
|
||||
}
|
||||
|
||||
return (int) bytes;
|
||||
|
|
|
@ -64,7 +64,7 @@ public @interface HumanReadableBytesRange
|
|||
Class<?>[] groups() default {};
|
||||
|
||||
//ConstraintValidator requires
|
||||
String message() default "value is out of range";
|
||||
String message() default "value must be in the range of [{min}, {max}]";
|
||||
|
||||
//ConstraintValidator requires
|
||||
Class<? extends Payload>[] payload() default {};
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.java.util.common;
|
|||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.utils.CollectionUtils;
|
||||
import org.hamcrest.Description;
|
||||
import org.hamcrest.Matcher;
|
||||
import org.junit.Assert;
|
||||
|
@ -359,6 +360,14 @@ public class HumanReadableBytesTest
|
|||
Assert.assertEquals(bytes, deserialized);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetInt()
|
||||
{
|
||||
expectedException.expectMessage("Number [2147483648] exceeds range of Integer.MAX_VALUE");
|
||||
HumanReadableBytes bytes = new HumanReadableBytes("2GiB");
|
||||
bytes.getBytesInInt();
|
||||
}
|
||||
|
||||
static class TestBytesRange
|
||||
{
|
||||
@HumanReadableBytesRange(min = 0, max = 5)
|
||||
|
@ -373,26 +382,26 @@ public class HumanReadableBytesTest
|
|||
@Test
|
||||
public void testBytesRange()
|
||||
{
|
||||
long errorCount = validate(new TestBytesRange(HumanReadableBytes.valueOf(-1)));
|
||||
Assert.assertEquals(1, errorCount);
|
||||
String message = validate(new TestBytesRange(HumanReadableBytes.valueOf(-1)));
|
||||
Assert.assertEquals("value must be in the range of [0, 5]", message);
|
||||
|
||||
errorCount = validate(new TestBytesRange(HumanReadableBytes.valueOf(0)));
|
||||
Assert.assertEquals(0, errorCount);
|
||||
message = validate(new TestBytesRange(HumanReadableBytes.valueOf(0)));
|
||||
Assert.assertEquals(null, message);
|
||||
|
||||
errorCount = validate(new TestBytesRange(HumanReadableBytes.valueOf(5)));
|
||||
Assert.assertEquals(0, errorCount);
|
||||
message = validate(new TestBytesRange(HumanReadableBytes.valueOf(5)));
|
||||
Assert.assertEquals(null, message);
|
||||
|
||||
errorCount = validate(new TestBytesRange(HumanReadableBytes.valueOf(6)));
|
||||
Assert.assertEquals(1, errorCount);
|
||||
message = validate(new TestBytesRange(HumanReadableBytes.valueOf(6)));
|
||||
Assert.assertEquals("value must be in the range of [0, 5]", message);
|
||||
}
|
||||
|
||||
private static <T> long validate(T obj)
|
||||
private static <T> String validate(T obj)
|
||||
{
|
||||
Validator validator = Validation.buildDefaultValidatorFactory()
|
||||
.getValidator();
|
||||
|
||||
Map<String, StringBuilder> errorMap = new HashMap<>();
|
||||
Set<ConstraintViolation<T>> set = validator.validate(obj, Default.class);
|
||||
return set == null ? 0 : set.size();
|
||||
return CollectionUtils.isNullOrEmpty(set) ? null : set.stream().findFirst().get().getMessage();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -900,7 +900,7 @@ The following configs only apply if the Overlord is running in remote mode. For
|
|||
|`druid.indexer.runner.taskAssignmentTimeout`|How long to wait after a task as been assigned to a MiddleManager before throwing an error.|PT5M|
|
||||
|`druid.indexer.runner.minWorkerVersion`|The minimum MiddleManager version to send tasks to. |"0"|
|
||||
|`druid.indexer.runner.compressZnodes`|Indicates whether or not the Overlord should expect MiddleManagers to compress Znodes.|true|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper. Human-readable format is supported, see [here](human-readable-byte.md). | 512 KiB |
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper, should be in the range of [10KiB, 2GiB). [Human-readable format](human-readable-byte.md) is supported.| 512 KiB |
|
||||
|`druid.indexer.runner.taskCleanupTimeout`|How long to wait before failing a task after a MiddleManager is disconnected from Zookeeper.|PT15M|
|
||||
|`druid.indexer.runner.taskShutdownLinkTimeout`|How long to wait on a shutdown request to a MiddleManager before timing out|PT1M|
|
||||
|`druid.indexer.runner.pendingTasksRunnerNumThreads`|Number of threads to allocate pending-tasks to workers, must be at least 1.|1|
|
||||
|
@ -1183,7 +1183,7 @@ Middle managers pass their configurations down to their child peons. The MiddleM
|
|||
|`druid.indexer.runner.javaCommand`|Command required to execute java.|java|
|
||||
|`druid.indexer.runner.javaOpts`|*DEPRECATED* A string of -X Java options to pass to the peon's JVM. Quotable parameters or parameters with spaces are encouraged to use javaOptsArray|""|
|
||||
|`druid.indexer.runner.javaOptsArray`|A JSON array of strings to be passed in as options to the peon's JVM. This is additive to javaOpts and is recommended for properly handling arguments which contain quotes or spaces like `["-XX:OnOutOfMemoryError=kill -9 %p"]`|`[]`|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper.|524288|
|
||||
|`druid.indexer.runner.maxZnodeBytes`|The maximum size Znode in bytes that can be created in Zookeeper, should be in the range of [10KiB, 2GiB). [Human-readable format](human-readable-byte.md) is supported.|512KiB|
|
||||
|`druid.indexer.runner.startPort`|Starting port used for peon processes, should be greater than 1023 and less than 65536.|8100|
|
||||
|`druid.indexer.runner.endPort`|Ending port used for peon processes, should be greater than or equal to `druid.indexer.runner.startPort` and less than 65536.|65535|
|
||||
|`druid.indexer.runner.ports`|A JSON array of integers to specify ports that used for peon processes. If provided and non-empty, ports for peon processes will be chosen from these ports. And `druid.indexer.runner.startPort/druid.indexer.runner.endPort` will be completely ignored.|`[]`|
|
||||
|
@ -1198,7 +1198,7 @@ Processing properties set on the Middlemanager will be passed through to Peons.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. Human-readable format is supported, see [here](human-readable-byte.md). |auto (max 1 GiB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1 GiB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|
@ -1339,7 +1339,7 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in the Indexer processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in the Indexer processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Indexer processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|
@ -1454,7 +1454,7 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB), for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|
@ -1617,7 +1617,7 @@ The broker uses processing configs for nested groupBy queries.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. It can be configured as described in [here](human-readable-byte.md). |auto (max 1GiB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|
||||
|
|
|
@ -44,7 +44,10 @@ public class RemoteTaskRunnerConfig extends WorkerTaskRunnerConfig
|
|||
private Period taskCleanupTimeout = new Period("PT15M");
|
||||
|
||||
@JsonProperty
|
||||
@HumanReadableBytesRange(min = 10 * 1024, max = Integer.MAX_VALUE)
|
||||
@HumanReadableBytesRange(min = 10 * 1024,
|
||||
max = Integer.MAX_VALUE,
|
||||
message = "maxZnodeBytes must be in the range of [10KiB, 2GiB)"
|
||||
)
|
||||
private HumanReadableBytes maxZnodeBytes = HumanReadableBytes.valueOf(CuratorUtils.DEFAULT_MAX_ZNODE_BYTES);
|
||||
|
||||
@JsonProperty
|
||||
|
@ -170,7 +173,7 @@ public class RemoteTaskRunnerConfig extends WorkerTaskRunnerConfig
|
|||
int result = taskAssignmentTimeout.hashCode();
|
||||
result = 31 * result + taskCleanupTimeout.hashCode();
|
||||
result = 31 * result + getMinWorkerVersion().hashCode();
|
||||
result = 31 * result + maxZnodeBytes.getBytesInInt();
|
||||
result = 31 * result + maxZnodeBytes.hashCode();
|
||||
result = 31 * result + taskShutdownLinkTimeout.hashCode();
|
||||
result = 31 * result + pendingTasksRunnerNumThreads;
|
||||
result = 31 * result + maxRetriesBeforeBlacklist;
|
||||
|
|
|
@ -20,18 +20,30 @@
|
|||
package org.apache.druid.indexing.overlord.config;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.ProvisionException;
|
||||
import org.apache.druid.guice.GuiceInjectors;
|
||||
import org.apache.druid.guice.IndexingServiceModuleHelper;
|
||||
import org.apache.druid.guice.JsonConfigProvider;
|
||||
import org.apache.druid.guice.JsonConfigurator;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
public class RemoteTaskRunnerConfigTest
|
||||
{
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
private static final ObjectMapper MAPPER = new DefaultObjectMapper();
|
||||
private static final Period DEFAULT_TIMEOUT = Period.ZERO;
|
||||
private static final String DEFAULT_VERSION = "";
|
||||
|
@ -793,6 +805,49 @@ public class RemoteTaskRunnerConfigTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxZnodeBytesLowerThanExpected()
|
||||
{
|
||||
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of(
|
||||
binder -> IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder))
|
||||
);
|
||||
|
||||
this.expectedException.expect(ProvisionException.class);
|
||||
this.expectedException.expectMessage("maxZnodeBytes must be in the range of [10KiB, 2GiB)");
|
||||
|
||||
Properties props = new Properties();
|
||||
props.put(IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".maxZnodeBytes", "9KiB");
|
||||
|
||||
JsonConfigProvider<RemoteTaskRunnerConfig> configProvider = JsonConfigProvider.of(
|
||||
IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX,
|
||||
RemoteTaskRunnerConfig.class
|
||||
);
|
||||
configProvider.inject(props, injector.getBinding(JsonConfigurator.class).getProvider().get());
|
||||
configProvider.get().get();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxZnodeBytesGreaterThanExpected()
|
||||
{
|
||||
final Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of(
|
||||
binder -> IndexingServiceModuleHelper.configureTaskRunnerConfigs(binder))
|
||||
);
|
||||
|
||||
this.expectedException.expect(ProvisionException.class);
|
||||
this.expectedException.expectMessage("maxZnodeBytes must be in the range of [10KiB, 2GiB)");
|
||||
|
||||
Properties props = new Properties();
|
||||
props.put(IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX + ".maxZnodeBytes", "2GiB");
|
||||
|
||||
JsonConfigProvider<RemoteTaskRunnerConfig> configProvider = JsonConfigProvider.of(
|
||||
IndexingServiceModuleHelper.INDEXER_RUNNER_PROPERTY_PREFIX,
|
||||
RemoteTaskRunnerConfig.class
|
||||
);
|
||||
configProvider.inject(props, injector.getBinding(JsonConfigurator.class).getProvider().get());
|
||||
configProvider.get().get();
|
||||
}
|
||||
|
||||
|
||||
private RemoteTaskRunnerConfig reflect(RemoteTaskRunnerConfig config) throws IOException
|
||||
{
|
||||
return MAPPER.readValue(MAPPER.writeValueAsString(config), RemoteTaskRunnerConfig.class);
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.query;
|
||||
|
||||
import org.apache.druid.java.util.common.HumanReadableBytes;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.concurrent.ExecutorServiceConfig;
|
||||
import org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
@ -34,7 +35,7 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem
|
|||
private static final Logger log = new Logger(DruidProcessingConfig.class);
|
||||
|
||||
public static final int DEFAULT_NUM_MERGE_BUFFERS = -1;
|
||||
public static final int DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = -1;
|
||||
public static final HumanReadableBytes DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = HumanReadableBytes.valueOf(-1);
|
||||
public static final int MAX_DEFAULT_PROCESSING_BUFFER_SIZE_BYTES = 1024 * 1024 * 1024;
|
||||
public static final int DEFAULT_MERGE_POOL_AWAIT_SHUTDOWN_MILLIS = 60_000;
|
||||
|
||||
|
@ -43,14 +44,17 @@ public abstract class DruidProcessingConfig extends ExecutorServiceConfig implem
|
|||
@Config({"druid.computation.buffer.size", "${base_path}.buffer.sizeBytes"})
|
||||
public HumanReadableBytes intermediateComputeSizeBytesConfigured()
|
||||
{
|
||||
return HumanReadableBytes.valueOf(DEFAULT_PROCESSING_BUFFER_SIZE_BYTES);
|
||||
return DEFAULT_PROCESSING_BUFFER_SIZE_BYTES;
|
||||
}
|
||||
|
||||
public int intermediateComputeSizeBytes()
|
||||
{
|
||||
int sizeBytesConfigured = intermediateComputeSizeBytesConfigured().getBytesInInt();
|
||||
if (sizeBytesConfigured != DEFAULT_PROCESSING_BUFFER_SIZE_BYTES) {
|
||||
return sizeBytesConfigured;
|
||||
HumanReadableBytes sizeBytesConfigured = intermediateComputeSizeBytesConfigured();
|
||||
if (!DEFAULT_PROCESSING_BUFFER_SIZE_BYTES.equals(sizeBytesConfigured)) {
|
||||
if (sizeBytesConfigured.getBytes() > Integer.MAX_VALUE) {
|
||||
throw new IAE("druid.processing.buffer.sizeBytes must be less than 2GiB");
|
||||
}
|
||||
return sizeBytesConfigured.getBytesInInt();
|
||||
} else if (computedBufferSizeBytes.get() != null) {
|
||||
return computedBufferSizeBytes.get();
|
||||
}
|
||||
|
|
|
@ -22,11 +22,15 @@ package org.apache.druid.query;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Guice;
|
||||
import com.google.inject.Injector;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.config.Config;
|
||||
import org.apache.druid.utils.JvmUtils;
|
||||
import org.apache.druid.utils.RuntimeInfo;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
||||
import java.util.Map;
|
||||
|
@ -41,6 +45,9 @@ public class DruidProcessingConfigTest
|
|||
private static final long DIRECT_SIZE = BUFFER_SIZE * (3L + 2L + 1L);
|
||||
private static final long HEAP_SIZE = BUFFER_SIZE * 2L;
|
||||
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
private static Injector makeInjector(int numProcessors, long directMemorySize, long heapSize)
|
||||
{
|
||||
return makeInjector(numProcessors, directMemorySize, heapSize, new Properties(), null);
|
||||
|
@ -146,6 +153,41 @@ public class DruidProcessingConfigTest
|
|||
Assert.assertEquals("/test/path", config.getTmpDir());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidSizeBytes()
|
||||
{
|
||||
Properties props = new Properties();
|
||||
props.setProperty("druid.processing.buffer.sizeBytes", "-1");
|
||||
|
||||
expectedException.expectCause(CoreMatchers.isA(IAE.class));
|
||||
|
||||
Injector injector = makeInjector(
|
||||
NUM_PROCESSORS,
|
||||
DIRECT_SIZE,
|
||||
HEAP_SIZE,
|
||||
props,
|
||||
ImmutableMap.of("base_path", "druid.processing")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSizeBytesUpperLimit()
|
||||
{
|
||||
Properties props = new Properties();
|
||||
props.setProperty("druid.processing.buffer.sizeBytes", "2GiB");
|
||||
Injector injector = makeInjector(
|
||||
NUM_PROCESSORS,
|
||||
DIRECT_SIZE,
|
||||
HEAP_SIZE,
|
||||
props,
|
||||
ImmutableMap.of("base_path", "druid.processing")
|
||||
);
|
||||
DruidProcessingConfig config = injector.getInstance(DruidProcessingConfig.class);
|
||||
|
||||
expectedException.expectMessage("druid.processing.buffer.sizeBytes must be less than 2GiB");
|
||||
config.intermediateComputeSizeBytes();
|
||||
}
|
||||
|
||||
static class MockRuntimeInfo extends RuntimeInfo
|
||||
{
|
||||
private final int availableProcessors;
|
||||
|
|
|
@ -1580,6 +1580,9 @@ dsql
|
|||
- ../docs/tutorials/tutorial-update-data.md
|
||||
bear-111
|
||||
- ../docs/configuration/index.md
|
||||
10KiB
|
||||
2GiB
|
||||
512KiB
|
||||
1GiB
|
||||
KiB
|
||||
GiB
|
||||
|
|
Loading…
Reference in New Issue