carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jack...@apache.org
Subject [04/38] incubator-carbondata git commit: reuse test case for integration module
Date Sat, 07 Jan 2017 16:36:38 GMT
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/AllDataTypesTestCaseSort.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/AllDataTypesTestCaseSort.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/AllDataTypesTestCaseSort.scala
deleted file mode 100644
index 5cbe93d..0000000
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/AllDataTypesTestCaseSort.scala
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.sortexpr
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util.CarbonHiveContext._
-import org.apache.spark.sql.common.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-/**
- * Test Class for sort expression query on multiple datatypes
- * @author N00902756
- *
- */
-
-class AllDataTypesTestCaseSort extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    sql("CREATE TABLE alldatatypestablesort (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int) STORED BY 'org.apache.carbondata.format'")
-    sql("LOAD DATA local inpath './src/test/resources/data.csv' INTO TABLE alldatatypestablesort OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')");
-
-    sql("CREATE TABLE alldatatypestablesort_hive (empno int, empname String, designation String, doj Timestamp, workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,utilization int,salary int)row format delimited fields terminated by ','")
-    sql("LOAD DATA local inpath './src/test/resources/datawithoutheader.csv' INTO TABLE alldatatypestablesort_hive");
-
-  }
-
-  test("select empno,empname,utilization,count(salary),sum(empno) from alldatatypestablesort where empname in ('arvind','ayushi') group by empno,empname,utilization order by empno") {
-    checkAnswer(
-      sql("select empno,empname,utilization,count(salary),sum(empno) from alldatatypestablesort where empname in ('arvind','ayushi') group by empno,empname,utilization order by empno"),
-      sql("select empno,empname,utilization,count(salary),sum(empno) from alldatatypestablesort_hive where empname in ('arvind','ayushi') group by empno,empname,utilization order by empno"))
-  }
-
-  override def afterAll {
-    sql("drop table alldatatypestablesort")
-    sql("drop table alldatatypestablesort_hive")
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/IntegerDataTypeTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/IntegerDataTypeTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/IntegerDataTypeTestCase.scala
deleted file mode 100644
index 3a97884..0000000
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/sortexpr/IntegerDataTypeTestCase.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.sortexpr
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util.CarbonHiveContext._
-import org.apache.spark.sql.common.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-/**
- * Test Class for sort expression query on int datatypes
- *
- * @author N00902756
- *
- */
-class IntegerDataTypeTestCase extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-    sql("CREATE TABLE inttypetablesort (empno int, workgroupcategory string, deptno int, projectcode int,attendance int) STORED BY 'org.apache.carbondata.format'")
-    sql("LOAD DATA local inpath './src/test/resources/data.csv' INTO TABLE inttypetablesort OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')")
-  }
-
-  test("select empno from inttypetablesort") {
-    checkAnswer(
-      sql("select empno from inttypetablesort"),
-      Seq(Row(11), Row(12), Row(13), Row(14), Row(15), Row(16), Row(17), Row(18), Row(19), Row(20)))
-  }
-
-  override def afterAll {
-    sql("drop table inttypetablesort")
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/windowsexpr/WindowsExprTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/windowsexpr/WindowsExprTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/windowsexpr/WindowsExprTestCase.scala
deleted file mode 100644
index 220e985..0000000
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/windowsexpr/WindowsExprTestCase.scala
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.windowsexpr
-
-import java.io.File
-
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.common.util.CarbonHiveContext._
-import org.apache.spark.sql.common.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-/**
-  * Test Class for all query on multiple datatypes
-  *
-  */
-class WindowsExprTestCase extends QueryTest with BeforeAndAfterAll {
-
-  override def beforeAll {
-
-    val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
-      .getCanonicalPath
-
-    sql("CREATE TABLE IF NOT EXISTS windowstable (ID double, date Timestamp, country String,name String, phonetype String, serialname String, salary double) STORED BY 'carbondata'");
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,"dd-MM-yyyy")
-    sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/windows.csv' INTO table windowstable options('DELIMITER'= ',' ,'QUOTECHAR'= '\"', 'FILEHEADER'= 'ID,date,country,name,phonetype,serialname,salary')");
-    sql("CREATE TABLE IF NOT EXISTS hivewindowstable (ID double, date Timestamp, country String,name String, phonetype String, serialname String, salary double) row format delimited fields terminated by ','");
-    sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/windows.csv' INTO table hivewindowstable ");
-
-  }
-
-  override def afterAll {
-    sql("drop table windowstable")
-    sql("drop table hivewindowstable")
-
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
-  }
-
-  test("SELECT country,name,salary FROM (SELECT country,name,salary,dense_rank() OVER (PARTITION BY country ORDER BY salary DESC) as rank FROM windowstable) tmp WHERE rank <= 2 order by country") {
-    checkAnswer(
-      sql("SELECT country,name,salary FROM (SELECT country,name,salary,dense_rank() OVER (PARTITION BY country ORDER BY salary DESC) as rank FROM windowstable) tmp WHERE rank <= 2 order by country"),
-      sql("SELECT country,name,salary FROM (SELECT country,name,salary,dense_rank() OVER (PARTITION BY country ORDER BY salary DESC) as rank FROM hivewindowstable) tmp WHERE rank <= 2 order by country"))
-  }
-
-  test("SELECT ID, country, SUM(salary) OVER (PARTITION BY country ) AS TopBorcT FROM windowstable") {
-    checkAnswer(
-      sql("SELECT ID, country, SUM(salary) OVER (PARTITION BY country ) AS TopBorcT FROM windowstable"),
-      sql("SELECT ID, country, SUM(salary) OVER (PARTITION BY country ) AS TopBorcT FROM hivewindowstable"))
-  }
-
-  test("SELECT country,name,salary,ROW_NUMBER() OVER (PARTITION BY country ORDER BY salary DESC) as rownum FROM windowstable") {
-    checkAnswer(
-      sql("SELECT country,name,salary,ROW_NUMBER() OVER (PARTITION BY country ORDER BY salary DESC) as rownum FROM windowstable"),
-      sql("SELECT country,name,salary,ROW_NUMBER() OVER (PARTITION BY country ORDER BY salary DESC) as rownum FROM hivewindowstable"))
-  }
-  
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
index 24e478f..ff8adca 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -20,8 +20,7 @@ package org.apache.carbondata.spark.util
 
 import java.io.File
 
