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

[carbondata] branch master updated: [CARBONDATA-3251] Fix spark-2.1 UT errors

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

qiangcai 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 5a7dc79  [CARBONDATA-3251] Fix spark-2.1 UT errors
5a7dc79 is described below

commit 5a7dc791a26113c05dd99d75ae0c52758d4ef2b8
Author: xubo245 <60...@qq.com>
AuthorDate: Tue Jan 15 11:12:21 2019 +0800

    [CARBONDATA-3251] Fix spark-2.1 UT errors
    
    This closes #3075
---
 .../examples/sdk/CarbonReaderExample.java          | 237 +++++++++++----------
 .../apache/carbondata/examples/HiveExample.scala   |   4 +-
 .../testsuite/dataload/TestLoadDataGeneral.scala   |   7 +-
 .../ValueCompressionDataTypeTestCase.scala         |  22 +-
 .../datasource/SparkCarbonDataSourceTest.scala     |  46 ++--
 .../AlterTableColumnRenameTestCase.scala           |   7 +-
 6 files changed, 179 insertions(+), 144 deletions(-)

diff --git a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
index dab86e6..edda539 100644
--- a/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
+++ b/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java
@@ -19,13 +19,12 @@ package org.apache.carbondata.examples.sdk;
 
 import java.io.File;
 import java.io.FilenameFilter;
+import java.io.IOException;
 import java.sql.Date;
 import java.sql.Timestamp;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.commons.io.FileUtils;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.sdk.file.CarbonReader;
 import org.apache.carbondata.sdk.file.CarbonSchemaReader;
@@ -33,127 +32,139 @@ import org.apache.carbondata.sdk.file.CarbonWriter;
 import org.apache.carbondata.sdk.file.Field;
 import org.apache.carbondata.sdk.file.Schema;
 
