You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ke...@apache.org on 2012/11/03 01:27:18 UTC

svn commit: r1405240 - in /hive/trunk: ./ ql/src/java/org/apache/hadoop/hive/ql/parse/ ql/src/test/org/apache/hadoop/hive/ql/hooks/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/

Author: kevinwilfong
Date: Sat Nov  3 00:27:17 2012
New Revision: 1405240

URL: http://svn.apache.org/viewvc?rev=1405240&view=rev
Log:
HIVE-3640. Reducer allocation is incorrect if enforce bucketing and mapred.reduce.tasks are both set. (Vighnesh Avadhani via kevinwilfong)

Added:
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyNumReducersForBucketsHook.java
    hive/trunk/ql/src/test/queries/clientpositive/bucket_num_reducers.q
    hive/trunk/ql/src/test/results/clientpositive/bucket_num_reducers.q.out
Modified:
    hive/trunk/build-common.xml
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java

Modified: hive/trunk/build-common.xml
URL: http://svn.apache.org/viewvc/hive/trunk/build-common.xml?rev=1405240&r1=1405239&r2=1405240&view=diff
==============================================================================
--- hive/trunk/build-common.xml (original)
+++ hive/trunk/build-common.xml Sat Nov  3 00:27:17 2012
@@ -57,7 +57,7 @@
   <property name="test.output" value="true"/>
   <property name="test.junit.output.format" value="xml"/>
   <property name="test.junit.output.usefile" value="true"/>
-  <property name="minimr.query.files" value="input16_cc.q,scriptfile1.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q"/>
+  <property name="minimr.query.files" value="input16_cc.q,scriptfile1.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q"/>
   <property name="minimr.query.negative.files" value="cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q" />
   <property name="test.silent" value="true"/>
   <property name="hadoopVersion" value="${hadoop.version.ant-internal}"/>

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1405240&r1=1405239&r2=1405240&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Sat Nov  3 00:27:17 2012
@@ -4350,6 +4350,9 @@ public class SemanticAnalyzer extends Ba
 
     if (enforceBucketing || enforceSorting) {
       int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS);
+      if (conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS) > 0) {
+        maxReducers = conf.getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS);
+      }
       int numBuckets  = dest_tab.getNumBuckets();
       if (numBuckets > maxReducers) {
         multiFileSpray = true;

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyNumReducersForBucketsHook.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyNumReducersForBucketsHook.java?rev=1405240&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyNumReducersForBucketsHook.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/hooks/VerifyNumReducersForBucketsHook.java Sat Nov  3 00:27:17 2012
@@ -0,0 +1,45 @@
+/**
+ * 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.hooks;
+
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.ql.MapRedStats;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+/**
+ *
+ * VerifyNumReducersForBucketsHook.
+ *
+ * This hook is meant to be used with bucket_num_reducers.q . It checks whether the
+ * number of reducers has been correctly set.
+ */
+public class VerifyNumReducersForBucketsHook implements ExecuteWithHookContext {
+
+  public void run(HookContext hookContext) {
+    SessionState ss = SessionState.get();
+    Assert.assertNotNull("SessionState returned null");
+
+    List<MapRedStats> stats = ss.getLastMapRedStatsList();
+    Assert.assertEquals("Number of MapReduce jobs is incorrect", 1, stats.size());
+
+    Assert.assertEquals("NumReducers is incorrect", 10, stats.get(0).getNumReduce());
+  }
+}

Added: hive/trunk/ql/src/test/queries/clientpositive/bucket_num_reducers.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/bucket_num_reducers.q?rev=1405240&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/bucket_num_reducers.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/bucket_num_reducers.q Sat Nov  3 00:27:17 2012
@@ -0,0 +1,15 @@
+set hive.enforce.bucketing = true;
+set hive.exec.mode.local.auto=false;
+set mapred.reduce.tasks = 10;
+
+-- This test sets number of mapred tasks to 10 for a database with 50 buckets, 
+-- and uses a post-hook to confirm that 10 tasks were created
+
+CREATE TABLE bucket_nr(key int, value string) CLUSTERED BY (key) INTO 50 BUCKETS;
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyNumReducersForBucketsHook;
+
+insert overwrite table bucket_nr
+select * from src;
+
+set hive.exec.post.hooks=;
+drop table bucket_nr;

Added: hive/trunk/ql/src/test/results/clientpositive/bucket_num_reducers.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucket_num_reducers.q.out?rev=1405240&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucket_num_reducers.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/bucket_num_reducers.q.out Sat Nov  3 00:27:17 2012
@@ -0,0 +1,20 @@
+PREHOOK: query: -- This test sets number of mapred tasks to 10 for a database with 50 buckets, 
+-- and uses a post-hook to confirm that 10 tasks were created
+
+CREATE TABLE bucket_nr(key int, value string) CLUSTERED BY (key) INTO 50 BUCKETS
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- This test sets number of mapred tasks to 10 for a database with 50 buckets, 
+-- and uses a post-hook to confirm that 10 tasks were created
+
+CREATE TABLE bucket_nr(key int, value string) CLUSTERED BY (key) INTO 50 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@bucket_nr
+PREHOOK: query: insert overwrite table bucket_nr
+select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@bucket_nr
+PREHOOK: query: drop table bucket_nr
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@bucket_nr
+PREHOOK: Output: default@bucket_nr