-import org.apache.spark.sql.common.util.CarbonHiveContext.sql
-import org.apache.spark.sql.common.util.{CarbonHiveContext, QueryTest}
+import org.apache.spark.sql.common.util.QueryTest
 import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
 import org.scalatest.BeforeAndAfterAll
 
@@ -74,9 +73,8 @@ class AllDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
   def buildTestData() = {
-    pwd = new File(this.getClass.getResource("/").getPath + "/../../").getCanonicalPath
-    sampleAllDictionaryFile = pwd + "/src/test/resources/alldictionary/sample/20160423/1400_1405/*.dictionary"
-    complexAllDictionaryFile = pwd + "/src/test/resources/alldictionary/complex/20160423/1400_1405/*.dictionary"
+    sampleAllDictionaryFile = s"${resourcesPath}/alldictionary/sample/20160423/1400_1405/*.dictionary"
+    complexAllDictionaryFile = s"${resourcesPath}/alldictionary/complex/20160423/1400_1405/*.dictionary"
   }
 
   def buildTable() = {
@@ -106,16 +104,16 @@ class AllDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
   def buildRelation() = {
     val catalog = CarbonEnv.get.carbonMetastore
     sampleRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "sample")(CarbonHiveContext).asInstanceOf[CarbonRelation]
+      "sample")(sqlContext).asInstanceOf[CarbonRelation]
     complexRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "complextypes")(CarbonHiveContext).asInstanceOf[CarbonRelation]
