You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by cw...@apache.org on 2011/08/19 20:55:31 UTC

svn commit: r1159742 - in /hive/trunk: cli/ cli/src/java/org/apache/hadoop/hive/cli/ cli/src/test/ cli/src/test/org/ cli/src/test/org/apache/ cli/src/test/org/apache/hadoop/ cli/src/test/org/apache/hadoop/hive/ cli/src/test/org/apache/hadoop/hive/cli/ ...

Author: cws
Date: Fri Aug 19 18:55:31 2011
New Revision: 1159742

URL: http://svn.apache.org/viewvc?rev=1159742&view=rev
Log:
HIVE-2334. DESCRIBE TABLE causes NPE when hive.cli.print.header=true (Jakob Homan via cws)

Added:
    hive/trunk/cli/src/test/
    hive/trunk/cli/src/test/org/
    hive/trunk/cli/src/test/org/apache/
    hive/trunk/cli/src/test/org/apache/hadoop/
    hive/trunk/cli/src/test/org/apache/hadoop/hive/
    hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/
    hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java   (with props)
Modified:
    hive/trunk/cli/build.xml
    hive/trunk/cli/ivy.xml
    hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/trunk/ql/src/test/queries/clientpositive/print_header.q
    hive/trunk/ql/src/test/results/clientpositive/print_header.q.out

Modified: hive/trunk/cli/build.xml
URL: http://svn.apache.org/viewvc/hive/trunk/cli/build.xml?rev=1159742&r1=1159741&r2=1159742&view=diff
==============================================================================
--- hive/trunk/cli/build.xml (original)
+++ hive/trunk/cli/build.xml Fri Aug 19 18:55:31 2011
@@ -41,8 +41,4 @@ to call at top-level: ant deploy-contrib
     </javac>
   </target>
 
-  <target name="test">
-    <echo message="Nothing to do!"/>
-  </target>
-
 </project>

Modified: hive/trunk/cli/ivy.xml
URL: http://svn.apache.org/viewvc/hive/trunk/cli/ivy.xml?rev=1159742&r1=1159741&r2=1159742&view=diff
==============================================================================
--- hive/trunk/cli/ivy.xml (original)
+++ hive/trunk/cli/ivy.xml Fri Aug 19 18:55:31 2011
@@ -28,5 +28,6 @@
           <artifact name="hadoop" type="source" ext="tar.gz"/>
         </dependency>
         <dependency org="commons-cli" name="commons-cli" rev="${commons-cli.version}"/>
+        <dependency org="org.mockito" name="mockito-all" rev="${mockito-all.version}" />
     </dependencies>
 </ivy-module>

Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1159742&r1=1159741&r2=1159742&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Fri Aug 19 18:55:31 2011
@@ -48,7 +48,6 @@ import org.apache.hadoop.hive.common.Log
 import org.apache.hadoop.hive.common.LogUtils.LogInitializationException;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
