Skip to content

Commit

Permalink
Avoid checking isSplittable for files smaller than initial split size
Browse files Browse the repository at this point in the history
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. This change allows files smaller
than the max initial split size to avoid that check and considers them
unsplittable instead.
  • Loading branch information
pettyjamesm authored and electrum committed Jul 27, 2020
1 parent e6c960e commit 2cf1ca3
Show file tree
Hide file tree
Showing 3 changed files with 92 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@
import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_INVALID_PARTITION_VALUE;
import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR;
import static io.prestosql.plugin.hive.HivePartitionManager.partitionMatches;
import static io.prestosql.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize;
import static io.prestosql.plugin.hive.HiveSessionProperties.isForceLocalScheduling;
import static io.prestosql.plugin.hive.metastore.MetastoreUtil.getHiveSchema;
import static io.prestosql.plugin.hive.metastore.MetastoreUtil.getPartitionLocation;
Expand Down Expand Up @@ -375,6 +376,7 @@ private ListenableFuture<?> loadPartition(HivePartitionMetadata partition)
partitionMatchSupplier,
partition.getTableToPartitionMapping(),
Optional.empty(),
getMaxInitialSplitSize(session),
isForceLocalScheduling(session),
s3SelectPushdownEnabled);
lastResult = addSplitsToSource(targetSplits, splitFactory);
Expand Down Expand Up @@ -413,6 +415,7 @@ private ListenableFuture<?> loadPartition(HivePartitionMetadata partition)
partitionMatchSupplier,
partition.getTableToPartitionMapping(),
bucketConversionRequiresWorkerParticipation ? bucketConversion : Optional.empty(),
getMaxInitialSplitSize(session),
isForceLocalScheduling(session),
s3SelectPushdownEnabled);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package io.prestosql.plugin.hive.util;

import com.google.common.collect.ImmutableList;
import io.airlift.units.DataSize;
import io.prestosql.plugin.hive.AcidInfo;
import io.prestosql.plugin.hive.HiveColumnHandle;
import io.prestosql.plugin.hive.HivePartitionKey;
Expand Down Expand Up @@ -62,6 +63,7 @@ public class InternalHiveSplitFactory
private final TableToPartitionMapping tableToPartitionMapping;
private final BooleanSupplier partitionMatchSupplier;
private final Optional<BucketConversion> bucketConversion;
private final long minimumTargetSplitSizeInBytes;
private final boolean forceLocalScheduling;
private final boolean s3SelectPushdownEnabled;

