You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ma...@apache.org on 2018/12/31 03:16:11 UTC

carbondata git commit: [CARBONDATA-3202]update the schema to session catalog after add column, drop column and column rename

Repository: carbondata
Updated Branches:
  refs/heads/master 3e4638b33 -> b0733ecbf


[CARBONDATA-3202]update the schema to session catalog after add column, drop column and column rename

Problem:

1. For alter table rename, once we change the table name in carbon, we fire alter table rename DDL using hive client. But for add, drop and column
rename Spark does not support there features, but hive supports. so after rename, or add or drop column, the new updated schema is not updated in catalog.
2. after column rename column comment is not getting copied to renamed column

Solution:

1. We can directly call the spark API alterTableDataSchema by passing the updated schema, which in turn updates the shema in sessioncatalog. Since
this API is supported from spark2.1 onward, codes changes will be for spark 2.2 and spark2.3, behavior with spark2.1 remains the same.
2. while updating the catalog schema, if column has comment, put in column metadata

This closes #3027


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

Branch: refs/heads/master
Commit: b0733ecbf380d7956dee57a9048dd7537620744e
Parents: 3e4638b
Author: akashrn5 <ak...@gmail.com>
Authored: Thu Dec 27 11:31:44 2018 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Mon Dec 31 08:50:22 2018 +0530

----------------------------------------------------------------------
 .../ThriftWrapperSchemaConverterImpl.java       |  1 +
 .../sql/hive/CarbonInMemorySessionState.scala   | 15 +++---
 .../spark/sql/hive/CarbonSessionState.scala     | 55 +++++++++-----------
 .../spark/sql/hive/CarbonSessionUtil.scala      | 55 +++++++++++++++++++-
 .../CarbonAlterTableAddColumnCommand.scala      | 12 +++--
 ...terTableColRenameDataTypeChangeCommand.scala | 24 ++++++---
 .../CarbonAlterTableDropColumnCommand.scala     |  9 ++--
 .../spark/sql/hive/CarbonSessionCatalog.scala   | 26 ++++++---
 .../org/apache/spark/util/AlterTableUtil.scala  | 16 ++----
 .../spark/sql/hive/CarbonSessionState.scala     | 31 ++---------
 .../restructure/AlterTableRevertTestCase.scala  |  1 -
 .../AlterTableColumnRenameTestCase.scala        | 11 +++-
 12 files changed, 158 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 13f592f..dca7fa2 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -548,6 +548,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       if (sortColumns != null) {
         wrapperColumnSchema.setSortColumn(true);
       }
+      wrapperColumnSchema.setColumnProperties(externalColumnSchema.getColumnProperties());
     }
     wrapperColumnSchema.setFunction(externalColumnSchema.getAggregate_function());
     List<org.apache.carbondata.format.ParentColumnTableRelation> parentColumnTableRelation =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
