diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java index db923e3af9..28e7bcb8c4 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java @@ -108,8 +108,8 @@ private boolean regularCopy(FileSystem destinationFs, Map.Entry maxCopyFileSize || numberOfFiles > maxNumberOfFiles; + boolean limitReachedForLocalCopy(long size, long numberOfFiles) { + boolean result = size > maxCopyFileSize && numberOfFiles > maxNumberOfFiles; if (result) { LOG.info("Source is {} bytes. (MAX: {})", size, maxCopyFileSize); LOG.info("Source is {} files. (MAX: {})", numberOfFiles, maxNumberOfFiles); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/CopyUtilsTest.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/CopyUtilsTest.java new file mode 100644 index 0000000000..7e1c75b8c9 --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/CopyUtilsTest.java @@ -0,0 +1,29 @@ +package org.apache.hadoop.hive.ql.parse.repl; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; + +public class CopyUtilsTest { + /* + Distcp currently does not copy a single file in a distributed manner hence we dont care about + the size of file, if there is only file, we dont want to launch distcp. + */ + @Test + public void distcpShouldNotBeCalledOnlyForOneFile() { + HiveConf conf = new HiveConf(); + conf.setLongVar(HiveConf.ConfVars.HIVE_EXEC_COPYFILE_MAXSIZE, 1); + CopyUtils copyUtils = new CopyUtils("", conf); + long MB_128 = 128 * 1024 * 1024; + assertFalse(copyUtils.limitReachedForLocalCopy(MB_128, 1L)); + } + + @Test + public void distcpShouldNotBeCalledForSmallerFileSize() { + HiveConf conf = new HiveConf(); + CopyUtils copyUtils = new CopyUtils("", conf); + long MB_16 = 16 * 1024 * 1024; + assertFalse(copyUtils.limitReachedForLocalCopy(MB_16, 100L)); + } +} \ No newline at end of file