+      "complextypes")(sqlContext).asInstanceOf[CarbonRelation]
   }
 
   test("Support generate global dictionary from all dictionary files") {
     val header = "id,name,city,age"
     val carbonLoadModel = buildCarbonLoadModel(sampleRelation, null, header, sampleAllDictionaryFile)
     GlobalDictionaryUtil
-      .generateGlobalDictionary(CarbonHiveContext,
+      .generateGlobalDictionary(sqlContext,
         carbonLoadModel,
         sampleRelation.tableMeta.storePath)
 
@@ -127,7 +125,7 @@ class AllDictionaryTestCase extends QueryTest with BeforeAndAfterAll {
     val header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC,locationinfo,proddate,gamePointId,contractNumber"
     val carbonLoadModel = buildCarbonLoadModel(complexRelation, null, header, complexAllDictionaryFile)
     GlobalDictionaryUtil
-      .generateGlobalDictionary(CarbonHiveContext,
+      .generateGlobalDictionary(sqlContext,
       carbonLoadModel,
       complexRelation.tableMeta.storePath)
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
index 39f7683..0738085 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
@@ -21,8 +21,7 @@ package org.apache.carbondata.spark.util
 import java.io.{BufferedWriter, File, FileWriter}
 import java.util.Random
 
-import org.apache.spark.sql.common.util.CarbonHiveContext.sql
-import org.apache.spark.sql.common.util.{CarbonHiveContext, QueryTest}
+import org.apache.spark.sql.common.util.QueryTest
 import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
 import org.scalatest.BeforeAndAfterAll
 
@@ -73,8 +72,7 @@ class AutoHighCardinalityIdentifyTestCase extends QueryTest with BeforeAndAfterA
   }
 
   def buildTestData() = {
-    val pwd = new File(this.getClass.getResource("/").getPath + "/../../").getCanonicalPath
-    filePath = pwd + "/target/highcarddata.csv"
+    filePath = s"${integrationPath}/spark/target/highcarddata.csv"
     val file = new File(filePath)
     val writer = new BufferedWriter(new FileWriter(file))
     writer.write("hc1,c2,c3")
@@ -85,6 +83,9 @@ class AutoHighCardinalityIdentifyTestCase extends QueryTest with BeforeAndAfterA
       writer.write("a" + i + "," +
           "b" + i%1000 + "," +
           i%1000000 + "\n")
+      if ( i % 10000 == 0) {
+        writer.flush()
+      }
     }
     writer.close
   }
@@ -113,14 +114,14 @@ class AutoHighCardinalityIdentifyTestCase extends QueryTest with BeforeAndAfterA
   def relation(tableName: String): CarbonRelation = {
     CarbonEnv.get.carbonMetastore
         .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-          tableName)(CarbonHiveContext)
+          tableName)(sqlContext)
         .asInstanceOf[CarbonRelation]
   }
   
   private def checkDictFile(table: CarbonTable) = {
     val tableIdentifier = new CarbonTableIdentifier(table.getDatabaseName,
         table.getFactTableName, "1")
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(CarbonHiveContext.hdfsCarbonBasePath,
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storeLocation,
         tableIdentifier)
     val newHc1 = table.getDimensionByName("highcard", "hc1")
     val newC2 = table.getDimensionByName("highcard", "c2")
@@ -162,7 +163,7 @@ class AutoHighCardinalityIdentifyTestCase extends QueryTest with BeforeAndAfterA
     // check dictionary file
     val tableIdentifier = new CarbonTableIdentifier(newTable.getDatabaseName,
         newTable.getFactTableName, "1")
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(CarbonHiveContext.hdfsCarbonBasePath,
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storeLocation,
         tableIdentifier)
     val newHc1 = newTable.getDimensionByName("colgrp_highcard", "hc1")
     val newC2 = newTable.getDimensionByName("colgrp_highcard", "c2")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
index c375f23..e176422 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -18,7 +18,7 @@
 package org.apache.carbondata.spark.util
 
 import org.apache.spark.sql.CarbonRelation
-import org.apache.spark.sql.common.util.CarbonHiveContext
+import org.apache.spark.sql.test.TestQueryExecutor
 
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier
@@ -43,8 +43,7 @@ object DictionaryTestCaseUtil {
     val columnIdentifier = new DictionaryColumnUniqueIdentifier(tableIdentifier,
       dimension.getColumnIdentifier, dimension.getDataType
     )
-    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier,
-      CarbonHiveContext.hdfsCarbonBasePath)
+    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier, TestQueryExecutor.storeLocation)
     assert(dict.getSurrogateKey(value) != CarbonCommonConstants.INVALID_SURROGATE_KEY)
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index e209ecc..625b74e 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -20,12 +20,10 @@ package org.apache.carbondata.spark.util
 
 import java.io.File
 
-import org.scalatest.BeforeAndAfterAll
-
-import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
-import org.apache.spark.sql.common.util.CarbonHiveContext
-import org.apache.spark.sql.common.util.CarbonHiveContext.sql
 import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.test.TestQueryExecutor