index ba6aae5..da60fb0 100644
--- a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonInMemorySessionState.scala
@@ -35,6 +35,7 @@ import org.apache.spark.sql.parser.CarbonSparkSqlParser
 import org.apache.spark.sql.types.{StructField, StructType}
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 
+import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema => ColumnSchema}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.format.TableInfo
@@ -79,15 +80,13 @@ class InMemorySessionCatalog(
 
   override def alterTable(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
+      cols: Option[Seq[ColumnSchema]]): Unit = {
     // NOt Required in case of In-memory catalog
   }
 
   override def alterAddColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      newColumns: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
+      newColumns: Option[Seq[ColumnSchema]]): Unit = {
     val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
     val structType = catalogTable.schema
     var newStructType = structType
@@ -101,8 +100,7 @@ class InMemorySessionCatalog(
 
   override def alterDropColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      dropCols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
+      dropCols: Option[Seq[ColumnSchema]]): Unit = {
     val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
     val fields = catalogTable.schema.fields.filterNot { field =>
       dropCols.get.exists { col =>
@@ -112,10 +110,9 @@ class InMemorySessionCatalog(
     alterSchema(new StructType(fields), catalogTable, tableIdentifier)
   }
 
-  override def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
+  override def alterColumnChangeDataTypeOrRename(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      columns: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
+      columns: Option[Seq[ColumnSchema]]): Unit = {
     val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
     val a = catalogTable.schema.fields.flatMap { field =>
       columns.get.map { col =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
index f3168d7..0c60e71 100644
--- a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -34,6 +34,7 @@ import org.apache.spark.sql.internal.{SQLConf, SessionState}
 import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonLateDecodeRule, CarbonUDFTransformRule}
 import org.apache.spark.sql.parser.CarbonSparkSqlParser
 
+import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema => ColumnSchema}
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 /**
@@ -105,47 +106,41 @@ class CarbonHiveSessionCatalog(
       .asInstanceOf[HiveExternalCatalog].client
   }
 
-  def alterTableRename(oldTableIdentifier: TableIdentifier,
-      newTableIdentifier: TableIdentifier,
-      newTablePath: String): Unit = {
-    getClient().runSqlHive(
-      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ oldTableIdentifier.table } " +
-      s"RENAME TO ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table }")
-    getClient().runSqlHive(
-      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table} " +
-      s"SET SERDEPROPERTIES" +
-      s"('tableName'='${ newTableIdentifier.table }', " +
-      s"'dbName'='${ oldTableIdentifier.database.get }', 'tablePath'='${ newTablePath }')")
-  }
-
-  override def alterTable(tableIdentifier: TableIdentifier,
+  override def alterAddColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    getClient()
-      .runSqlHive(s"ALTER TABLE ${tableIdentifier.database.get}.${ tableIdentifier.table } " +
-                  s"SET TBLPROPERTIES(${ schemaParts })")
+      cols: Option[Seq[ColumnSchema]]): Unit = {
+    updateCatalogTableForAlter(tableIdentifier, schemaParts, cols)
   }
 
-  override def alterAddColumns(tableIdentifier: TableIdentifier,
+  override def alterDropColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    alterTable(tableIdentifier, schemaParts, cols)
+      cols: Option[Seq[ColumnSchema]]): Unit = {
+    updateCatalogTableForAlter(tableIdentifier, schemaParts, cols)
   }
 
-  override def alterDropColumns(tableIdentifier: TableIdentifier,
+  override def alterColumnChangeDataTypeOrRename(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    alterTable(tableIdentifier, schemaParts, cols)
+      cols: Option[Seq[ColumnSchema]]): Unit = {
+    updateCatalogTableForAlter(tableIdentifier, schemaParts, cols)
   }
 
-  override def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
+  /**
+   * This method alters table to set serde properties and updates the catalog table with new updated
+   * schema for all the alter operations like add column, drop column, change datatype or rename
+   * column
+   * @param tableIdentifier
+   * @param schemaParts
+   * @param cols
+   */
+  private def updateCatalogTableForAlter(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
+      cols: Option[Seq[ColumnSchema]]): Unit = {
     alterTable(tableIdentifier, schemaParts, cols)
+    CarbonSessionUtil
+      .alterExternalCatalogForTableWithUpdatedSchema(tableIdentifier,
+        cols,
+        schemaParts,
+        sparkSession)
   }
 
   override def createPartitions(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
index 1a22e99..2accbd6 100644
--- a/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
+++ b/integration/spark2/src/main/commonTo2.2And2.3/org/apache/spark/sql/hive/CarbonSessionUtil.scala
@@ -21,11 +21,16 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTablePartition, ExternalCatalogUtils}
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, SparkSession}
+import org.apache.spark.sql.types.MetadataBuilder
 import org.apache.spark.util.CarbonReflectionUtils
 
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.util.SparkTypeConverter
+
+import scala.collection.mutable.ArrayBuffer
 
 /**
  * This class refresh the relation from cache if the carbontable in
@@ -93,4 +98,52 @@ object CarbonSessionUtil {
     )
   }
 
+  /**
+   * This method alter the table for datatype change or column rename operation, and update the
+   * external catalog directly
+   *
+   * @param tableIdentifier tableIdentifier for table
+   * @param cols            all the column of table, which are updated with datatype change of
+   *                        new column name
+   * @param schemaParts     schemaParts
+   * @param sparkSession    sparkSession
+   */
+  def alterExternalCatalogForTableWithUpdatedSchema(tableIdentifier: TableIdentifier,
+      cols: Option[Seq[ColumnSchema]],
+      schemaParts: String,
+      sparkSession: SparkSession): Unit = {
+    val carbonTable = CarbonEnv.getCarbonTable(tableIdentifier)(sparkSession)
+    val colArray: scala.collection.mutable.ArrayBuffer[StructField] = ArrayBuffer()
+    cols.get.foreach(column =>
+      if (!column.isInvisible) {
+        val structFiled =
+          if (null != column.getColumnProperties &&
+              column.getColumnProperties.get("comment") != null) {
+            StructField(column.getColumnName,
+              SparkTypeConverter
+                .convertCarbonToSparkDataType(column,
+                  carbonTable),
+              true,
+              // update the column comment if present in the schema
+              new MetadataBuilder().putString("comment", column.getColumnProperties.get("comment"))
+                .build())
+          } else {
+            StructField(column.getColumnName,
+              SparkTypeConverter
+                .convertCarbonToSparkDataType(column,
+                  carbonTable))
+          }
+        colArray += structFiled
+      }
+    )
+    // Alter the data schema of a table identified by the provided database and table name.
+    // updated schema should contain all the existing data columns. This alterTableDataSchema API
+    // should be called after any alter with existing schema which updates the catalog table with
+    // new updated schema
+    sparkSession.sessionState.catalog.externalCatalog
+      .alterTableDataSchema(tableIdentifier.database.get,
+        tableIdentifier.table,
+        StructType(colArray))
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
index 9c92614..3fab2fb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -93,13 +93,17 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       schemaEvolutionEntry.setAdded(newCols.toList.asJava)
       val thriftTable = schemaConverter
         .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-      val (tableIdentifier, schemaParts, cols) = AlterTableUtil.updateSchemaInfo(
+      // carbon columns based on schema order
+      val carbonColumns = carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+        .collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema }
+      // sort the new columns based on schema order
+      val sortedColsBasedActualSchemaOrder = newCols.sortBy(a => a.getSchemaOrdinal)
+      val (tableIdentifier, schemaParts) = AlterTableUtil.updateSchemaInfo(
           carbonTable,
           schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
-          thriftTable,
-          Some(newCols))(sparkSession)
+          thriftTable)(sparkSession)
       sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog].alterAddColumns(
-        tableIdentifier, schemaParts, cols)
+        tableIdentifier, schemaParts, Some(carbonColumns ++ sortedColsBasedActualSchemaOrder))
       sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
       val alterTablePostExecutionEvent: AlterTableAddColumnPostEvent =
         new AlterTableAddColumnPostEvent(sparkSession,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
index 49d5093..49f651e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
@@ -214,7 +214,8 @@ private[sql] case class CarbonAlterTableColRenameDataTypeChangeCommand(
         carbonTable,
         tableInfo,
         addColumnSchema,
-        schemaEvolutionEntry)
+        schemaEvolutionEntry,
+        oldCarbonColumn.head)
       val alterTableColRenameAndDataTypeChangePostEvent
       : AlterTableColRenameAndDataTypeChangePostEvent =
         AlterTableColRenameAndDataTypeChangePostEvent(sparkSession, carbonTable,
@@ -262,13 +263,24 @@ private[sql] case class CarbonAlterTableColRenameDataTypeChangeCommand(
       carbonTable: CarbonTable,
       tableInfo: TableInfo,
       addColumnSchema: ColumnSchema,
-      schemaEvolutionEntry: SchemaEvolutionEntry): Unit = {
+      schemaEvolutionEntry: SchemaEvolutionEntry,
+      oldCarbonColumn: CarbonColumn): Unit = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    val a = List(schemaConverter.fromExternalToWrapperColumnSchema(addColumnSchema))
-    val (tableIdentifier, schemaParts, cols) = AlterTableUtil.updateSchemaInfo(
-      carbonTable, schemaEvolutionEntry, tableInfo, Some(a))(sparkSession)
+    // get the carbon column in schema order
+    val carbonColumns = carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+      .collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema }
+    // get the schema ordinal of the column for which the dataType changed or column is renamed
+    val schemaOrdinal = carbonColumns.indexOf(carbonColumns
+      .filter { column => column.getColumnName.equalsIgnoreCase(oldCarbonColumn.getColName) }.head)
+    // update the schema changed column at the specific index in carbonColumns based on schema order
+    carbonColumns
+      .update(schemaOrdinal, schemaConverter.fromExternalToWrapperColumnSchema(addColumnSchema))
+    val (tableIdentifier, schemaParts) = AlterTableUtil.updateSchemaInfo(
+      carbonTable,
+      schemaEvolutionEntry,
+      tableInfo)(sparkSession)
     sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog]
-      .alterColumnChangeDataType(tableIdentifier, schemaParts, cols)
+      .alterColumnChangeDataTypeOrRename(tableIdentifier, schemaParts, Some(carbonColumns))
     sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 9ef6fd8..7d5cb41 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -138,11 +138,14 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       val delCols = deletedColumnSchema.map { deleteCols =>
         schemaConverter.fromExternalToWrapperColumnSchema(deleteCols)
       }
-      val (tableIdentifier, schemaParts, cols) = AlterTableUtil.updateSchemaInfo(
+      val (tableIdentifier, schemaParts) = AlterTableUtil.updateSchemaInfo(
         carbonTable,
         schemaEvolutionEntry,
-        tableInfo,
-        Some(delCols))(sparkSession)
+        tableInfo)(sparkSession)
+      // get the columns in schema order and filter the dropped column in the column set
+      val cols = Some(carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+        .collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema}
+        .filterNot(column => delCols.contains(column)))
       sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog]
         .alterDropColumns(tableIdentifier, schemaParts, cols)
       sparkSession.catalog.refreshTable(tableIdentifier.quotedString)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionCatalog.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionCatalog.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionCatalog.scala