Expand All @@ -75,6 +77,7 @@ public InternalHiveSplitFactory(
BooleanSupplier partitionMatchSupplier,
TableToPartitionMapping tableToPartitionMapping,
Optional<BucketConversion> bucketConversion,
DataSize minimumTargetSplitSize,
boolean forceLocalScheduling,
boolean s3SelectPushdownEnabled)
{
Expand All @@ -89,6 +92,8 @@ public InternalHiveSplitFactory(
this.bucketConversion = requireNonNull(bucketConversion, "bucketConversion is null");
this.forceLocalScheduling = forceLocalScheduling;
this.s3SelectPushdownEnabled = s3SelectPushdownEnabled;
this.minimumTargetSplitSizeInBytes = requireNonNull(minimumTargetSplitSize, "minimumTargetSplitSize is null").toBytes();
checkArgument(minimumTargetSplitSizeInBytes > 0, "minimumTargetSplitSize must be > 0, found: %s", minimumTargetSplitSize);
}

public String getPartitionName()
Expand All @@ -98,7 +103,9 @@ public String getPartitionName()

public Optional<InternalHiveSplit> createInternalHiveSplit(LocatedFileStatus status, OptionalInt bucketNumber, boolean splittable, Optional<AcidInfo> acidInfo)
{
splittable = splittable && isSplittable(inputFormat, fileSystem, status.getPath());
splittable = splittable &&
status.getLen() > minimumTargetSplitSizeInBytes &&
isSplittable(inputFormat, fileSystem, status.getPath());
return createInternalHiveSplit(
status.getPath(),
status.getBlockLocations(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
*/
package io.prestosql.plugin.hive;

import com.google.common.base.Throwables;
import com.google.common.collect.AbstractIterator;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ImmutableList;
Expand Down Expand Up @@ -46,6 +47,12 @@
import org.apache.hadoop.hive.common.ValidReaderWriteIdList;
import org.apache.hadoop.hive.common.ValidWriteIdList;
import org.apache.hadoop.hive.metastore.TableType;
import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.util.Progressable;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -82,6 +89,7 @@
import static io.prestosql.plugin.hive.HiveColumnHandle.createBaseColumn;
import static io.prestosql.plugin.hive.HiveColumnHandle.pathColumnHandle;
import static io.prestosql.plugin.hive.HiveErrorCode.HIVE_INVALID_BUCKET_FILES;
import static io.prestosql.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize;
import static io.prestosql.plugin.hive.HiveStorageFormat.CSV;
import static io.prestosql.plugin.hive.HiveTestUtils.HDFS_ENVIRONMENT;
import static io.prestosql.plugin.hive.HiveTestUtils.SESSION;
Expand All @@ -105,6 +113,7 @@
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertThrows;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;

public class TestBackgroundHiveSplitLoader
{
Expand Down Expand Up @@ -156,14 +165,79 @@ public void testNoPathFilter()
public void testCsv()
throws Exception
{
assertSplitCount(CSV, ImmutableMap.of(), 33);
assertSplitCount(CSV, ImmutableMap.of("skip.header.line.count", "1"), 33);
assertSplitCount(CSV, ImmutableMap.of("skip.header.line.count", "2"), 1);
assertSplitCount(CSV, ImmutableMap.of("skip.footer.line.count", "1"), 1);
assertSplitCount(CSV, ImmutableMap.of("skip.header.line.count", "1", "skip.footer.line.count", "1"), 1);
DataSize fileSize = DataSize.of(2, GIGABYTE);
assertSplitCount(CSV, ImmutableMap.of(), fileSize, 33);
assertSplitCount(CSV, ImmutableMap.of("skip.header.line.count", "1"), fileSize, 33);
assertSplitCount(CSV, ImmutableMap.of("skip.header.line.count", "2"), fileSize, 1);
assertSplitCount(CSV, ImmutableMap.of("skip.footer.line.count", "1"), fileSize, 1);
assertSplitCount(CSV, ImmutableMap.of("skip.header.line.count", "1", "skip.footer.line.count", "1"), fileSize, 1);
}

private void assertSplitCount(HiveStorageFormat storageFormat, Map<String, String> tableProperties, int expectedSplitCount)
@Test
public void testSplittableNotCheckedOnSmallFiles()
throws Exception
{
DataSize initialSplitSize = getMaxInitialSplitSize(SESSION);

Table table = table(
ImmutableList.of(),
Optional.empty(),
ImmutableMap.of(),
StorageFormat.create(LazySimpleSerDe.class.getName(), TestSplittableFailureInputFormat.class.getName(), TestSplittableFailureInputFormat.class.getName()));

// Exactly minimum split size, no isSplittable check
BackgroundHiveSplitLoader backgroundHiveSplitLoader = backgroundHiveSplitLoader(
ImmutableList.of(locatedFileStatus(new Path(SAMPLE_PATH), initialSplitSize.toBytes())),
TupleDomain.all(),
Optional.empty(),
table,
Optional.empty());

HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader);
backgroundHiveSplitLoader.start(hiveSplitSource);

assertEquals(drainSplits(hiveSplitSource).size(), 1);

// Large enough for isSplittable to be called
backgroundHiveSplitLoader = backgroundHiveSplitLoader(
ImmutableList.of(locatedFileStatus(new Path(SAMPLE_PATH), initialSplitSize.toBytes() + 1)),
TupleDomain.all(),
Optional.empty(),
table,
Optional.empty());

hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader);
backgroundHiveSplitLoader.start(hiveSplitSource);

try {
drainSplits(hiveSplitSource);
fail("Expected split generation to call isSplittable and fail");
}
catch (PrestoException e) {
Throwable cause = Throwables.getRootCause(e);
assertTrue(cause instanceof IllegalStateException);
assertEquals(cause.getMessage(), "isSplittable called");
}
}

public static final class TestSplittableFailureInputFormat
extends FileInputFormat<Void, Void>
{
@Override
protected boolean isSplitable(FileSystem fs, Path filename)
{
throw new IllegalStateException("isSplittable called");
}

@Override
public RecordReader<Void, Void> getRecordReader(InputSplit inputSplit, JobConf jobConf, Reporter reporter)
throws IOException
{
throw new UnsupportedOperationException();
}
}

private void assertSplitCount(HiveStorageFormat storageFormat, Map<String, String> tableProperties, DataSize fileSize, int expectedSplitCount)
throws Exception
{
Table table = table(
Expand All @@ -173,7 +247,7 @@ private void assertSplitCount(HiveStorageFormat storageFormat, Map<String, Strin
StorageFormat.fromHiveStorageFormat(storageFormat));

BackgroundHiveSplitLoader backgroundHiveSplitLoader = backgroundHiveSplitLoader(
ImmutableList.of(locatedFileStatus(new Path(SAMPLE_PATH), DataSize.of(2, GIGABYTE).toBytes())),
ImmutableList.of(locatedFileStatus(new Path(SAMPLE_PATH), fileSize.toBytes())),
TupleDomain.all(),
Optional.empty(),
table,
Expand Down

0 comments on commit 2cf1ca3

Please sign in to comment.