+import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
+import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.carbon.CarbonDataLoadSchema
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -54,19 +52,16 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
   var header2: String = _
 
   def buildTestData() = {
-    pwd = new File(this.getClass.getResource("/").getPath + "/../../").
-        getCanonicalPath.replace("\\", "/")
-    filePath = pwd + "/src/test/resources/sample.csv"
-    complexFilePath1 = pwd + "/src/test/resources/complexdata2.csv"
-    complexFilePath2 = pwd + "/src/test/resources/verticalDelimitedData.csv"
-    extColDictFilePath1 = "deviceInformationId:" + pwd +
-      "/src/test/resources/deviceInformationId.csv,"
-      "mobile.imei:" + pwd + "/src/test/resources/mobileimei.csv,"
-      "mac:" + pwd + "/src/test/resources/mac.csv,"
-      "locationInfo.ActiveCountry:" + pwd + "/src/test/resources/locationInfoActiveCountry.csv"
-    extColDictFilePath2 = "deviceInformationId:" + pwd +
-      "/src/test/resources/deviceInformationId2.csv"
-    extColDictFilePath3 = "channelsId:" + pwd + "/src/test/resources/channelsId.csv"
+
+    filePath = s"${resourcesPath}/sample.csv"
+    complexFilePath1 = s"${resourcesPath}/complexdata2.csv"
+    complexFilePath2 = s"${resourcesPath}/verticalDelimitedData.csv"
+    extColDictFilePath1 = s"deviceInformationId:${resourcesPath}/deviceInformationId.csv," +
+      s"mobile.imei:${resourcesPath}/mobileimei.csv," +
+      s"mac:${resourcesPath}/mac.csv," +
+      s"locationInfo.ActiveCountry:${resourcesPath}/locationInfoActiveCountry.csv"
+    extColDictFilePath2 = s"deviceInformationId:${resourcesPath}/deviceInformationId2.csv"
+    extColDictFilePath3 = s"channelsId:${resourcesPath}/channelsId.csv"
     header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC," +
       "locationinfo,proddate,gamePointId,contractNumber"
     header2 = "deviceInformationId|channelsId|contractNumber"
@@ -117,13 +112,13 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
   def buildRelation() = {
     val catalog = CarbonEnv.get.carbonMetastore
     extComplexRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "extComplextypes")(CarbonHiveContext)
+      "extComplextypes")(sqlContext)
       .asInstanceOf[CarbonRelation]
     verticalDelimiteRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "verticalDelimitedTable")(CarbonHiveContext)
+      "verticalDelimitedTable")(sqlContext)
       .asInstanceOf[CarbonRelation]
     loadSqlRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "loadSqlTest")(CarbonHiveContext)
+      "loadSqlTest")(sqlContext)
       .asInstanceOf[CarbonRelation]
   }
 
@@ -162,7 +157,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
     // load the first time
     var carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
       header, extColDictFilePath1)
-    GlobalDictionaryUtil.generateGlobalDictionary(CarbonHiveContext, carbonLoadModel,
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
       extComplexRelation.tableMeta.storePath)
     // check whether the dictionary is generated
     DictionaryTestCaseUtil.checkDictionary(
@@ -171,7 +166,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
     // load the second time
     carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
       header, extColDictFilePath2)