@@ -83,7 +82,7 @@ public class CliDriver {
   public static final String HIVERCFILE = ".hiverc";
 
   private final LogHelper console;
-  private final Configuration conf;
+  private Configuration conf;
 
   public CliDriver() {
     SessionState ss = SessionState.get();
@@ -94,10 +93,8 @@ public class CliDriver {
 
   public int processCmd(String cmd) {
     CliSessionState ss = (CliSessionState) SessionState.get();
-
     String cmd_trimmed = cmd.trim();
-    String[] tokens = cmd_trimmed.split("\\s+");
-    String cmd_1 = cmd_trimmed.substring(tokens[0].length()).trim();
+    String[] tokens = tokenizeCmd(cmd_trimmed);
     int ret = 0;
 
     if (cmd_trimmed.toLowerCase().equals("quit") || cmd_trimmed.toLowerCase().equals("exit")) {
@@ -109,6 +106,8 @@ public class CliDriver {
       System.exit(0);
 
     } else if (tokens[0].equalsIgnoreCase("source")) {
+      String cmd_1 = getFirstCmd(cmd_trimmed, tokens[0].length());
+
       File sourceFile = new File(cmd_1);
       if (! sourceFile.isFile()){
         console.printError("File: "+ cmd_1 + " is not a file.");
@@ -207,91 +206,132 @@ public class CliDriver {
         }
       }
     } else { // local mode
-      CommandProcessor proc = CommandProcessorFactory.get(tokens[0], (HiveConf)conf);
-      int tryCount = 0;
-      boolean needRetry;
+      CommandProcessor proc = CommandProcessorFactory.get(tokens[0], (HiveConf) conf);
+      ret = processLocalCmd(cmd, proc, ss);
+    }
 
-      do {
-        try {
-          needRetry = false;
-          if (proc != null) {
-            if (proc instanceof Driver) {
-              Driver qp = (Driver) proc;
-              PrintStream out = ss.out;
-              long start = System.currentTimeMillis();
-              if (ss.getIsVerbose()) {
-                out.println(cmd);
-              }
+    return ret;
+  }
 
-              qp.setTryCount(tryCount);
-              ret = qp.run(cmd).getResponseCode();
-              if (ret != 0) {
-                qp.close();
-                return ret;
-              }
+  /**
+   * For testing purposes to inject Configuration dependency
+   * @param conf to replace default
+   */
+  void setConf(Configuration conf) {
+    this.conf = conf;
+  }
 
-              ArrayList<String> res = new ArrayList<String>();
+  /**
+   * Extract and clean up the first command in the input.
+   */
+  private String getFirstCmd(String cmd, int length) {
+    return cmd.substring(length).trim();
+  }
 
-              if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) {
-                // Print the column names
-                boolean first_col = true;
-                Schema sc = qp.getSchema();
-                for (FieldSchema fs : sc.getFieldSchemas()) {
-                  if (!first_col) {
-                    out.print('\t');
-                  }
-                  out.print(fs.getName());
-                  first_col = false;
-                }
-                out.println();
-              }
+  private String[] tokenizeCmd(String cmd) {
+    return cmd.split("\\s+");
+  }
+
+  int processLocalCmd(String cmd, CommandProcessor proc, CliSessionState ss) {
+    int tryCount = 0;
+    boolean needRetry;
+    int ret = 0;
+
+    do {
+      try {
+        needRetry = false;
+        if (proc != null) {
+          if (proc instanceof Driver) {
+            Driver qp = (Driver) proc;
+            PrintStream out = ss.out;
+            long start = System.currentTimeMillis();
+            if (ss.getIsVerbose()) {
+              out.println(cmd);
+            }
 
-              try {
-                while (qp.getResults(res)) {
-                  for (String r : res) {
-                    out.println(r);
-                  }
-                  res.clear();
-                  if (out.checkError()) {
-                    break;
-                  }
+            qp.setTryCount(tryCount);
+            ret = qp.run(cmd).getResponseCode();
+            if (ret != 0) {
+              qp.close();
+              return ret;
+            }
+
+            ArrayList<String> res = new ArrayList<String>();
+
+            printHeader(qp, out);
+
+            try {
+              while (qp.getResults(res)) {
+                for (String r : res) {
+                  out.println(r);
+                }
+                res.clear();
+                if (out.checkError()) {
+                  break;
                 }
-              } catch (IOException e) {
-                console.printError("Failed with exception " + e.getClass().getName() + ":"
-                    + e.getMessage(), "\n"
-                    + org.apache.hadoop.util.StringUtils.stringifyException(e));
-                ret = 1;
               }
+            } catch (IOException e) {
+              console.printError("Failed with exception " + e.getClass().getName() + ":"
+                  + e.getMessage(), "\n"
+                  + org.apache.hadoop.util.StringUtils.stringifyException(e));
+              ret = 1;
+            }
 
-              int cret = qp.close();
-              if (ret == 0) {
-                ret = cret;
-              }
+            int cret = qp.close();
+            if (ret == 0) {
+              ret = cret;
+            }
 
-              long end = System.currentTimeMillis();
-              if (end > start) {
-                double timeTaken = (end - start) / 1000.0;
-                console.printInfo("Time taken: " + timeTaken + " seconds", null);
-              }
+            long end = System.currentTimeMillis();
+            if (end > start) {
+              double timeTaken = (end - start) / 1000.0;
+              console.printInfo("Time taken: " + timeTaken + " seconds", null);
+            }
 
-            } else {
-              if (ss.getIsVerbose()) {
-                ss.out.println(tokens[0] + " " + cmd_1);
-              }
-              ret = proc.run(cmd_1).getResponseCode();
+          } else {
+            String firstToken = tokenizeCmd(cmd.trim())[0];
+            String cmd_1 = getFirstCmd(cmd.trim(), firstToken.length());
+
+            if (ss.getIsVerbose()) {
+              ss.out.println(firstToken + " " + cmd_1);
             }
+            ret = proc.run(cmd_1).getResponseCode();
           }
-        } catch (CommandNeedRetryException e) {
-          console.printInfo("Retry query with a different approach...");
-          tryCount++;
-          needRetry = true;
         }
-      } while (needRetry);
-    }
+      } catch (CommandNeedRetryException e) {
+        console.printInfo("Retry query with a different approach...");
+        tryCount++;
+        needRetry = true;
+      }
+    } while (needRetry);
 
     return ret;
   }
 
+  /**
+   * If enabled and applicable to this command, print the field headers
+   * for the output.
+   *
+   * @param qp Driver that executed the command
+   * @param out Printstream which to send output to
+   */
+  private void printHeader(Driver qp, PrintStream out) {
+    List<FieldSchema> fieldSchemas = qp.getSchema().getFieldSchemas();
+    if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)
+          && fieldSchemas != null) {
+      // Print the column names
+      boolean first_col = true;
+      for (FieldSchema fs : fieldSchemas) {
+        if (!first_col) {
+          out.print('\t');
+        }
+        out.print(fs.getName());
+        first_col = false;
+      }
+      out.println();
+    }
+  }
+
   public int processLine(String line) {
     return processLine(line, false);
   }

Added: hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java?rev=1159742&view=auto
==============================================================================
--- hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java (added)
+++ hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java Fri Aug 19 18:55:31 2011
@@ -0,0 +1,102 @@
+/**
+ * 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.cli;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Schema;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+// Cannot call class TestCliDriver since that's the name of the generated
+// code for the script-based testing
+public class TestCliDriverMethods extends TestCase {
+
+  // If the command has an associated schema, make sure it gets printed to use
+  public void testThatCliDriverPrintsHeaderForCommandsWithSchema() throws CommandNeedRetryException {
+    Schema mockSchema = mock(Schema.class);
+    List<FieldSchema> fieldSchemas = new ArrayList<FieldSchema>();
+    String fieldName = "FlightOfTheConchords";
+    fieldSchemas.add(new FieldSchema(fieldName, "type", "comment"));
+
+    when(mockSchema.getFieldSchemas()).thenReturn(fieldSchemas);
+
+    PrintStream mockOut = headerPrintingTestDriver(mockSchema);
+    // Should have printed out the header for the field schema
+    verify(mockOut, times(1)).print(fieldName);
+  }
+
+  // If the command has no schema, make sure nothing is printed
+  public void testThatCliDriverPrintsNoHeaderForCommandsWithNoSchema() throws CommandNeedRetryException {
+    Schema mockSchema = mock(Schema.class);
+    when(mockSchema.getFieldSchemas()).thenReturn(null);
+
+    PrintStream mockOut = headerPrintingTestDriver(mockSchema);
+    // Should not have tried to print any thing.
+    verify(mockOut, never()).print(anyString());
+  }
+
+  /**
+   * Do the actual testing against a mocked CliDriver based on what type of schema
+   * @param mockSchema Schema to throw against test
+   * @return Output that would have been sent to the user
+   * @throws CommandNeedRetryException won't actually be thrown
+   */
+  private PrintStream headerPrintingTestDriver(Schema mockSchema) throws CommandNeedRetryException {
+    CliDriver cliDriver = new CliDriver();
+
+    // We want the driver to try to print the header...
+    Configuration conf = mock(Configuration.class);
+    when(conf.getBoolean(eq(ConfVars.HIVE_CLI_PRINT_HEADER.varname), anyBoolean())).thenReturn(true);
+    cliDriver.setConf(conf);
+
+    Driver proc = mock(Driver.class);
+
+    CommandProcessorResponse cpr = mock(CommandProcessorResponse.class);
+    when(cpr.getResponseCode()).thenReturn(0);
+    when(proc.run(anyString())).thenReturn(cpr);
+
+    // and then see what happens based on the provided schema
+    when(proc.getSchema()).thenReturn(mockSchema);
+
+    CliSessionState mockSS = mock(CliSessionState.class);
+    PrintStream mockOut = mock(PrintStream.class);
+
+    mockSS.out = mockOut;
+
+    cliDriver.processLocalCmd("use default;", proc, mockSS);
+    return mockOut;
+  }
+
+}

Propchange: hive/trunk/cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hive/trunk/ql/src/test/queries/clientpositive/print_header.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/print_header.q?rev=1159742&r1=1159741&r2=1159742&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/print_header.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/print_header.q Fri Aug 19 18:55:31 2011
@@ -11,3 +11,4 @@ SELECT src1.key as k1, src1.value as v1,
 
 SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 10;
 
+use default;

Modified: hive/trunk/ql/src/test/results/clientpositive/print_header.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/print_header.q.out?rev=1159742&r1=1159741&r2=1159742&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/print_header.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/print_header.q.out Fri Aug 19 18:55:31 2011
@@ -7,7 +7,7 @@ PREHOOK: query: SELECT src1.key as k1, s
   LIMIT 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
-PREHOOK: Output: file:/tmp/pbutler/hive_2010-11-22_13-41-11_283_258399688654159660/-mr-10000
+PREHOOK: Output: file:/var/folders/Uq/UqBVUjdIE+qhrtZHnLNzc++++TM/-Tmp-/jhoman/hive_2011-08-08_12-36-14_972_2863485419468463022/-mr-10000
 POSTHOOK: query: SELECT src1.key as k1, src1.value as v1, 
        src2.key as k2, src2.value as v2 FROM 
   (SELECT * FROM src WHERE src.key < 10) src1 
@@ -17,7 +17,7 @@ POSTHOOK: query: SELECT src1.key as k1, 
   LIMIT 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
-POSTHOOK: Output: file:/tmp/pbutler/hive_2010-11-22_13-41-11_283_258399688654159660/-mr-10000
+POSTHOOK: Output: file:/var/folders/Uq/UqBVUjdIE+qhrtZHnLNzc++++TM/-Tmp-/jhoman/hive_2011-08-08_12-36-14_972_2863485419468463022/-mr-10000
 k1	v1	k2	v2
 0	val_0	0	val_0
 0	val_0	0	val_0
@@ -32,11 +32,11 @@ k1	v1	k2	v2
 PREHOOK: query: SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
-PREHOOK: Output: file:/tmp/pbutler/hive_2010-11-22_13-41-30_510_2595029549749893604/-mr-10000
+PREHOOK: Output: file:/var/folders/Uq/UqBVUjdIE+qhrtZHnLNzc++++TM/-Tmp-/jhoman/hive_2011-08-08_12-36-28_635_5275683580628567895/-mr-10000
 POSTHOOK: query: SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
-POSTHOOK: Output: file:/tmp/pbutler/hive_2010-11-22_13-41-30_510_2595029549749893604/-mr-10000
+POSTHOOK: Output: file:/var/folders/Uq/UqBVUjdIE+qhrtZHnLNzc++++TM/-Tmp-/jhoman/hive_2011-08-08_12-36-28_635_5275683580628567895/-mr-10000
 key	_c1
 0	0.0
 10	10.0
@@ -48,3 +48,7 @@ key	_c1
 111	111.0
 113	226.0
 114	114.0
+PREHOOK: query: use default
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: use default
+POSTHOOK: type: SWITCHDATABASE