index f00739e..20d43df 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionCatalog.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionCatalog.scala
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable
 import org.apache.spark.sql.catalyst.expressions.Expression
 
 import org.apache.carbondata.common.annotations.{InterfaceAudience, InterfaceStability}
+import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema => ColumnSchema}
 
 /**
  * This interface defines those common api used by carbon for spark-2.1 and spark-2.2 integration,
@@ -77,7 +78,16 @@ trait CarbonSessionCatalog {
    */
   def alterTableRename(oldTableIdentifier: TableIdentifier,
       newTableIdentifier: TableIdentifier,
-      newTablePath: String): Unit
+      newTablePath: String): Unit = {
+    getClient().runSqlHive(
+      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ oldTableIdentifier.table } " +
+      s"RENAME TO ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table }")
+    getClient().runSqlHive(
+      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table } " +
+      s"SET SERDEPROPERTIES" +
+      s"('tableName'='${ newTableIdentifier.table }', " +
+      s"'dbName'='${ oldTableIdentifier.database.get }', 'tablePath'='${ newTablePath }')")
+  }
 
   /**
    * Below method will be used to update serd properties
@@ -87,7 +97,11 @@ trait CarbonSessionCatalog {
    */
   def alterTable(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]]): Unit
+      cols: Option[Seq[ColumnSchema]]): Unit = {
+    getClient()
+      .runSqlHive(s"ALTER TABLE ${ tableIdentifier.database.get }.${ tableIdentifier.table } " +
+                  s"SET TBLPROPERTIES(${ schemaParts })")
+  }
 
   /**
    * Below method will be used to add new column
@@ -97,7 +111,7 @@ trait CarbonSessionCatalog {
    */
   def alterAddColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]]): Unit