-    GlobalDictionaryUtil.generateGlobalDictionary(CarbonHiveContext, carbonLoadModel,
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
       extComplexRelation.tableMeta.storePath)
     // check the old dictionary and whether the new distinct value is generated
     DictionaryTestCaseUtil.checkDictionary(
@@ -184,7 +179,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
     //  when csv delimiter is comma
     var carbonLoadModel = buildCarbonLoadModel(extComplexRelation, complexFilePath1,
       header, extColDictFilePath3)
-    GlobalDictionaryUtil.generateGlobalDictionary(CarbonHiveContext, carbonLoadModel,
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
       extComplexRelation.tableMeta.storePath)
     // check whether the dictionary is generated
     DictionaryTestCaseUtil.checkDictionary(
@@ -193,7 +188,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
     //  when csv delimiter is not comma
     carbonLoadModel = buildCarbonLoadModel(verticalDelimiteRelation, complexFilePath2,
       header2, extColDictFilePath3, "|")
-    GlobalDictionaryUtil.generateGlobalDictionary(CarbonHiveContext, carbonLoadModel,
+    GlobalDictionaryUtil.generateGlobalDictionary(sqlContext, carbonLoadModel,
       verticalDelimiteRelation.tableMeta.storePath)
     // check whether the dictionary is generated
     DictionaryTestCaseUtil.checkDictionary(

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
index 4210574..f565eda 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
@@ -19,30 +19,25 @@
 package org.apache.carbondata.spark.util
 
 import java.io.File
-
-import java.util.concurrent.Executors
-import java.util.concurrent.Callable
+import java.util.concurrent.{Callable, Executors}
 
 import scala.collection.mutable.ListBuffer
 
-import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
-import org.apache.spark.sql.common.util.CarbonHiveContext
-import org.apache.spark.sql.common.util.CarbonHiveContext.sql
 import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.common.ext.PathFactory
 import org.apache.carbondata.core.carbon.CarbonDataLoadSchema
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.processing.constants.TableOptionConstant
 import org.apache.carbondata.processing.model.CarbonLoadModel
 
 class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAfterAll {
 
   var sampleRelation: CarbonRelation = _
-  var workDirectory: String = _
 
   def buildCarbonLoadModel(relation: CarbonRelation,
                            filePath: String,
@@ -69,15 +64,11 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
   }
 
   override def beforeAll {
-    buildTestData
     // second time comment this line
     buildTable
     buildRelation
   }
 
-  def buildTestData() = {
-    workDirectory = new File(this.getClass.getResource("/").getPath + "/../../").getCanonicalPath.replace("\\", "/")
-  }
   def buildTable() = {
     try {
       sql(
@@ -90,7 +81,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
   def buildRelation() = {
     val catalog = CarbonEnv.get.carbonMetastore
     sampleRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "employee")(CarbonHiveContext)
+      "employee")(sqlContext)
       .asInstanceOf[CarbonRelation]
   }
   def writedummydata(filePath: String, recCount: Int) = {
@@ -109,7 +100,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
     val files = new ListBuffer[String]()
     val loadModels = new ListBuffer[CarbonLoadModel]()
     for (i <- 0 until noOfFiles) {
-      val filePath: String = workDirectory + s"/src/test/resources/singlecolumn_${10 * (i + 1)}.csv"
+      val filePath: String = s"${integrationPath}/spark/target/singlecolumn_${10 * (i + 1)}.csv"
       files += filePath
       loadModels += buildCarbonLoadModel(sampleRelation, filePath, "empid")
       writedummydata(filePath, 10 * (i + 1))
@@ -161,7 +152,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
      var result = "Pass"
       try {
         GlobalDictionaryUtil
-          .generateGlobalDictionary(CarbonHiveContext,
+          .generateGlobalDictionary(sqlContext,
             loadModel,
             sampleRelation.tableMeta.storePath)
       } catch {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
index a7601ee..c2e3790 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
@@ -20,10 +20,8 @@ package org.apache.carbondata.spark.util
 
 import java.io.File
 
-import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
-import org.apache.spark.sql.common.util.CarbonHiveContext
-import org.apache.spark.sql.common.util.CarbonHiveContext.sql
 import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.carbon.CarbonDataLoadSchema
@@ -31,7 +29,6 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.processing.constants.TableOptionConstant
 import org.apache.carbondata.processing.model.CarbonLoadModel
 
-
 /**
   * Test Case for org.apache.carbondata.spark.util.GlobalDictionaryUtil
   */
@@ -42,7 +39,6 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
   var complexRelation: CarbonRelation = _
   var incrementalLoadTableRelation: CarbonRelation = _
   var filePath: String = _
-  var workDirectory: String = _
   var dimFilePath: String = _
   var complexfilePath: String = _
   var complexfilePath1: String = _
@@ -80,12 +76,11 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
   def buildTestData() = {
-    workDirectory = new File(this.getClass.getResource("/").getPath + "/../../").getCanonicalPath.replace("\\", "/")
-    filePath = workDirectory + "/src/test/resources/sample.csv"
-    dimFilePath = "dimTableSample:" + workDirectory + "/src/test/resources/dimTableSample.csv"
-    complexfilePath1 = workDirectory + "/src/test/resources/complexdata1.csv"
-    complexfilePath2 = workDirectory + "/src/test/resources/complexdata2.csv"
-    complexfilePath = workDirectory + "/src/test/resources/complexdata.csv"
+    filePath = s"${resourcesPath}/sample.csv"
+    dimFilePath = s"dimTableSample:${resourcesPath}/dimTableSample.csv"
+    complexfilePath1 = s"${resourcesPath}/complexdata1.csv"
+    complexfilePath2 = s"${resourcesPath}/complexdata2.csv"
+    complexfilePath = s"${resourcesPath}/complexdata.csv"
   }
 
   def buildTable() = {
@@ -139,16 +134,16 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
   def buildRelation() = {
     val catalog = CarbonEnv.get.carbonMetastore
     sampleRelation = catalog.lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      "sample")(CarbonHiveContext)
+      "sample")(sqlContext)
       .asInstanceOf[CarbonRelation]
     dimSampleRelation = catalog
-      .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), "dimSample")(CarbonHiveContext)
+      .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), "dimSample")(sqlContext)
       .asInstanceOf[CarbonRelation]
     complexRelation = catalog
-      .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), "complextypes")(CarbonHiveContext)
+      .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), "complextypes")(sqlContext)
       .asInstanceOf[CarbonRelation]
     incrementalLoadTableRelation = catalog
-      .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), "incrementalLoadTable")(CarbonHiveContext)
+      .lookupRelation1(Option(CarbonCommonConstants.DATABASE_DEFAULT_NAME), "incrementalLoadTable")(sqlContext)
       .asInstanceOf[CarbonRelation]
   }
 
@@ -156,8 +151,7 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
 
     val carbonLoadModel = buildCarbonLoadModel(sampleRelation, filePath, null)
     GlobalDictionaryUtil
-      .generateGlobalDictionary(CarbonHiveContext,
-        carbonLoadModel,
+      .generateGlobalDictionary(sqlContext, carbonLoadModel,
         sampleRelation.tableMeta.storePath
       )
 
@@ -174,8 +168,7 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
       "proddate,gamePointId,contractNumber"
     val carbonLoadModel = buildCarbonLoadModel(complexRelation, complexfilePath, header)
     GlobalDictionaryUtil
-      .generateGlobalDictionary(CarbonHiveContext,
-        carbonLoadModel,
+      .generateGlobalDictionary(sqlContext, carbonLoadModel,
         complexRelation.tableMeta.storePath
       )
   }
@@ -189,8 +182,7 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
       header
     )
     GlobalDictionaryUtil
