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 2017/11/08 04:15:55 UTC

[21/24] carbondata git commit: [CARBONDATA-1618] Fix issue of not support table comment

[CARBONDATA-1618] Fix issue of not support table comment

Background: Current carbon do not support table comment when create table.
This PR will support table comment.

This closes #1437


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/ae280e23
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/ae280e23
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/ae280e23

Branch: refs/heads/pre-aggregate
Commit: ae280e239aa76ff547ceedfdb8fb031ae5af078e
Parents: 0578ba0
Author: chenerlu <ch...@huawei.com>
Authored: Thu Oct 26 11:12:42 2017 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Nov 7 19:10:24 2017 +0530

----------------------------------------------------------------------
 .../core/metadata/schema/table/TableInfo.java   | 11 +++
 .../TestCreateTableWithTableComment.scala       | 75 ++++++++++++++++++++
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  6 +-
 .../command/carbonTableSchemaCommon.scala       |  6 +-
 .../CarbonDescribeFormattedCommand.scala        |  2 +
 .../spark/sql/parser/CarbonSparkSqlParser.scala |  5 +-
 6 files changed, 101 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 0c807f6..717eada 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -79,6 +79,9 @@ public class TableInfo implements Serializable, Writable {
   // this idenifier is a lazy field which will be created when it is used first time
   private AbsoluteTableIdentifier identifier;
 
+  // table comment
+  private String tableComment;
+
   public TableInfo() {
   }
 
@@ -160,6 +163,14 @@ public class TableInfo implements Serializable, Writable {
     this.storePath = storePath;
   }
 
+  public String getTableComment() {
+    return tableComment;
+  }
+
+  public void setTableComment(String tableComment) {
+    this.tableComment = tableComment;
+  }
+
   /**
    * to generate the hash code
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala
new file mode 100644
index 0000000..86b8327
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableWithTableComment.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.createTable
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * test functionality for create table with table comment
+ */
+class TestCreateTableWithTableComment extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("use default")
+    sql("drop table if exists withTableComment")
+    sql("drop table if exists withoutTableComment")
+  }
+
+  test("test create table with table comment") {
+    sql(
+      s"""
+         | create table withTableComment(
+         | id int,
+         | name string
+         | )
+         | comment "This table has table comment"
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    val result = sql("describe formatted withTableComment")
+
+    checkExistence(result, true, "Comment:")
+    checkExistence(result, true, "This table has table comment")
+  }
+
+  test("test create table without table comment") {
+    sql(
+      s"""
+         | create table withoutTableComment(
+         | id int,
+         | name string
+         | )
+         | STORED BY 'carbondata'
+       """.stripMargin
+    )
+
+    val result = sql("describe formatted withoutTableComment")
+
+    checkExistence(result, true, "Comment:")
+    checkExistence(result, false, "This table has table comment")
+  }
+
+  override def afterAll: Unit = {
+    sql("use default")
+    sql("drop table if exists withTableComment")
+    sql("drop table if exists withoutTableComment")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index edb471e..aae4f25 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -246,7 +246,8 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       , tableName: String, fields: Seq[Field],
       partitionCols: Seq[PartitionerField],
       tableProperties: mutable.Map[String, String],
-      bucketFields: Option[BucketFields], isAlterFlow: Boolean = false): TableModel = {
+      bucketFields: Option[BucketFields], isAlterFlow: Boolean = false,
+      tableComment: Option[String] = None): TableModel = {
 
     fields.zipWithIndex.foreach { case (field, index) =>
       field.schemaOrdinal = index
@@ -286,7 +287,8 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       groupCols,
       Some(colProps),
       bucketFields: Option[BucketFields],
-      partitionInfo)
+      partitionInfo,
+      tableComment)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 285abf4..fba3085 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -60,7 +60,8 @@ case class TableModel(
     colProps: Option[util.Map[String,
     util.List[ColumnProperty]]] = None,
     bucketFields: Option[BucketFields],
-    partitionInfo: Option[PartitionInfo])
+    partitionInfo: Option[PartitionInfo],
+    tableComment: Option[String] = None)
 
 case class Field(column: String, var dataType: Option[String], name: Option[String],
     children: Option[List[Field]], parent: String = null,
@@ -510,6 +511,8 @@ class TableNewProcessor(cm: TableModel) {
     cm.tableProperties.foreach {
       x => tablePropertiesMap.put(x._1, x._2)
     }
+    // Add table comment to table properties
+    tablePropertiesMap.put("comment", cm.tableComment.getOrElse(""))
     tableSchema.setTableProperties(tablePropertiesMap)
     if (cm.bucketFields.isDefined) {
       val bucketCols = cm.bucketFields.get.bucketColumns.map { b =>
@@ -548,6 +551,7 @@ class TableNewProcessor(cm: TableModel) {
     tableInfo.setTableUniqueName(cm.databaseName + "_" + cm.tableName)
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
+    tableInfo.setTableComment(cm.tableComment.getOrElse(""))
     tableInfo
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
index e5f6b75..e57f490 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
@@ -105,6 +105,8 @@ private[sql] case class CarbonDescribeFormattedCommand(
     results ++= Seq(("Table Name: ", relation.tableMeta.carbonTableIdentifier.getTableName, ""))
     results ++= Seq(("CARBON Store Path: ", relation.tableMeta.storePath, ""))
     val carbonTable = relation.tableMeta.carbonTable
+    // Carbon table support table comment
+    results ++= Seq(("Comment: ", carbonTable.getTableInfo.getTableComment, ""))
     results ++= Seq(("Table Block Size : ", carbonTable.getBlockSizeInMB + " MB", ""))
     results ++= Seq(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
       .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ae280e23/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index 52008f2..81ce73f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -103,6 +103,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
       val partitionerFields = partitionByStructFields.map { structField =>
         PartitionerField(structField.name, Some(structField.dataType.toString), null)
       }
+      val tableComment = Option(ctx.STRING()).map(string)
       val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList)
       val properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues)
         .getOrElse(Map.empty)
@@ -144,7 +145,9 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         fields,
         partitionerFields,
         tableProperties,
-        bucketFields)
+        bucketFields,
+        false,
+        tableComment)
 
       CarbonCreateTableCommand(tableModel)
     } else {