+      cols: Option[Seq[ColumnSchema]]): Unit
 
   /**
    * Below method will be used to drop column
@@ -107,7 +121,7 @@ trait CarbonSessionCatalog {
    */
   def alterDropColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]]): Unit
+      cols: Option[Seq[ColumnSchema]]): Unit
 
   /**
    * Below method will be used to alter data type of column in schema
@@ -115,7 +129,7 @@ trait CarbonSessionCatalog {
    * @param schemaParts schema parts
    * @param cols cols
    */
-  def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
+  def alterColumnChangeDataTypeOrRename(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]]): Unit
+      cols: Option[Seq[ColumnSchema]]): Unit
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/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 5fc1993..f74d3ce 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
@@ -107,13 +107,8 @@ object AlterTableUtil {
    */
   def updateSchemaInfo(carbonTable: CarbonTable,
       schemaEvolutionEntry: SchemaEvolutionEntry = null,
-      thriftTable: TableInfo,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]] =
-      None)
-    (sparkSession: SparkSession):
-    (TableIdentifier,
-      String,
-      Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]]) = {
+      thriftTable: TableInfo)
+    (sparkSession: SparkSession): (TableIdentifier, String) = {
     val dbName = carbonTable.getDatabaseName
     val tableName = carbonTable.getTableName
     CarbonEnv.getInstance(sparkSession).carbonMetaStore
@@ -127,7 +122,7 @@ object AlterTableUtil {
     val schema = CarbonEnv.getInstance(sparkSession).carbonMetaStore
       .lookupRelation(tableIdentifier)(sparkSession).schema.json
     val schemaParts = prepareSchemaJsonForAlterTable(sparkSession.sparkContext.getConf, schema)
-    (tableIdentifier, schemaParts, cols)
+    (tableIdentifier, schemaParts)
   }
 
   /**
@@ -403,11 +398,10 @@ object AlterTableUtil {
         // check if duplicate columns are present in both local dictionary include and exclude
         CarbonScalaUtil.validateDuplicateLocalDictIncludeExcludeColmns(tblPropertiesMap)
       }
-      val (tableIdentifier, schemParts, cols: Option[Seq[org.apache.carbondata.core.metadata
-      .schema.table.column.ColumnSchema]]) = updateSchemaInfo(
+      val (tableIdentifier, schemParts) = updateSchemaInfo(
         carbonTable = carbonTable,
         thriftTable = thriftTable)(sparkSession)
-      catalog.alterTable(tableIdentifier, schemParts, cols)
+      catalog.alterTable(tableIdentifier, schemParts, None)
       sparkSession.catalog.refreshTable(tableIdentifier.quotedString)
       // check and clear the block/blocklet cache
       checkAndClearBlockletCache(carbonTable,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
index 47b6b67..47feae0 100644
--- a/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.1/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -40,6 +40,7 @@ import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, Experime
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema => ColumnSchema}
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
@@ -85,45 +86,23 @@ class CarbonHiveSessionCatalog(
     carbonEnv
   }
 
-  def alterTableRename(oldTableIdentifier: TableIdentifier,
-      newTableIdentifier: TableIdentifier,
-      newTablePath: String): Unit = {
-    getClient().runSqlHive(
-      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ oldTableIdentifier.table }" +
-      s" RENAME TO ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table }")
-    getClient().runSqlHive(
-      s"ALTER TABLE ${ oldTableIdentifier.database.get }.${ newTableIdentifier.table }" +
-      s" SET SERDEPROPERTIES" +
-      s"('tableName'='${ newTableIdentifier.table }', " +
-      s"'dbName'='${ oldTableIdentifier.database.get }', 'tablePath'='${ newTablePath }')")
-  }
-
-  def alterTable(tableIdentifier: TableIdentifier,
-      schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
-  : Unit = {
-    getClient()
-      .runSqlHive(s"ALTER TABLE ${tableIdentifier.database.get}.${tableIdentifier.table } " +
-                  s"SET TBLPROPERTIES(${ schemaParts })")
-  }
-
   def alterAddColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+      cols: Option[Seq[ColumnSchema]])
   : Unit = {
     alterTable(tableIdentifier, schemaParts, cols)
   }
 
   def alterDropColumns(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+      cols: Option[Seq[ColumnSchema]])
   : Unit = {
     alterTable(tableIdentifier, schemaParts, cols)
   }
 
-  def alterColumnChangeDataType(tableIdentifier: TableIdentifier,
+  def alterColumnChangeDataTypeOrRename(tableIdentifier: TableIdentifier,
       schemaParts: String,
-      cols: Option[Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]])
+      cols: Option[Seq[ColumnSchema]])
   : Unit = {
     alterTable(tableIdentifier, schemaParts, cols)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
index 34abc92..dc3c365 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
@@ -22,7 +22,6 @@ import java.io.File
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.common.util.Spark2QueryTest
 import org.apache.spark.sql.test.TestQueryExecutor
-import org.apache.spark.util.AlterTableUtil
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.metadata.CarbonMetadata

http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0733ecb/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
index 710cd0d..7b09278 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
@@ -307,6 +307,14 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
     sql("drop table if exists biginttable")
   }
 
+  test("test column comment after column rename") {
+    dropTable()
+    createTable()
+    checkExistence(sql("describe formatted rename"), true, "This column has comment ")
+    sql("alter table rename change deptno classno bigint")
+    checkExistence(sql("describe formatted rename"), true, "This column has comment ")
+  }
+
   override def afterAll(): Unit = {
     dropTable()
   }
@@ -335,7 +343,8 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
   def createTable(): Unit = {
     sql(
       "CREATE TABLE rename (empno int, empname String, designation String, doj Timestamp, " +
-      "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
+      "workgroupcategory int, workgroupcategoryname String, deptno int comment \"This column " +
+      "has comment\", deptname String, " +
       "projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int," +
       "utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
   }