+import org.apache.commons.io.FileUtils;
+
 /**
  * Example fo CarbonReader with close method
  * After readNextRow of CarbonReader, User should close the reader,
  * otherwise main will continue run some time
  */
 public class CarbonReaderExample {
-    public static void main(String[] args) {
-        String path = "./testWriteFiles";
-        try {
-            FileUtils.deleteDirectory(new File(path));
+  public static void main(String[] args) {
+    String path = "./testWriteFiles";
+    try {
+      FileUtils.deleteDirectory(new File(path));
+      CarbonProperties.getInstance()
+          .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+              CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+          .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+              CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+      Field[] fields = new Field[11];
+      fields[0] = new Field("stringField", DataTypes.STRING);
+      fields[1] = new Field("shortField", DataTypes.SHORT);
+      fields[2] = new Field("intField", DataTypes.INT);
+      fields[3] = new Field("longField", DataTypes.LONG);
+      fields[4] = new Field("doubleField", DataTypes.DOUBLE);
+      fields[5] = new Field("boolField", DataTypes.BOOLEAN);
+      fields[6] = new Field("dateField", DataTypes.DATE);
+      fields[7] = new Field("timeField", DataTypes.TIMESTAMP);
+      fields[8] = new Field("decimalField", DataTypes.createDecimalType(8, 2));
+      fields[9] = new Field("varcharField", DataTypes.VARCHAR);
+      fields[10] = new Field("arrayField", DataTypes.createArrayType(DataTypes.STRING));
+      CarbonWriter writer = CarbonWriter.builder()
+          .outputPath(path)
+          .withLoadOption("complex_delimiter_level_1", "#")
+          .withCsvInput(new Schema(fields))
+          .writtenBy("CarbonReaderExample")
+          .build();
 
-            Field[] fields = new Field[11];
-            fields[0] = new Field("stringField", DataTypes.STRING);
-            fields[1] = new Field("shortField", DataTypes.SHORT);
-            fields[2] = new Field("intField", DataTypes.INT);
-            fields[3] = new Field("longField", DataTypes.LONG);
-            fields[4] = new Field("doubleField", DataTypes.DOUBLE);
-            fields[5] = new Field("boolField", DataTypes.BOOLEAN);
-            fields[6] = new Field("dateField", DataTypes.DATE);
-            fields[7] = new Field("timeField", DataTypes.TIMESTAMP);
-            fields[8] = new Field("decimalField", DataTypes.createDecimalType(8, 2));
-            fields[9] = new Field("varcharField", DataTypes.VARCHAR);
-            fields[10] = new Field("arrayField", DataTypes.createArrayType(DataTypes.STRING));
-            CarbonWriter writer = CarbonWriter.builder()
-                .outputPath(path)
-                .withLoadOption("complex_delimiter_level_1", "#")
-                .withCsvInput(new Schema(fields))
-                .writtenBy("CarbonReaderExample")
-                .build();
+      for (int i = 0; i < 10; i++) {
+        String[] row2 = new String[]{
+            "robot" + (i % 10),
+            String.valueOf(i % 10000),
+            String.valueOf(i),
+            String.valueOf(Long.MAX_VALUE - i),
+            String.valueOf((double) i / 2),
+            String.valueOf(true),
+            "2019-03-02",
+            "2019-02-12 03:03:34",
+            "12.345",
+            "varchar",
+            "Hello#World#From#Carbon"
+        };
+        writer.write(row2);
+      }
+      writer.close();
 
-            for (int i = 0; i < 10; i++) {
-                String[] row2 = new String[]{
-                    "robot" + (i % 10),
-                    String.valueOf(i%10000),
-                    String.valueOf(i),
-                    String.valueOf(Long.MAX_VALUE - i),
-                    String.valueOf((double) i / 2),
-                    String.valueOf(true),
-                    "2019-03-02",
-                    "2019-02-12 03:03:34",
-                    "12.345",
-                    "varchar",
-                    "Hello#World#From#Carbon"
-                };
-                writer.write(row2);
-            }
-            writer.close();
-
-            File[] dataFiles = new File(path).listFiles(new FilenameFilter() {
-                @Override
-                public boolean accept(File dir, String name) {
-                    if (name == null) {
-                        return false;
-                    }
-                    return name.endsWith("carbonindex");
-                }
-            });
-            if (dataFiles == null || dataFiles.length < 1) {
-                throw new RuntimeException("Carbon index file not exists.");
-            }
-            Schema schema = CarbonSchemaReader
-                .readSchema(dataFiles[0].getAbsolutePath())
-                .asOriginOrder();
-            // Transform the schema
-            String[] strings = new String[schema.getFields().length];
-            for (int i = 0; i < schema.getFields().length; i++) {
-                strings[i] = (schema.getFields())[i].getFieldName();
-            }
+      File[] dataFiles = new File(path).listFiles(new FilenameFilter() {
+        @Override
+        public boolean accept(File dir, String name) {
+          if (name == null) {
+            return false;
+          }
+          return name.endsWith("carbonindex");
+        }
+      });
+      if (dataFiles == null || dataFiles.length < 1) {
+        throw new RuntimeException("Carbon index file not exists.");
+      }
+      Schema schema = CarbonSchemaReader
+          .readSchema(dataFiles[0].getAbsolutePath())
+          .asOriginOrder();
+      // Transform the schema
+      String[] strings = new String[schema.getFields().length];
+      for (int i = 0; i < schema.getFields().length; i++) {
+        strings[i] = (schema.getFields())[i].getFieldName();
+      }
 
-            // Read data
-            CarbonReader reader = CarbonReader
-                .builder(path, "_temp")
-                .projection(strings)
-                .build();
+      // Read data
+      CarbonReader reader = CarbonReader
+          .builder(path, "_temp")
+          .projection(strings)
+          .build();
 
-            System.out.println("\nData:");
-            long day = 24L * 3600 * 1000;
-            int i = 0;
-            while (reader.hasNext()) {
-                Object[] row = (Object[]) reader.readNextRow();
-                System.out.println(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t",
-                    i, row[0], row[1], row[2], row[3], row[4], row[5],
-                    new Date((day * ((int) row[6]))), new Timestamp((long) row[7] / 1000),
-                    row[8], row[9]
-                ));
-                Object[] arr = (Object[]) row[10];
-                for (int j = 0; j < arr.length; j++) {
-                    System.out.print(arr[j] + " ");
-                }
-                assert (arr[0].equals("Hello"));
-                assert (arr[3].equals("Carbon"));
-                System.out.println();
-                i++;
-            }
-            reader.close();
+      System.out.println("\nData:");
+      long day = 24L * 3600 * 1000;
+      int i = 0;
+      while (reader.hasNext()) {
+        Object[] row = (Object[]) reader.readNextRow();
+        System.out.println(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t",
+            i, row[0], row[1], row[2], row[3], row[4], row[5],
+            new Date((day * ((int) row[6]))), new Timestamp((long) row[7] / 1000),
+            row[8], row[9]
+        ));
+        Object[] arr = (Object[]) row[10];
+        for (int j = 0; j < arr.length; j++) {
+          System.out.print(arr[j] + " ");
+        }
+        assert (arr[0].equals("Hello"));
+        assert (arr[3].equals("Carbon"));
+        System.out.println();
+        i++;
+      }
+      reader.close();
 
-            // Read data
-            CarbonReader reader2 = CarbonReader
-                .builder(path, "_temp")
-                .build();
+      // Read data
+      CarbonReader reader2 = CarbonReader
+          .builder(path, "_temp")
+          .build();
 
-            System.out.println("\nData:");
-            i = 0;
-            while (reader2.hasNext()) {
-                Object[] row = (Object[]) reader2.readNextRow();
-                System.out.print(String.format("%s\t%s\t%s\t%s\t%s\t",
-                    i, row[0], new Date((day * ((int) row[1]))), new Timestamp((long) row[2] / 1000),
-                    row[3]));
-                Object[] arr = (Object[]) row[4];
-                for (int j = 0; j < arr.length; j++) {
-                    System.out.print(arr[j] + " ");
-                }
-                System.out.println(String.format("\t%s\t%s\t%s\t%s\t%s\t%s\t",
-                    row[5], row[6], row[7], row[8], row[9], row[10]));
-                i++;
-            }
-            reader2.close();
-            FileUtils.deleteDirectory(new File(path));
-        } catch (Throwable e) {
-            e.printStackTrace();
-            System.out.println(e.getMessage());
+      System.out.println("\nData:");
+      i = 0;
+      while (reader2.hasNext()) {
+        Object[] row = (Object[]) reader2.readNextRow();
+        System.out.print(String.format("%s\t%s\t%s\t%s\t%s\t",
+            i, row[0], new Date((day * ((int) row[1]))), new Timestamp((long) row[2] / 1000),
+            row[3]));
+        Object[] arr = (Object[]) row[4];
+        for (int j = 0; j < arr.length; j++) {
+          System.out.print(arr[j] + " ");
         }
+        System.out.println(String.format("\t%s\t%s\t%s\t%s\t%s\t%s\t",
+            row[5], row[6], row[7], row[8], row[9], row[10]));
+        i++;
+      }
+      reader2.close();
+    } catch (Throwable e) {
+      e.printStackTrace();
+      assert (false);
+      System.out.println(e.getMessage());
+    } finally {
+      try {
+        FileUtils.deleteDirectory(new File(path));
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
     }
+  }
 }
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala
index 69678a8..b50e763 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/HiveExample.scala
@@ -37,7 +37,7 @@ object HiveExample {
     exampleBody(carbonSession, CarbonProperties.getStorePath
       + CarbonCommonConstants.FILE_SEPARATOR
       + CarbonCommonConstants.DATABASE_DEFAULT_NAME)
-    carbonSession.close()
+    carbonSession.stop()
 
     System.exit(0)
   }
@@ -64,8 +64,6 @@ object HiveExample {
 
     carbonSession.sql("SELECT * FROM HIVE_CARBON_EXAMPLE").show()
 
-    carbonSession.stop()
-
     try {
       Class.forName(driverName)
     }
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index 561adba..8361862 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -22,13 +22,12 @@ import java.math.BigDecimal
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql.Row
+import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterEach
 
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.spark.sql.test.util.QueryTest
-
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datamap.Segment
 import org.apache.carbondata.core.util.CarbonProperties
@@ -192,7 +191,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
       sql(s"load data local inpath '$testdata' into table load32000bytes OPTIONS ('FILEHEADER'='name')")
     }.getMessage.contains("DataLoad failure: Dataload failed, String size cannot exceed 32000 bytes"))
 
