You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2019/08/22 12:17:01 UTC

[carbondata] branch master updated: [CARBONDATA-3466] Fix NPE for carboncli command

This is an automated email from the ASF dual-hosted git repository.

jackylk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new a264b6f  [CARBONDATA-3466] Fix NPE for carboncli command
a264b6f is described below

commit a264b6f0bd6bf7361531267a7776c9dc466f1abc
Author: Manhua <ke...@qq.com>
AuthorDate: Mon Jul 8 10:37:50 2019 +0800

    [CARBONDATA-3466] Fix NPE for carboncli command
    
    What is changed:
    
    require option for carboncli command when parsing
    unify to fill result in array when running carbon cli , use adapter to print to stream on demand
    Problem details:
    if no options specified, like "carboncli for table source", NPE will occur when running commandOptions.split("\\s+") in CarbonCliCommand#processData.
    Since option, as a role of command, is a must in CarbonCli, we require it when parsing command in this PR.
    
    This closes #3318
---
 .../TestAlterTableSortColumnsProperty.scala        |  1 -
 .../command/management/CarbonCliCommand.scala      |  2 +-
 .../spark/sql/parser/CarbonSpark2SqlParser.scala   | 11 +--
 .../carbondata/spark/testsuite/TestCarbonCli.scala | 82 ++++++++++++++++++++
 .../java/org/apache/carbondata/tool/CarbonCli.java | 88 ++++++++--------------
 5 files changed, 116 insertions(+), 68 deletions(-)

diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
index dab9934..2ac6889 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/alterTable/TestAlterTableSortColumnsProperty.scala
@@ -601,7 +601,6 @@ class TestAlterTableSortColumnsProperty extends QueryTest with BeforeAndAfterAll
       val out: ByteArrayOutputStream = new ByteArrayOutputStream
       val stream: PrintStream = new PrintStream(out)
       CarbonCli.run(args, stream)
