Skip to content

Commit

Permalink
Set splitsize for hadoop InputFormat to Presto max_split_size
Browse files Browse the repository at this point in the history
  • Loading branch information
agrawalreetika committed Sep 28, 2024
1 parent 649b45a commit 19bef1c
Show file tree
Hide file tree
Showing 2 changed files with 95 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT;
import static com.facebook.presto.hive.HiveMetadata.shouldCreateFilesForMissingBuckets;
import static com.facebook.presto.hive.HiveSessionProperties.getMaxInitialSplitSize;
import static com.facebook.presto.hive.HiveSessionProperties.getMaxSplitSize;
import static com.facebook.presto.hive.HiveSessionProperties.isFileSplittable;
import static com.facebook.presto.hive.HiveSessionProperties.isOrderBasedExecutionEnabled;
import static com.facebook.presto.hive.HiveSessionProperties.isSkipEmptyFilesEnabled;
Expand Down Expand Up @@ -111,6 +112,7 @@ public class StoragePartitionLoader
private final Deque<Iterator<InternalHiveSplit>> fileIterators;
private final boolean schedulerUsesHostAddresses;
private final boolean partialAggregationsPushedDown;
private static final String SPLIT_MINSIZE = "mapreduce.input.fileinputformat.split.minsize";

public StoragePartitionLoader(
Table table,
Expand Down Expand Up @@ -185,6 +187,7 @@ private ListenableFuture<?> handleSymlinkTextInputFormat(ExtendedFileSystem fs,
JobConf targetJob = toJobConf(targetFilesystem.getConf());
targetJob.setInputFormat(TextInputFormat.class);
targetInputFormat.configure(targetJob);
targetJob.set(SPLIT_MINSIZE, Long.toString(getMaxSplitSize(session).toBytes()));
FileInputFormat.setInputPaths(targetJob, targetPath);
InputSplit[] targetSplits = targetInputFormat.getSplits(targetJob, 0);

Expand Down Expand Up @@ -214,6 +217,7 @@ private ListenableFuture<?> handleGetSplitsFromInputFormat(Configuration configu
FileInputFormat.setInputPaths(jobConf, path);
// SerDes parameters and Table parameters passing into input format
fromProperties(schema).forEach(jobConf::set);
jobConf.set(SPLIT_MINSIZE, Long.toString(getMaxSplitSize(session).toBytes()));
InputSplit[] splits = inputFormat.getSplits(jobConf, 0);

return addSplitsToSource(splits, splitFactory, hiveSplitSource, stopped);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
/*
* 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 com.facebook.presto.hive;

import com.facebook.airlift.log.Logger;
import com.google.common.io.Resources;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.TextInputFormat;
import org.testng.annotations.Test;

import static com.facebook.presto.hive.util.ConfigurationUtils.toJobConf;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;

public class TestCustomInputSplits
{
private static final Logger log = Logger.get(TestCustomInputSplits.class);
private static final String SPLIT_MINSIZE = "mapreduce.input.fileinputformat.split.minsize";
private static final String LOCAL_BLOCK_SIZE = "fs.local.block.size";
private static final String DFS_BLOCK_SIZE = "dfs.block.size";

private Configuration getHadoopConfiguration()
{
long splitSize = 1000;
long blockSize = 500;
Configuration hadoopConf = new Configuration(false);
hadoopConf.set("fs.defaultFS", "file:///");
hadoopConf.set("fs.file.impl", LocalFileSystem.class.getName());
hadoopConf.set("fs.hdfs.impl", LocalFileSystem.class.getName());
hadoopConf.setLong(LOCAL_BLOCK_SIZE, blockSize);
hadoopConf.setLong(DFS_BLOCK_SIZE, blockSize);
hadoopConf.set(SPLIT_MINSIZE, Long.toString(splitSize));

return hadoopConf;
}

@Test
public void testCustomSplitSize() throws Exception
{
String filePath = Resources.getResource("addressbook.parquet").getFile();
Path path = new Path(filePath);

TextInputFormat targetInputFormat = new DummyTextInputFormat();
FileSystem fileSystem = path.getFileSystem(getHadoopConfiguration());
JobConf targetJob = toJobConf(fileSystem.getConf());

targetInputFormat.configure(targetJob);
FileInputFormat.setInputPaths(targetJob, path);
log.info("##### Before getSplits - BLOCK_SIZE 1 = " + targetJob.get(LOCAL_BLOCK_SIZE));
log.info("##### Before getSplits - BLOCK_SIZE 2 = " + targetJob.get(DFS_BLOCK_SIZE));
log.info("##### Before getSplits - SPLIT_MINSIZE = " + targetJob.get(SPLIT_MINSIZE));

InputSplit[] targetSplits = targetInputFormat.getSplits(targetJob, 0);

log.info("##### In testCustomSplitSize blocksize 1 = " + targetJob.get(LOCAL_BLOCK_SIZE));
log.info("##### In testCustomSplitSize blocksize 2 = " + targetJob.get(DFS_BLOCK_SIZE));
log.info("##### In testCustomSplitSize SPLIT_MINSIZE = " + targetJob.get(SPLIT_MINSIZE));
log.info("##### In testCustomSplitSize FileSystem1 = " + FileSystem.get(targetJob));
log.info("##### In testCustomSplitSize FileSystem2 = " + fileSystem.getScheme());
log.info("##### In testCustomSplitSize filesize = " + fileSystem.getFileStatus(path).getLen());
log.info("##### In testCustomSplitSize filesystem blocksize = " + fileSystem.getDefaultBlockSize(path));

assertNotNull(targetSplits);
assertEquals(targetSplits.length, 4);
}
public class DummyTextInputFormat
extends TextInputFormat
{
protected boolean isSplitable(FileSystem fs, Path file)
{
return true;
}
}
}

0 comments on commit 19bef1c

Please sign in to comment.