You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/06/25 16:06:01 UTC

carbondata git commit: [CARBONDATA-2630] fix for exception thrown by Alter table set Table comment in spark-2.2 cluster

Repository: carbondata
Updated Branches:
  refs/heads/master d5e86db52 -> c6b2b6a25


[CARBONDATA-2630] fix for exception thrown by Alter table set Table comment in spark-2.2 cluster

As spark restricts to give the length of char . so this PR contains only the test case change in char data type with length.

This closes #2392


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

Branch: refs/heads/master
Commit: c6b2b6a257e4342a9394599b4cf02a5f96751c6d
Parents: d5e86db
Author: rahul <ra...@knoldus.in>
Authored: Thu Jun 21 11:29:37 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Mon Jun 25 21:35:49 2018 +0530

----------------------------------------------------------------------
 .../StandardPartitionTableQueryTestCase.scala               | 9 +--------
 .../management/CarbonAlterTableCompactionCommand.scala      | 5 +++--
 .../command/schema/CarbonAlterTableSetCommand.scala         | 4 +++-
 .../command/schema/CarbonAlterTableUnsetCommand.scala       | 4 +++-
 .../main/scala/org/apache/spark/util/AlterTableUtil.scala   | 5 ++---
 5 files changed, 12 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c6b2b6a2/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
index a6e7c32..87adaae 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableQueryTestCase.scala
@@ -248,17 +248,10 @@ class StandardPartitionTableQueryTestCase extends QueryTest with BeforeAndAfterA
 test("Creation of partition table should fail if the colname in table schema and partition column is same even if both are case sensitive"){
 
   val exception = intercept[Exception]{
-    sql("CREATE TABLE uniqdata_char2(name char,id int) partitioned by (NAME char)stored by 'carbondata' ")
+    sql("CREATE TABLE uniqdata_char2(name char(10),id int) partitioned by (NAME char(10))stored by 'carbondata' ")
   }
-  if(Spark2TestQueryExecutor.spark.version.startsWith("2.1.0")){
     assert(exception.getMessage.contains("Operation not allowed: Partition columns should not be " +
                                          "specified in the schema: [\"name\"]"))
-  }
-  else{
-    //spark 2.2 allow creating char data type only with digits thats why this assert is here as it will throw this exception
-    assert(exception.getMessage.contains("DataType char is not supported"))
-
-  }
 }
 
   test("Creation of partition table should fail for both spark version with same exception when char data type is created with specified digit and colname in table schema and partition column is same even if both are case sensitive"){

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c6b2b6a2/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 6b43107..9fdeb9d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.{AlterTableModel, AtomicRunnableCommand, CarbonMergerMapping, CompactionModel}
-import org.apache.spark.sql.hive.{CarbonRelation}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalog}
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.AlterTableUtil
@@ -325,7 +325,8 @@ case class CarbonAlterTableCompactionCommand(
           tableIdentifier,
           Map("streaming" -> "false"),
           Seq.empty,
-          true)(sparkSession)
+          true)(sparkSession,
+          sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog])
         // 5. remove checkpoint
         FileFactory.deleteAllFilesOfDir(
           new File(CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath)))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c6b2b6a2/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableSetCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableSetCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableSetCommand.scala
index ffd69df..51c0e6e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableSetCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableSetCommand.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.command.schema
 import org.apache.spark.sql.{Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.hive.CarbonSessionCatalog
 import org.apache.spark.util.AlterTableUtil
 
 private[sql] case class CarbonAlterTableSetCommand(
@@ -37,7 +38,8 @@ private[sql] case class CarbonAlterTableSetCommand(
       tableIdentifier,
       properties,
       Nil,
-      set = true)(sparkSession)
+      set = true)(sparkSession,
+      sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog])
     Seq.empty
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c6b2b6a2/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableUnsetCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableUnsetCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableUnsetCommand.scala
index d5bdd80..2490f9e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableUnsetCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableUnsetCommand.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.command.schema
 import org.apache.spark.sql.{Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.hive.CarbonSessionCatalog
 import org.apache.spark.util.AlterTableUtil
 
 
@@ -36,7 +37,8 @@ private[sql] case class CarbonAlterTableUnsetCommand(
 
   override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
     AlterTableUtil.modifyTableProperties(tableIdentifier, Map.empty[String, String],
-      propKeys, false)(sparkSession)
+      propKeys, false)(sparkSession,
+      sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog])
     Seq.empty
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c6b2b6a2/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index ac2bf9f..c291ae2 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -309,7 +309,7 @@ object AlterTableUtil {
    */
   def modifyTableProperties(tableIdentifier: TableIdentifier, properties: Map[String, String],
       propKeys: Seq[String], set: Boolean)
-    (sparkSession: SparkSession): Unit = {
+    (sparkSession: SparkSession, catalog: CarbonSessionCatalog): Unit = {
     val tableName = tableIdentifier.table
     val dbName = tableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
     LOGGER.audit(s"Alter table properties request has been received for $dbName.$tableName")
@@ -362,8 +362,7 @@ object AlterTableUtil {
       val (tableIdentifier, schemParts, cols) = updateSchemaInfo(carbonTable,
         schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
         thriftTable)(sparkSession)
-      sparkSession.asInstanceOf[CarbonSession].sessionState.catalog
-        .asInstanceOf[CarbonSessionCatalog].alterTable(tableIdentifier, schemParts, cols)
+      catalog.alterTable(tableIdentifier, schemParts, cols)
       sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
       LOGGER.info(s"Alter table properties is successful for table $dbName.$tableName")
       LOGGER.audit(s"Alter table properties is successful for table $dbName.$tableName")