diff --git a/docs/src/main/sphinx/connector/delta-lake.md b/docs/src/main/sphinx/connector/delta-lake.md index 0a7b27e31d38..736489d6fa95 100644 --- a/docs/src/main/sphinx/connector/delta-lake.md +++ b/docs/src/main/sphinx/connector/delta-lake.md @@ -1162,8 +1162,7 @@ keep a backup of the original values if you change them. - `Integer.MAX_VALUE` * - `delta.max-split-size` - Sets the largest [](prop-type-data-size) for a single read section - assigned to a worker after `max-initial-splits` have been processed. You can - also use the corresponding catalog session property + assigned to a worker. You can also use the corresponding catalog session property `.max_split_size`. - `64MB` * - `delta.minimum-assigned-split-weight` diff --git a/docs/src/main/sphinx/connector/hive.md b/docs/src/main/sphinx/connector/hive.md index 013c674f1082..2496c1343e49 100644 --- a/docs/src/main/sphinx/connector/hive.md +++ b/docs/src/main/sphinx/connector/hive.md @@ -1293,16 +1293,6 @@ cause instability and performance degradation. be used to reduce the load on the storage system. By default, there is no limit, which results in Trino maximizing the parallelization of data access. - -* - `hive.max-initial-splits` - - For each table scan, the coordinator first assigns file sections of up to - `max-initial-split-size`. After `max-initial-splits` have been assigned, - `max-split-size` is used for the remaining splits. - - `200` -* - `hive.max-initial-split-size` - - The size of a single file section assigned to a worker until - `max-initial-splits` have been assigned. Smaller splits results in more - parallelism, which gives a boost to smaller queries. - - `32 MB` * - `hive.max-split-size` - The largest size of a single file section assigned to a worker. Smaller splits result in more parallelism and thus can decrease latency, but diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index 99e32b168a29..5555531aed68 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -98,7 +98,7 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE; import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; -import static io.trino.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize; +import static io.trino.plugin.hive.HiveSessionProperties.getMaxSplitSize; import static io.trino.plugin.hive.HiveSessionProperties.isForceLocalScheduling; import static io.trino.plugin.hive.HiveSessionProperties.isValidateBucketing; import static io.trino.plugin.hive.HiveStorageFormat.TEXTFILE; @@ -427,7 +427,7 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) partition.getHiveColumnCoercions(), Optional.empty(), Optional.empty(), - getMaxInitialSplitSize(session), + getMaxSplitSize(session), isForceLocalScheduling(session), maxSplitFileSize); @@ -479,7 +479,7 @@ private ListenableFuture loadPartition(HivePartitionMetadata partition) partition.getHiveColumnCoercions(), bucketConversionRequiresWorkerParticipation ? bucketConversion : Optional.empty(), bucketValidation, - getMaxInitialSplitSize(session), + getMaxSplitSize(session), isForceLocalScheduling(session), maxSplitFileSize); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java index 5bb13734d3d4..c4eec83761a5 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveConfig.java @@ -63,6 +63,8 @@ "hive.s3select-pushdown.enabled", "hive.s3select-pushdown.experimental-textfile-pushdown-enabled", "hive.s3select-pushdown.max-connections", + "hive.max-initial-splits", + "hive.max-initial-split-size" }) public class HiveConfig { @@ -78,10 +80,8 @@ public class HiveConfig private int maxSplitIteratorThreads = 1_000; private int minPartitionBatchSize = 10; private int maxPartitionBatchSize = 100; - private int maxInitialSplits = 200; private int splitLoaderConcurrency = 64; private Integer maxSplitsPerSecond; - private DataSize maxInitialSplitSize; private int domainCompactionThreshold = 1000; private boolean forceLocalScheduling; private boolean recursiveDirWalkerEnabled; @@ -189,33 +189,6 @@ public HiveConfig setSingleStatementWritesOnly(boolean singleStatementWritesOnly return this; } - public int getMaxInitialSplits() - { - return maxInitialSplits; - } - - @Config("hive.max-initial-splits") - public HiveConfig setMaxInitialSplits(int maxInitialSplits) - { - this.maxInitialSplits = maxInitialSplits; - return this; - } - - public DataSize getMaxInitialSplitSize() - { - if (maxInitialSplitSize == null) { - return DataSize.ofBytes(maxSplitSize.toBytes() / 2).to(maxSplitSize.getUnit()); - } - return maxInitialSplitSize; - } - - @Config("hive.max-initial-split-size") - public HiveConfig setMaxInitialSplitSize(DataSize maxInitialSplitSize) - { - this.maxInitialSplitSize = maxInitialSplitSize; - return this; - } - @Min(1) public int getSplitLoaderConcurrency() { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java index f2375f946309..68e90ffe021c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSessionProperties.java @@ -104,7 +104,6 @@ public final class HiveSessionProperties private static final String PARQUET_WRITER_BATCH_SIZE = "parquet_writer_batch_size"; private static final String PARQUET_OPTIMIZED_WRITER_VALIDATION_PERCENTAGE = "parquet_optimized_writer_validation_percentage"; private static final String MAX_SPLIT_SIZE = "max_split_size"; - private static final String MAX_INITIAL_SPLIT_SIZE = "max_initial_split_size"; private static final String RCFILE_OPTIMIZED_WRITER_VALIDATE = "rcfile_optimized_writer_validate"; private static final String SORTED_WRITING_ENABLED = "sorted_writing_enabled"; private static final String PROPAGATE_TABLE_SCAN_SORTING_PROPERTIES = "propagate_table_scan_sorting_properties"; @@ -420,11 +419,6 @@ public HiveSessionProperties( "Max split size", hiveConfig.getMaxSplitSize(), true), - dataSizeProperty( - MAX_INITIAL_SPLIT_SIZE, - "Max initial split size", - hiveConfig.getMaxInitialSplitSize(), - true), booleanProperty( RCFILE_OPTIMIZED_WRITER_VALIDATE, "RCFile: Validate writer files", @@ -786,11 +780,6 @@ public static DataSize getMaxSplitSize(ConnectorSession session) return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); } - public static DataSize getMaxInitialSplitSize(ConnectorSession session) - { - return session.getProperty(MAX_INITIAL_SPLIT_SIZE, DataSize.class); - } - public static boolean isRcfileOptimizedWriterValidate(ConnectorSession session) { return session.getProperty(RCFILE_OPTIMIZED_WRITER_VALIDATE, Boolean.class); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java index 5c3ef5d2cc31..a7f0023fbac9 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java @@ -113,7 +113,6 @@ public class HiveSplitManager private final DataSize maxOutstandingSplitsSize; private final int minPartitionBatchSize; private final int maxPartitionBatchSize; - private final int maxInitialSplits; private final int splitLoaderConcurrency; private final int maxSplitsPerSecond; private final boolean recursiveDfsWalkerEnabled; @@ -143,7 +142,6 @@ public HiveSplitManager( hiveConfig.getMaxOutstandingSplitsSize(), hiveConfig.getMinPartitionBatchSize(), hiveConfig.getMaxPartitionBatchSize(), - hiveConfig.getMaxInitialSplits(), hiveConfig.getSplitLoaderConcurrency(), hiveConfig.getMaxSplitsPerSecond(), hiveConfig.getRecursiveDirWalkerEnabled(), @@ -162,7 +160,6 @@ public HiveSplitManager( DataSize maxOutstandingSplitsSize, int minPartitionBatchSize, int maxPartitionBatchSize, - int maxInitialSplits, int splitLoaderConcurrency, @Nullable Integer maxSplitsPerSecond, boolean recursiveDfsWalkerEnabled, @@ -180,7 +177,6 @@ public HiveSplitManager( this.maxOutstandingSplitsSize = maxOutstandingSplitsSize; this.minPartitionBatchSize = minPartitionBatchSize; this.maxPartitionBatchSize = maxPartitionBatchSize; - this.maxInitialSplits = maxInitialSplits; this.splitLoaderConcurrency = splitLoaderConcurrency; this.maxSplitsPerSecond = firstNonNull(maxSplitsPerSecond, Integer.MAX_VALUE); this.recursiveDfsWalkerEnabled = recursiveDfsWalkerEnabled; @@ -278,7 +274,6 @@ public ConnectorSplitSource getSplits( session, table.getDatabaseName(), table.getTableName(), - maxInitialSplits, maxOutstandingSplits, maxOutstandingSplitsSize, maxSplitsPerSecond, diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java index 79cfa783be39..b483a9436278 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java @@ -50,7 +50,6 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_EXCEEDED_SPLIT_BUFFERING_LIMIT; import static io.trino.plugin.hive.HiveErrorCode.HIVE_FILE_NOT_FOUND; import static io.trino.plugin.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; -import static io.trino.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize; import static io.trino.plugin.hive.HiveSessionProperties.getMaxSplitSize; import static io.trino.plugin.hive.HiveSessionProperties.getMinimumAssignedSplitWeight; import static io.trino.plugin.hive.HiveSessionProperties.isSizeBasedSplitWeightsEnabled; @@ -73,9 +72,7 @@ class HiveSplitSource private final AtomicInteger bufferedInternalSplitCount = new AtomicInteger(); private final long maxOutstandingSplitsBytes; - private final DataSize maxSplitSize; - private final DataSize maxInitialSplitSize; - private final AtomicInteger remainingInitialSplits; + private final long maxSplitBytes; private final HiveSplitLoader splitLoader; private final AtomicReference stateReference; @@ -95,7 +92,6 @@ private HiveSplitSource( String databaseName, String tableName, PerBucket queues, - int maxInitialSplits, DataSize maxOutstandingSplitsSize, HiveSplitLoader splitLoader, AtomicReference stateReference, @@ -113,10 +109,8 @@ private HiveSplitSource( this.stateReference = requireNonNull(stateReference, "stateReference is null"); this.highMemorySplitSourceCounter = requireNonNull(highMemorySplitSourceCounter, "highMemorySplitSourceCounter is null"); - this.maxSplitSize = getMaxSplitSize(session); - this.maxInitialSplitSize = getMaxInitialSplitSize(session); - this.remainingInitialSplits = new AtomicInteger(maxInitialSplits); - this.splitWeightProvider = isSizeBasedSplitWeightsEnabled(session) ? new SizeBasedSplitWeightProvider(getMinimumAssignedSplitWeight(session), maxSplitSize) : HiveSplitWeightProvider.uniformStandardWeightProvider(); + this.maxSplitBytes = getMaxSplitSize(session).toBytes(); + this.splitWeightProvider = isSizeBasedSplitWeightsEnabled(session) ? new SizeBasedSplitWeightProvider(getMinimumAssignedSplitWeight(session), getMaxSplitSize(session)) : HiveSplitWeightProvider.uniformStandardWeightProvider(); this.cachingHostAddressProvider = requireNonNull(cachingHostAddressProvider, "cachingHostAddressProvider is null"); this.recordScannedFiles = recordScannedFiles; } @@ -125,7 +119,6 @@ public static HiveSplitSource allAtOnce( ConnectorSession session, String databaseName, String tableName, - int maxInitialSplits, int maxOutstandingSplits, DataSize maxOutstandingSplitsSize, int maxSplitsPerSecond, @@ -168,7 +161,6 @@ public boolean isFinished() return queue.isFinished(); } }, - maxInitialSplits, maxOutstandingSplitsSize, splitLoader, stateReference, @@ -277,12 +269,6 @@ public CompletableFuture getNextBatch(int maxSize) continue; } - long maxSplitBytes = maxSplitSize.toBytes(); - if (remainingInitialSplits.get() > 0) { - if (remainingInitialSplits.getAndDecrement() > 0) { - maxSplitBytes = maxInitialSplitSize.toBytes(); - } - } InternalHiveBlock block = internalSplit.currentBlock(); long splitBytes; if (internalSplit.isSplittable()) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java index 987e4260fcbc..252c09aefe1c 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/util/InternalHiveSplitFactory.java @@ -58,7 +58,7 @@ public class InternalHiveSplitFactory private final BooleanSupplier partitionMatchSupplier; private final Optional bucketConversion; private final Optional bucketValidation; - private final long minimumTargetSplitSizeInBytes; + private final long maxSplitSizeInBytes; private final Optional maxSplitFileSize; private final boolean forceLocalScheduling; @@ -72,7 +72,7 @@ public InternalHiveSplitFactory( Map hiveColumnCoercions, Optional bucketConversion, Optional bucketValidation, - DataSize minimumTargetSplitSize, + DataSize maxSplitSize, boolean forceLocalScheduling, Optional maxSplitFileSize) { @@ -86,9 +86,9 @@ public InternalHiveSplitFactory( this.bucketConversion = requireNonNull(bucketConversion, "bucketConversion is null"); this.bucketValidation = requireNonNull(bucketValidation, "bucketValidation is null"); this.forceLocalScheduling = forceLocalScheduling; - this.minimumTargetSplitSizeInBytes = minimumTargetSplitSize.toBytes(); + this.maxSplitSizeInBytes = maxSplitSize.toBytes(); this.maxSplitFileSize = requireNonNull(maxSplitFileSize, "maxSplitFileSize is null"); - checkArgument(minimumTargetSplitSizeInBytes > 0, "minimumTargetSplitSize must be > 0, found: %s", minimumTargetSplitSize); + checkArgument(this.maxSplitSizeInBytes > 0, "maxTargetSplitSize must be > 0, found: %s", maxSplitSize); } private static Map stripUnnecessaryProperties(Map schema) @@ -108,7 +108,11 @@ public String getPartitionName() public Optional createInternalHiveSplit(TrinoFileStatus status, OptionalInt readBucketNumber, OptionalInt tableBucketNumber, boolean splittable, Optional acidInfo) { splittable = splittable && - status.getLength() > minimumTargetSplitSizeInBytes && + // For some input formats, the isSplittable check is non-trivial and can + // add a significant amount of time to split generation when handling a + // large number of very small files. If file is smaller or equal to the + // maxSplitSizeInBytes, then this check can be skipped. + status.getLength() > maxSplitSizeInBytes && storageFormat.isSplittable(status.getPath()); return createInternalHiveSplit( status.getPath(), diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java index a80c3ed9c239..731a0f201c95 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/HiveQueryRunner.java @@ -241,7 +241,6 @@ public DistributedQueryRunner build() if (tpchBucketedCatalogEnabled) { Map hiveBucketedProperties = ImmutableMap.builder() .putAll(hiveProperties) - .put("hive.max-initial-split-size", "10kB") // so that each bucket has multiple splits .put("hive.max-split-size", "10kB") // so that each bucket has multiple splits .put("hive.storage-format", "TEXTFILE") // so that there's no minimum split size for the file .buildOrThrow(); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java index 51dac62efeef..fd95997a53dd 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java @@ -167,8 +167,8 @@ public void testCsv() throws Exception { FileEntry file = new FileEntry(LOCATION, DataSize.of(2, GIGABYTE).toBytes(), Instant.now(), Optional.empty()); - assertCsvSplitCount(file, Map.of(), 33); - assertCsvSplitCount(file, Map.of(HEADER_COUNT, "1"), 33); + assertCsvSplitCount(file, Map.of(), 32); + assertCsvSplitCount(file, Map.of(HEADER_COUNT, "1"), 32); assertCsvSplitCount(file, Map.of(HEADER_COUNT, "2"), 1); assertCsvSplitCount(file, Map.of(FOOTER_COUNT, "1"), 1); assertCsvSplitCount(file, Map.of(HEADER_COUNT, "1", FOOTER_COUNT, "1"), 1); @@ -517,7 +517,7 @@ public void testMultipleSplitsPerBucket() HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); backgroundHiveSplitLoader.start(hiveSplitSource); - assertThat(drainSplits(hiveSplitSource).size()).isEqualTo(17); + assertThat(drainSplits(hiveSplitSource).size()).isEqualTo(16); } @Test @@ -1280,7 +1280,6 @@ private HiveSplitSource hiveSplitSource(HiveSplitLoader hiveSplitLoader) SIMPLE_TABLE.getDatabaseName(), SIMPLE_TABLE.getTableName(), 1, - 1, DataSize.of(32, MEGABYTE), Integer.MAX_VALUE, hiveSplitLoader, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java index 670d3ac0e259..01e905dd1aac 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveConfig.java @@ -50,8 +50,6 @@ public void testDefaults() .setPerTransactionMetastoreCacheMaximumSize(1000) .setMinPartitionBatchSize(10) .setMaxPartitionBatchSize(100) - .setMaxInitialSplits(200) - .setMaxInitialSplitSize(DataSize.of(32, Unit.MEGABYTE)) .setSplitLoaderConcurrency(64) .setMaxSplitsPerSecond(null) .setDomainCompactionThreshold(1000) @@ -135,8 +133,6 @@ public void testExplicitPropertyMappings() .put("hive.per-transaction-metastore-cache-maximum-size", "500") .put("hive.metastore.partition-batch-size.min", "1") .put("hive.metastore.partition-batch-size.max", "1000") - .put("hive.max-initial-splits", "10") - .put("hive.max-initial-split-size", "16MB") .put("hive.split-loader-concurrency", "1") .put("hive.max-splits-per-second", "1") .put("hive.domain-compaction-threshold", "42") @@ -217,8 +213,6 @@ public void testExplicitPropertyMappings() .setPerTransactionMetastoreCacheMaximumSize(500) .setMinPartitionBatchSize(1) .setMaxPartitionBatchSize(1000) - .setMaxInitialSplits(10) - .setMaxInitialSplitSize(DataSize.of(16, Unit.MEGABYTE)) .setSplitLoaderConcurrency(1) .setMaxSplitsPerSecond(1) .setDomainCompactionThreshold(42) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java index 89a60c4cc08b..c2559522e250 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java @@ -35,7 +35,7 @@ import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.plugin.hive.HiveErrorCode.HIVE_EXCEEDED_SPLIT_BUFFERING_LIMIT; -import static io.trino.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize; +import static io.trino.plugin.hive.HiveSessionProperties.getMaxSplitSize; import static io.trino.plugin.hive.HiveTestUtils.SESSION; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.lang.Math.toIntExact; @@ -52,7 +52,6 @@ public void testOutstandingSplitCount() "database", "table", 10, - 10, DataSize.of(1, MEGABYTE), Integer.MAX_VALUE, new TestingHiveSplitLoader(), @@ -88,7 +87,6 @@ public void testDynamicPartitionPruning() "database", "table", 10, - 10, DataSize.of(1, MEGABYTE), Integer.MAX_VALUE, new TestingHiveSplitLoader(), @@ -110,13 +108,12 @@ public void testDynamicPartitionPruning() @Test public void testEvenlySizedSplitRemainder() { - DataSize initialSplitSize = getMaxInitialSplitSize(SESSION); + DataSize maxSplitSize = getMaxSplitSize(SESSION); HiveSplitSource hiveSplitSource = HiveSplitSource.allAtOnce( SESSION, "database", "table", 10, - 10, DataSize.of(1, MEGABYTE), Integer.MAX_VALUE, new TestingHiveSplitLoader(), @@ -125,8 +122,8 @@ public void testEvenlySizedSplitRemainder() new DefaultCachingHostAddressProvider(), false); - // One byte larger than the initial split max size - DataSize fileSize = DataSize.ofBytes(initialSplitSize.toBytes() + 1); + // One byte larger than the max split size + DataSize fileSize = DataSize.ofBytes(maxSplitSize.toBytes() + 1); long halfOfSize = fileSize.toBytes() / 2; hiveSplitSource.addToQueue(new TestSplit(1, OptionalInt.empty(), fileSize)); @@ -145,7 +142,6 @@ public void testFail() "database", "table", 10, - 10, DataSize.of(1, MEGABYTE), Integer.MAX_VALUE, new TestingHiveSplitLoader(), @@ -197,7 +193,6 @@ public void testReaderWaitsForSplits() "database", "table", 10, - 10, DataSize.of(1, MEGABYTE), Integer.MAX_VALUE, new TestingHiveSplitLoader(), @@ -252,7 +247,6 @@ public void testOutstandingSplitSize() SESSION, "database", "table", - 10, 10000, maxOutstandingSplitsSize, Integer.MAX_VALUE,