-    val source = scala.io.Source.fromFile(testdata)
+    val source = scala.io.Source.fromFile(testdata, CarbonCommonConstants.DEFAULT_CHARSET)
     val data = source.mkString
 
     intercept[Exception] {
@@ -216,7 +215,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
       sql(s"load data local inpath '$testdata' into table load32000bytes OPTIONS ('FILEHEADER'='name')")
     }.getMessage.contains("generate global dictionary failed, Dataload failed, String size cannot exceed 32000 bytes"))
 
-    val source = scala.io.Source.fromFile(testdata)
+    val source = scala.io.Source.fromFile(testdata, CarbonCommonConstants.DEFAULT_CHARSET)
     val data = source.mkString
 
     intercept[Exception] {
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala
index 6a112f1..2136efd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ValueCompressionDataTypeTestCase.scala
@@ -22,19 +22,19 @@ import org.apache.carbondata.core.datastore.impl.FileFactory.FileType
 import org.apache.spark.sql.test.util.QueryTest
 
 class ValueCompressionDataTypeTestCase extends QueryTest with BeforeAndAfterAll {
-  val tempDirPath = s"$resourcesPath/temp"
+  val tempDirPath = s"$resourcesPath/tempdir"
 
   override def beforeAll {
     FileFactory.mkdirs(tempDirPath,FileType.LOCAL)
   }
 
   test("ActualDataType:double,ChangedDatatype:Short,CompressionType:NonDecimalMaxMin") {
-    val tempFilePath = s"$resourcesPath/temp/double2short.csv"
+    val tempFilePath = s"$tempDirPath/double2short.csv"
     try {
       sql("CREATE TABLE double2short (name String, value double) STORED BY 'org.apache.carbondata.format'")
       sql("CREATE TABLE double2short_hive (name String, value double)row format delimited fields terminated by ','")
       val data ="a,3.141111\nb,3.141212\nc,3.141313\nd,3.141515\ne,3.141616\nf,3.141616\ng,3.141717\nh,3.141818";
-      writedata(tempFilePath, data)
+      writeData(tempFilePath, data)
       sql(s"LOAD data local inpath '${tempFilePath}' into table double2short options('fileheader'='name,value')")
       sql(s"LOAD data local inpath '${tempFilePath}' into table double2short_hive")
       checkAnswer(sql("select * from double2short"),
@@ -49,14 +49,14 @@ class ValueCompressionDataTypeTestCase extends QueryTest with BeforeAndAfterAll
       deleteFile(tempFilePath)
     }
   }
-  
+
   test("ActualDataType:double,ChangedDatatype:byte,CompressionType:NonDecimalMaxMin") {
-    val tempFilePath = s"$resourcesPath/temp/double2byte.csv"
+    val tempFilePath = s"$tempDirPath/double2byte.csv"
     try {
       sql("CREATE TABLE double2byte (name String, value double) STORED BY 'org.apache.carbondata.format'")
       sql("CREATE TABLE double2byte_hive (name String, value double)row format delimited fields terminated by ','")
       val data ="a,4.200001\nb,4.200009";
-      writedata(tempFilePath, data)
+      writeData(tempFilePath, data)
       sql(s"LOAD data local inpath '${tempFilePath}' into table double2byte options('fileheader'='name,value')")
       sql(s"LOAD data local inpath '${tempFilePath}' into table double2byte_hive")
       checkAnswer(sql("select * from double2byte"),
@@ -73,14 +73,14 @@ class ValueCompressionDataTypeTestCase extends QueryTest with BeforeAndAfterAll
   }
 
   test("When the values of Double datatype are negative values") {
-    val tempFilePath = s"$resourcesPath/temp/doubleISnegtive.csv"
+    val tempFilePath = s"$tempDirPath/doubleISnegtive.csv"
     try {
       sql("drop table if exists doubleISnegtive")
       sql("drop table if exists doubleISnegtive_hive")
       sql("CREATE TABLE doubleISnegtive (name String, value double) STORED BY 'org.apache.carbondata.format'")
       sql("CREATE TABLE doubleISnegtive_hive (name String, value double)row format delimited fields terminated by ','")
       val data ="a,-7489.7976000000\nb,-11234567489.797\nc,-11234567489.7\nd,-1.2\ne,-2\nf,-11234567489.7976000000\ng,-11234567489.7976000000"
-      writedata(tempFilePath, data)
+      writeData(tempFilePath, data)
       sql(s"LOAD data local inpath '${tempFilePath}' into table doubleISnegtive options('fileheader'='name,value')")
       sql(s"LOAD data local inpath '${tempFilePath}' into table doubleISnegtive_hive")
 
@@ -97,14 +97,14 @@ class ValueCompressionDataTypeTestCase extends QueryTest with BeforeAndAfterAll
   }
 
   test("When the values of Double datatype have both postive and negative values") {
-    val tempFilePath = s"$resourcesPath/temp/doublePAN.csv"
+    val tempFilePath = s"$tempDirPath/doublePAN.csv"
     try {
       sql("drop table if exists doublePAN")
       sql("drop table if exists doublePAN_hive")
       sql("CREATE TABLE doublePAN (name String, value double) STORED BY 'org.apache.carbondata.format'")
       sql("CREATE TABLE doublePAN_hive (name String, value double)row format delimited fields terminated by ','")
       val data ="a,-7489.7976000000\nb,11234567489.797\nc,-11234567489.7\nd,-1.2\ne,2\nf,-11234567489.7976000000\ng,11234567489.7976000000"
-      writedata(tempFilePath, data)
+      writeData(tempFilePath, data)
       sql(s"LOAD data local inpath '${tempFilePath}' into table doublePAN options('fileheader'='name,value')")
       sql(s"LOAD data local inpath '${tempFilePath}' into table doublePAN_hive")
 
@@ -120,7 +120,7 @@ class ValueCompressionDataTypeTestCase extends QueryTest with BeforeAndAfterAll
     }
   }
 
-  def writedata(filePath: String, data: String) = {
+  def writeData(filePath: String, data: String) = {
     val dis = FileFactory.getDataOutputStream(filePath, FileFactory.getFileType(filePath))
     dis.writeBytes(data.toString())
     dis.close()
diff --git a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
index f89c1a8..329a250 100644
--- a/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
+++ b/integration/spark-datasource/src/test/scala/org/apache/spark/sql/carbondata/datasource/SparkCarbonDataSourceTest.scala
@@ -16,7 +16,6 @@
  */
 package org.apache.spark.sql.carbondata.datasource
 
-
 import java.io.File
 import java.util
 
@@ -39,7 +38,6 @@ import org.apache.carbondata.sdk.file.{CarbonWriter, Field, Schema}
 
 class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
 
-
   test("test write using dataframe") {
     import spark.implicits._
     val df = spark.sparkContext.parallelize(1 to 10)
@@ -96,7 +94,7 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       sql("INSERT INTO carbon_table SELECT * FROM test_parquet")
       TestUtil.checkAnswer(sql("SELECT * FROM carbon_table WHERE c1='a1'"),
         sql("SELECT * FROM test_parquet WHERE c1='a1'"))
-      if (!sparkContext.version.startsWith("2.1")) {
+      if (!SparkUtil.isSparkVersionEqualTo("2.1")) {
         val mapSize = DataMapStoreManager.getInstance().getAllDataMaps.size()
         DataMapStoreManager.getInstance()
           .clearDataMaps(AbsoluteTableIdentifier.from(warehouse1 + "/carbon_table"))
@@ -107,7 +105,11 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       assert(false)
     } catch {
       case e: Exception =>
-        assert(e.getMessage.contains("ALTER ADD COLUMNS does not support datasource table with type carbon."))
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+          assert(e.getMessage.contains("Operation not allowed: ALTER TABLE ADD COLUMNS"))
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+          assert(e.getMessage.contains("ALTER ADD COLUMNS does not support datasource table with type carbon."))
+        }
     } finally {
       sql("DROP TABLE IF EXISTS test_parquet")
       sql("DROP TABLE IF EXISTS carbon_table")
@@ -238,7 +240,7 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       sql("INSERT INTO carbon_table SELECT * FROM test_parquet")
       TestUtil.checkAnswer(sql("SELECT * FROM carbon_table WHERE c1='a1'"),
         sql("SELECT * FROM test_parquet WHERE c1='a1'"))
-      if (!sparkContext.version.startsWith("2.1")) {
+      if (!SparkUtil.isSparkVersionEqualTo("2.1")) {
         val mapSize = DataMapStoreManager.getInstance().getAllDataMaps.size()
         DataMapStoreManager.getInstance()
           .clearDataMaps(AbsoluteTableIdentifier.from(warehouse1 + "/carbon_table"))
@@ -249,7 +251,11 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       assert(false)
     } catch {
       case e: Exception =>
-        assert(e.getMessage.contains("ALTER TABLE CHANGE COLUMN is not supported for changing column"))
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+          assert(e.getMessage.contains("Operation not allowed: ALTER TABLE change"))
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+          assert(e.getMessage.contains("ALTER TABLE CHANGE COLUMN is not supported for changing column"))
+        }
     } finally {
       sql("DROP TABLE IF EXISTS test_parquet")
       sql("DROP TABLE IF EXISTS carbon_table")
@@ -279,8 +285,12 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       TestUtil.checkAnswer(sql("SELECT COUNT(*) FROM test_parquet2"), Seq(Row(2)))
     } catch {
       case e: Exception =>
-        e.printStackTrace()
-        assert(false)
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+          assert(e.getMessage.contains("ALTER TABLE test_parquet ADD COLUMNS"))
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+          e.printStackTrace()
+          assert(false)
+        }
     } finally {
       sql("DROP TABLE IF EXISTS test_parquet")
       sql("DROP TABLE IF EXISTS test_parquet2")
@@ -362,8 +372,12 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       checkAnswer(sql("SELECT COUNT(*) FROM test_parquet22"), Seq(Row(1)));
     } catch {
       case e: Exception =>
-        e.printStackTrace()
-        assert(false)
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+          assert(e.getMessage.contains("Operation not allowed: ALTER TABLE CHANGE"))
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+          e.printStackTrace()
+          assert(false)
+        }
     } finally {
       sql("DROP TABLE IF EXISTS test_parquet2")
       sql("DROP TABLE IF EXISTS test_parquet22")
@@ -386,7 +400,11 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       assert(false)
     } catch {
       case e: Exception =>
-        assert(e.getMessage.contains("ALTER TABLE CHANGE COLUMN is not supported for changing column"))
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+          assert(e.getMessage.contains("Operation not allowed: ALTER TABLE CHANGE"))
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+          assert(e.getMessage.contains("ALTER TABLE CHANGE COLUMN is not supported for changing column"))
+        }
     } finally {
       sql("DROP TABLE IF EXISTS test_parquet")
     }
@@ -398,7 +416,11 @@ class SparkCarbonDataSourceTest extends FunSuite with BeforeAndAfterAll {
       assert(false)
     } catch {
       case e: Exception =>
-        assert(e.getMessage.contains("ALTER TABLE CHANGE COLUMN is not supported for changing column"))
+        if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+          assert(e.getMessage.contains("Operation not allowed: ALTER TABLE CHANGE"))
+        } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+          assert(e.getMessage.contains("ALTER TABLE CHANGE COLUMN is not supported for changing column"))
+        }
     } finally {
       sql("DROP TABLE IF EXISTS test_parquet2")
     }
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 7b09278..d927724 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
@@ -18,6 +18,7 @@
 package org.apache.spark.carbondata.restructure.vectorreader
 
 import org.apache.spark.sql.common.util.Spark2QueryTest
+import org.apache.spark.util.SparkUtil
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.metadata.CarbonMetadata
@@ -312,7 +313,11 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
     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 ")
+    if (SparkUtil.isSparkVersionEqualTo("2.1")) {
+      checkExistence(sql("describe formatted rename"), false, "This column has comment ")
+    } else if (SparkUtil.isSparkVersionXandAbove("2.2")) {
+      checkExistence(sql("describe formatted rename"), true, "This column has comment ")
+    }
   }
 
   override def afterAll(): Unit = {