-      .generateGlobalDictionary(CarbonHiveContext,
-        carbonLoadModel,
+      .generateGlobalDictionary(sqlContext, carbonLoadModel,
         sampleRelation.tableMeta.storePath
       )
     DictionaryTestCaseUtil.
@@ -202,8 +194,7 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
       header
     )
     GlobalDictionaryUtil
-      .generateGlobalDictionary(CarbonHiveContext,
-        carbonLoadModel,
+      .generateGlobalDictionary(sqlContext, carbonLoadModel,
         sampleRelation.tableMeta.storePath
       )
     DictionaryTestCaseUtil.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala b/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
index 39d1491..8c681a9 100644
--- a/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
@@ -22,6 +22,7 @@ import scala.collection.mutable.Map
 
 import org.apache.spark.sql.common.util.QueryTest
 import org.apache.spark.sql.execution.command.Field
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/spark/sql/common/util/CarbonHiveContext.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/CarbonHiveContext.scala b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/CarbonHiveContext.scala
deleted file mode 100644
index 1990070..0000000
--- a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/CarbonHiveContext.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.spark.sql.common.util
-
-import java.io.File
-
-import org.apache.spark.sql.CarbonContext
-import org.apache.spark.{SparkConf, SparkContext}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-
-class LocalSQLContext(val hdfsCarbonBasePath: String)
-  extends CarbonContext(new SparkContext(new SparkConf()
-    .setAppName("CarbonSpark")
-    .setMaster("local[2]")
-    .set("spark.sql.shuffle.partitions", "20")
-    .set("use_kettle_default", "true")),
-    hdfsCarbonBasePath,
-    hdfsCarbonBasePath) {
-
-}
-
-object CarbonHiveContext extends LocalSQLContext(new File("./target/test/").getCanonicalPath) {
-    sparkContext.setLogLevel("ERROR")
-    CarbonProperties.getInstance()
-      .addProperty("carbon.kettle.home", "../../processing/carbonplugins")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
-    CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
-        System.getProperty("java.io.tmpdir"))
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
index 44d3bfa..bd9b743 100644
--- a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
@@ -20,10 +20,11 @@ package org.apache.spark.sql.common.util
 import java.util.{Locale, TimeZone}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-
 import scala.collection.JavaConversions._
+
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.test.TestQueryExecutor
 import org.apache.spark.sql.{DataFrame, Row, SQLContext}
 
 class QueryTest extends PlanTest {
@@ -79,9 +80,17 @@ class QueryTest extends PlanTest {
   protected def checkAnswer(df: DataFrame, expectedAnswer: DataFrame): Unit = {
     checkAnswer(df, expectedAnswer.collect())
   }
+
+  def sql(sqlText: String): DataFrame = TestQueryExecutor.INSTANCE.sql(sqlText)
+
+  val sqlContext: SQLContext = TestQueryExecutor.INSTANCE.sqlContext
+  val storeLocation = TestQueryExecutor.storeLocation
+  val resourcesPath = TestQueryExecutor.resourcesPath
+  val integrationPath = TestQueryExecutor.integrationPath
 }
 
 object QueryTest {
+
   def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): String = {
     checkAnswer(df, expectedAnswer.toSeq) match {
       case Some(errorMessage) => errorMessage

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
index befa772..2879130 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
@@ -33,15 +33,16 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
     plan match {
       case LoadDataCommand(identifier, path, isLocal, isOverwrite, partition)
         if CarbonEnv.get.carbonMetastore.tableExists(identifier)(sparkSession) =>
-        ExecutedCommandExec(LoadTable(identifier.database, identifier.table, path, Seq(),
-          Map(), isOverwrite)) :: Nil
+        ExecutedCommandExec(LoadTable(identifier.database, identifier.table.toLowerCase, path,
+          Seq(), Map(), isOverwrite)) :: Nil
       case DropTableCommand(identifier, ifNotExists, isView, _)
         if CarbonEnv.get.carbonMetastore
           .isTablePathExists(identifier)(sparkSession) =>
         ExecutedCommandExec(
-          CarbonDropTableCommand(ifNotExists, identifier.database, identifier.table)) :: Nil
+          CarbonDropTableCommand(ifNotExists, identifier.database,
+            identifier.table.toLowerCase)) :: Nil
       case ShowLoadsCommand(databaseName, table, limit) =>
-        ExecutedCommandExec(ShowLoads(databaseName, table, limit, plan.output)) :: Nil
+        ExecutedCommandExec(ShowLoads(databaseName, table.toLowerCase, limit, plan.output)) :: Nil
       case createDb@CreateDatabaseCommand(dbName, ifNotExists, _, _, _) =>
         CarbonEnv.get.carbonMetastore.createDatabaseDirectory(dbName)
         ExecutedCommandExec(createDb) :: Nil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 8b66594..a38c77a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -23,6 +23,7 @@ import java.text.SimpleDateFormat
 import scala.collection.JavaConverters._
 import scala.language.implicitConversions
 
+import org.apache.commons.lang3.StringUtils
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
@@ -79,7 +80,7 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
 
   def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableModel.tableName
+    val tableName = alterTableModel.tableName.toLowerCase
     val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
     if (null == org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
       .getCarbonTable(databaseName + "_" + tableName)) {
@@ -311,6 +312,12 @@ case class LoadTable(
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
+  private def checkDefaultValue(value: String, default: String) = if (StringUtils.isEmpty(value)) {
+    default
+  } else {
+    value
+  }
+
   def run(sparkSession: SparkSession): Seq[Row] = {
     if (dataFrame.isDefined && !updateModel.isDefined) {
       val rdd = dataFrame.get.rdd
@@ -406,10 +413,10 @@ case class LoadTable(
       val dateFormat = options.getOrElse("dateformat", null)
       validateDateFormat(dateFormat, table)
       val maxColumns = options.getOrElse("maxcolumns", null)
-      carbonLoadModel.setMaxColumns(maxColumns)
-      carbonLoadModel.setEscapeChar(escapeChar)
-      carbonLoadModel.setQuoteChar(quoteChar)
-      carbonLoadModel.setCommentChar(commentchar)
+      carbonLoadModel.setMaxColumns(checkDefaultValue(maxColumns, null))
+      carbonLoadModel.setEscapeChar(checkDefaultValue(escapeChar, "\\"))
+      carbonLoadModel.setQuoteChar(checkDefaultValue(quoteChar, "\""))
+      carbonLoadModel.setCommentChar(checkDefaultValue(commentchar, "#"))
       carbonLoadModel.setDateFormat(dateFormat)
       carbonLoadModel
         .setSerializationNullFormat(
@@ -729,7 +736,7 @@ private[sql] case class DescribeCommandFormatted(
       .getDimensionByTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
     results ++= getColumnGroups(dimension.asScala.toList)
     results.map { case (name, dataType, comment) =>
-      Row(f"$name%-36s $dataType%-80s $comment%-72s")
+      Row(f"$name%-36s", f"$dataType%-80s", f"$comment%-72s")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
index d7a74f2..f020a1c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
@@ -399,7 +399,7 @@ class CarbonMetastore(conf: RuntimeConfig, val storePath: String) {
 
   def isTablePathExists(tableIdentifier: TableIdentifier)(sparkSession: SparkSession): Boolean = {
     val dbName = tableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
-    val tableName = tableIdentifier.table
+    val tableName = tableIdentifier.table.toLowerCase
 
     val tablePath = CarbonStorePath.getCarbonTablePath(this.storePath,
       new CarbonTableIdentifier(dbName, tableName, "")).getPath

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
----------------------------------------------------------------------
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 5c0bbbb..0c343e1 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
@@ -41,7 +41,8 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
             x
           case logicalPlan => logicalPlan
         }
-        case failureOrError => sys.error(failureOrError.toString)
+        case failureOrError =>
+          sys.error(failureOrError.toString)
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/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 3dabefa..6f07ec5 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
@@ -17,11 +17,13 @@
 package org.apache.spark.sql.parser
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable
 
 import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, ParseException, SqlBaseParser}
 import org.apache.spark.sql.catalyst.parser.ParserUtils._
 import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{CreateTableContext, TablePropertyListContext}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.trees.Origin
 import org.apache.spark.sql.execution.SparkSqlAstBuilder
 import org.apache.spark.sql.execution.command.{BucketFields, CreateTable, Field, TableModel}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
@@ -47,7 +49,9 @@ class CarbonSparkSqlParser(conf: SQLConf) extends AbstractSqlParser {
     try {
       super.parsePlan(sqlText)
     } catch {
-      case e: Throwable =>
+      case ce: MalformedCarbonCommandException =>
+        throw ce
+      case ex =>
         astBuilder.parser.parse(sqlText)
     }
   }
@@ -139,13 +143,16 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
           None
         }
       }
+
+      val tableProperties = mutable.Map[String, String]()
+      properties.foreach(tableProperties += _)
       // prepare table model of the collected tokens
       val tableModel: TableModel = parser.prepareTableModel(ifNotExists,
         name.database,
-        name.table,
+        name.table.toLowerCase,
         fields,
         Seq(),
-        properties.asJava.asScala,
+        tableProperties,
         bucketFields)
 
       CreateTable(tableModel)
@@ -164,7 +171,9 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
       operationNotAllowed(
         s"Values must be specified for key(s): ${ badKeys.mkString("[", ",", "]") }", ctx)
     }
-    props
+    props.map{ case (key, value) =>
+      (key.toLowerCase, value.toLowerCase)
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
new file mode 100644
index 0000000..0df98b8
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
@@ -0,0 +1,61 @@
+/*
+ * 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.spark.sql.test
+
+import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext}
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+/**
+ * This class is a sql executor of unit test case for spark version 2.x.
+ */
+
+class Spark2TestQueryExecutor extends TestQueryExecutorRegister {
+
+  override def sql(sqlText: String): DataFrame = Spark2TestQueryExecutor.spark.sql(sqlText)
+
+  override def sqlContext: SQLContext = Spark2TestQueryExecutor.spark.sqlContext
+}
+
+object Spark2TestQueryExecutor {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  LOGGER.info("use TestQueryExecutorImplV2")
+  CarbonProperties.getInstance()
+    .addProperty(CarbonCommonConstants.STORE_LOCATION, TestQueryExecutor.storeLocation)
+    .addProperty("carbon.kettle.home", TestQueryExecutor.kettleHome)
+    .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, TestQueryExecutor.timestampFormat)
+    .addProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
+      System.getProperty("java.io.tmpdir"))
+    .addProperty(CarbonCommonConstants.LOCK_TYPE, CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL)
+
+
+  import org.apache.spark.sql.CarbonSession._
+  val spark = SparkSession
+    .builder()
+    .master("local[2]")
+    .appName("CarbonExample")
+    .enableHiveSupport()
+    .config("spark.sql.warehouse.dir", TestQueryExecutor.warehouse)
+    .config("javax.jdo.option.ConnectionURL",
+      s"jdbc:derby:;databaseName=${TestQueryExecutor.metastoredb};create=true")
+    .getOrCreateCarbonSession()
+  spark.sparkContext.setLogLevel("ERROR")
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
----------------------------------------------------------------------
diff --git a/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000..607e962
--- /dev/null
+++ b/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1 @@
+org.apache.spark.sql.CarbonSource
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.test.TestQueryExecutorRegister
----------------------------------------------------------------------
diff --git a/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.test.TestQueryExecutorRegister b/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.test.TestQueryExecutorRegister
new file mode 100644
index 0000000..f713860
--- /dev/null
+++ b/integration/spark2/src/resources/META-INF/services/org.apache.spark.sql.test.TestQueryExecutorRegister
@@ -0,0 +1 @@
+org.apache.spark.sql.test.Spark2TestQueryExecutor
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/af2f204e/integration/spark2/src/test/resources/data.csv
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/resources/data.csv b/integration/spark2/src/test/resources/data.csv
deleted file mode 100644
index 4ff67da..0000000
--- a/integration/spark2/src/test/resources/data.csv
+++ /dev/null
@@ -1,11 +0,0 @@
-empno,empname,designation,doj,workgroupcategory,workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate,attendance,utilization,salary
-11,arvind,SE,17-01-2007,1,developer,10,network,928478,17-02-2007,29-11-2016,96,96.2,5040.56
-12,krithin,SSE,29-05-2008,1,developer,11,protocol,928378,29-06-2008,30-12-2016,85,95.1,7124.21
-13,madhan,TPL,07-07-2009,2,tester,10,network,928478,07-08-2009,30-12-2016,88,99,9054.235
-14,anandh,SA,29-12-2010,3,manager,11,protocol,928278,29-01-2011,29-06-2016,77,92.2,11248.25
-15,ayushi,SSA,09-11-2011,1,developer,12,security,928375,09-12-2011,29-05-2016,99,91.5,13245.48
-16,pramod,SE,14-10-2012,1,developer,13,configManagement,928478,14-11-2012,29-12-2016,86,93,5040.56
-17,gawrav,PL,22-09-2013,2,tester,12,security,928778,22-10-2013,15-11-2016,78,97.45,9574.24
-18,sibi,TL,15-08-2014,2,tester,14,Learning,928176,15-09-2014,29-05-2016,84,98.23,7245.25
-19,shivani,PL,12-05-2015,1,developer,10,network,928977,12-06-2015,12-11-2016,88,91.678,11254.24
-20,bill,PM,01-12-2015,3,manager,14,Learning,928479,01-01-2016,30-11-2016,75,94.22,13547.25


Mime
View raw message