-      CarbonCli.cleanOutPuts()
       val output: String = new String(out.toByteArray)
       if (segmentId == 2) {
         assertResult(s"Input Folder: $segmentPath\nsorted by intfield,stringfield\n")(output)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCliCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCliCommand.scala
index d1a54d0..5dd0c12 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCliCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCliCommand.scala
@@ -56,7 +56,7 @@ case class CarbonCliCommand(
       case x => Seq(x.trim)
     }.flatten
     val summaryOutput = new util.ArrayList[String]()
-    CarbonCli.run(finalCommands.toArray, summaryOutput)
+    CarbonCli.run(finalCommands.toArray, summaryOutput, false)
     summaryOutput.asScala.map(x =>
       Row(x)
     )
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 6db6d01..22548ff 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -523,14 +523,9 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   protected lazy val cli: Parser[LogicalPlan] =
-    (CARBONCLI ~> FOR ~> TABLE) ~> (ident <~ ".").? ~ ident ~
-    (OPTIONS ~> "(" ~> commandOptions <~ ")").? <~
-    opt(";") ^^ {
-      case databaseName ~ tableName ~ commandList =>
-        var commandOptions: String = null
-        if (commandList.isDefined) {
-          commandOptions = commandList.get
-        }
+    CARBONCLI ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident ~
+    (OPTIONS ~> "(" ~> commandOptions <~ ")") <~ opt(";") ^^ {
+      case databaseName ~ tableName ~ commandOptions =>
         CarbonCliCommand(
           convertDbNameToLowerCase(databaseName),
           tableName.toLowerCase(),
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/TestCarbonCli.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/TestCarbonCli.scala
new file mode 100644
index 0000000..ba54e8f
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/TestCarbonCli.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.carbondata.spark.testsuite
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.common.util.Spark2QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestCarbonCli extends Spark2QueryTest with BeforeAndAfterAll{
+
+  override protected def beforeAll(): Unit = {
+    sql("drop table if exists OneRowTable")
+    sql("create table OneRowTable(col1 string, col2 string, col3 int, col4 double) stored by 'carbondata'")
+    sql("insert into OneRowTable select '0.1', 'a.b', 1, 1.2")
+  }
+
+  test("CarbonCli table summary") {
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd summary -a')"),
+      true, "## Summary")
+
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd summary -v')"),
+      true, "## version Details")
+
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd summary -s')"),
+      true, "## Schema")
+
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd summary -t')"),
+      true, "## Table Properties")
+
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd summary -m')"),
+      true, "## Segment")
+  }
+
+  test("CarbonCli column details") {
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd summary -c col1')"),
+      true, "## Column Statistics for 'col1'")
+  }
+
+  test("CarbonCli benchmark") {
+    checkExistence(
+      sql("carboncli for table OneRowTable options('-cmd benchmark -c col1')"),
+      true, "## Benchmark")
+  }
+
+  test("CarbonCli invalid cmd"){
+
+    assert(intercept[AnalysisException] {
+      sql("carboncli for table OneRowTable").show()
+    }.getMessage().contains("mismatched input 'carboncli'"))
+
+    assert(intercept[Exception] {
+      sql("carboncli for table OneRowTable options('')")
+    }.getMessage().contains("Missing required option: cmd"))
+
+    checkExistence(sql("carboncli for table OneRowTable options('-cmd test')"),
+      true, "command test is not supported")
+  }
+
+  override protected def afterAll(): Unit = {
+    sql("drop table if exists OneRowTable")
+  }
+}
diff --git a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
index d562b8e..ef9a50e 100644
--- a/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
+++ b/tools/cli/src/main/java/org/apache/carbondata/tool/CarbonCli.java
@@ -22,7 +22,6 @@ import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
-import java.util.List;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
@@ -44,12 +43,6 @@ import org.apache.commons.cli.PosixParser;
 @InterfaceStability.Unstable
 public class CarbonCli {
 
-  // List to collect all the outputs of option details
-  private static List<String> outPuts;
-
-  // a boolean variable to decide whether to print the output in console or return the list,
-  // by default true, and it will be set to false if the cli is trigerred via sql command
-  private static boolean isPrintInConsole = true;
 
   static class OptionsHolder {
     static Options instance = buildOptions();
@@ -117,47 +110,45 @@ public class CarbonCli {
     run(args, System.out);
   }
 
-  public static void run(String[] args, ArrayList<String> e) {
-    // this boolean to check whether to print in console or not
-    isPrintInConsole = false;
-    outPuts = e;
-    Options options = OptionsHolder.instance;
-    CommandLineParser parser = new PosixParser();
 
-    CommandLine line;
-    try {
-      line = parser.parse(options, args);
-    } catch (ParseException ex) {
-      throw new RuntimeException("Parsing failed. Reason: " + ex.getMessage(), ex);
+  /**
+   * adapter to run CLI and print to stream
+   * @param args input arguments
+   * @param out output stream
+   */
+  public static void run(String[] args, PrintStream out) {
+    ArrayList<String> outputs = new ArrayList<String>();
+    run(args, outputs, true);
+    for (String line: outputs) {
+      out.println(line);
     }
-
-    runCli(System.out, options, line);
   }
 
-  public static void run(String[] args, PrintStream out) {
+
+  /**
+   * run CLI and fill result into array
+   * @param args input arguments
+   * @param outPuts array for filling result
+   * @param isPrintInConsole flag to decide whether to print error in console or return the list
+   */
+  public static void run(String[] args, ArrayList<String> outPuts, boolean isPrintInConsole) {
     Options options = OptionsHolder.instance;
     CommandLineParser parser = new PosixParser();
 
-    CommandLine line;
+    CommandLine line = null;
     try {
       line = parser.parse(options, args);
-    } catch (ParseException exp) {
-      out.println("Parsing failed. Reason: " + exp.getMessage());
-      return;
+    } catch (ParseException ex) {
+      if (isPrintInConsole) {
+        outPuts.add("Parsing failed. Reason: " + ex.getMessage());
+        return;
+      } else {
+        throw new RuntimeException("Parsing failed. Reason: " + ex.getMessage(), ex);
+      }
     }
 
-    runCli(out, options, line);
-  }
-
-  private static void  runCli(PrintStream out, Options options, CommandLine line) {
-    if (outPuts == null) {
-      outPuts = new ArrayList<>();
-    }
     if (line.hasOption("h")) {
-      collectHelpInfo(options);
-      for (String output : outPuts) {
-        out.println(output);
-      }
+      outPuts.add(collectHelpInfo(options));
       return;
     }
 
@@ -180,48 +171,29 @@ public class CarbonCli {
         } catch (IOException e) {
           e.printStackTrace();
         }
-        for (String output : outPuts) {
-          out.println(output);
-        }
       }
       return;
     } else {
-      out.println("command " + cmd + " is not supported");
       outPuts.add("command " + cmd + " is not supported");
-      collectHelpInfo(options);
-      for (String output : outPuts) {
-        out.println(output);
-      }
+      outPuts.add(collectHelpInfo(options));
       return;
     }
 
     try {
       command.run(line);
-      if (isPrintInConsole) {
-        for (String output : outPuts) {
-          out.println(output);
-        }
-      }
-      out.flush();
     } catch (IOException | MemoryException e) {
       e.printStackTrace();
-    } finally {
-      out.close();
     }
   }
 
-  private static void collectHelpInfo(Options options) {
+  private static String collectHelpInfo(Options options) {
     HelpFormatter formatter = new HelpFormatter();
     StringWriter stringWriter = new StringWriter();
     PrintWriter printWriter = new PrintWriter(stringWriter);
     formatter.printHelp(printWriter, formatter.getWidth(), "CarbonCli", null, options,
         formatter.getLeftPadding(), formatter.getDescPadding(), null, false);
     printWriter.flush();
-    outPuts.add(stringWriter.toString());
-  }
-
-  public static void cleanOutPuts() {
-    outPuts = null;
+    return stringWriter.toString();
   }
 
 }