Repository: hive Updated Branches: refs/heads/master 70cb7f0b2 -> 2e5ead142
HIVE-17408 : replication distcp should only be invoked if number of files AND file size cross configured limits (Anishek Agarwal reviewed by Thejas Nair) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2e5ead14 Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2e5ead14 Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2e5ead14 Branch: refs/heads/master Commit: 2e5ead142f88333899223aaf810976c4dcf02efb Parents: 70cb7f0 Author: Anishek Agarwal <[email protected]> Authored: Wed Aug 30 17:43:01 2017 -0700 Committer: Thejas M Nair <[email protected]> Committed: Wed Aug 30 17:43:01 2017 -0700 ---------------------------------------------------------------------- .../hadoop/hive/ql/parse/repl/CopyUtils.java | 4 +- .../hive/ql/parse/repl/CopyUtilsTest.java | 47 ++++++++++++++++++++ 2 files changed, 49 insertions(+), 2 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/2e5ead14/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java ---------------------------------------------------------------------- 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 db923e3..28e7bcb 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 @@ public class CopyUtils { return true; } - private boolean limitReachedForLocalCopy(long size, long numberOfFiles) { - boolean result = size > 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); http://git-wip-us.apache.org/repos/asf/hive/blob/2e5ead14/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/CopyUtilsTest.java ---------------------------------------------------------------------- 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 0000000..e643d8f --- /dev/null +++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/repl/CopyUtilsTest.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 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
