You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2018/02/28 10:53:05 UTC

carbondata git commit: [CARBONDATA-2207] Fix testcases after using hive metastore

Repository: carbondata
Updated Branches:
  refs/heads/master 1afe47f88 -> 8da252c52


[CARBONDATA-2207] Fix testcases after using hive metastore

CarbonTable was getting null in case of hivemetatore so, fetch the same from metastore instead of carbon.

This closes #2005


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

Branch: refs/heads/master
Commit: 8da252c5273a2372c76b00981b763ab0e08df4f4
Parents: 1afe47f
Author: Jatin <ja...@knoldus.in>
Authored: Tue Feb 27 16:13:40 2018 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Wed Feb 28 16:22:39 2018 +0530

----------------------------------------------------------------------
 .../spark/testsuite/iud/DeleteCarbonTableTestCase.scala   |  8 +++++++-
 .../testsuite/partition/TestDDLForPartitionTable.scala    |  8 ++++----
 .../TestDDLForPartitionTableWithDefaultProperties.scala   | 10 +++++-----
 .../partition/CarbonAlterTableSplitPartitionCommand.scala |  2 +-
 4 files changed, 17 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da252c5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index 22aa385..d05f022 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -196,7 +196,13 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
       .getCarbonTable(Some("iud_db"), "update_status_files")(sqlContext.sparkSession)
     val metaPath = carbonTable.getMetaDataFilepath
     val files = FileFactory.getCarbonFile(metaPath)
-    assert(files.listFiles().length == 2)
+    val result = CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetastore.getClass
+    if(result.getCanonicalName.contains("CarbonFileMetastore")) {
+      assert(files.listFiles().length == 2)
+    }
+    else
+      assert(files.listFiles().length == 1)
+
     sql("drop table update_status_files")
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da252c5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
index fd32f51..2cbafa8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
@@ -17,10 +17,10 @@
 
 package org.apache.carbondata.spark.testsuite.partition
 
+import org.apache.spark.sql.{CarbonEnv, SQLContext}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
@@ -50,7 +50,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
         | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
       """.stripMargin)
 
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "hashTable")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
@@ -73,7 +73,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
         |  'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59', 'DICTIONARY_INCLUDE'='doj')
       """.stripMargin)
 
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rangeTable")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
@@ -100,7 +100,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
         | TBLPROPERTIES('PARTITION_TYPE'='LIST',
         |  'LIST_INFO'='0, 1, (2, 3)')
       """.stripMargin)
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "listTable")
+    val carbonTable =  CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("workgroupcategory"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da252c5/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
index efef44c..88f4487 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.spark.testsuite.partition
 
+import org.apache.spark.sql.CarbonEnv
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
@@ -45,7 +45,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
         | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
       """.stripMargin)
 
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "hashTable")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
@@ -68,7 +68,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
         |  'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59','DICTIONARY_INCLUDE'='doj')
       """.stripMargin)
 
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rangeTable")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
@@ -96,7 +96,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
         |  'LIST_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59',
         |  'DICTIONARY_INCLUDE'='projectenddate')
       """.stripMargin)
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "listTable")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
@@ -128,7 +128,7 @@ class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with Befo
         | TBLPROPERTIES('PARTITION_TYPE'='LIST',
         |  'LIST_INFO'='2017-06-11 , 2017-06-13')
       """.stripMargin)
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "listTableDate")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTableDate")(sqlContext.sparkSession)
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
     assert(partitionInfo != null)
     assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/8da252c5/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
index 7aefbbe..78cf2b8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
@@ -68,7 +68,7 @@ case class CarbonAlterTableSplitPartitionCommand(
       throwMetadataException(dbName, tableName, "table not found")
     }
     carbonMetaStore.checkSchemasModifiedTimeAndReloadTable(TableIdentifier(tableName, Some(dbName)))
-    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName, tableName)) {
+    if (null == (CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession))) {
       LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
       throwMetadataException(dbName, tableName, "table not found")
     }