You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2017/11/18 15:29:50 UTC

[16/28] carbondata git commit: [CARBONDATA-1745] Use default metastore path from Hive

[CARBONDATA-1745] Use default metastore path from Hive

This closes #1513


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

Branch: refs/heads/fgdatamap
Commit: dfc7442a483d839282edb6e1305c191aa60da65a
Parents: 75ec79e
Author: Jacky Li <ja...@qq.com>
Authored: Fri Nov 17 15:01:43 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Fri Nov 17 16:25:59 2017 +0800

----------------------------------------------------------------------
 .../core/metadata/AbsoluteTableIdentifier.java  |  1 +
 .../carbondata/examples/AlterTableExample.scala |  3 +-
 .../examples/CarbonDataFrameExample.scala       |  1 -
 .../examples/CarbonPartitionExample.scala       |  1 -
 .../carbondata/examples/ExampleUtils.scala      |  1 -
 .../org/apache/spark/sql/CarbonSession.scala    | 38 ++++++++++----------
 .../spark/sql/hive/cli/CarbonSQLCLIDriver.scala |  3 +-
 7 files changed, 23 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
index d5434d8..603a1c1 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
@@ -62,6 +62,7 @@ public class AbsoluteTableIdentifier implements Serializable {
     CarbonTableIdentifier identifier = new CarbonTableIdentifier(dbName, tableName, "");
     return new AbsoluteTableIdentifier(tablePath, identifier);
   }
+
   public String getTablePath() {
     return tablePath;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
index dd2a28a..472dc44 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/AlterTableExample.scala
@@ -37,7 +37,6 @@ object AlterTableExample {
 
     val storeLocation = s"$rootPath/examples/spark2/target/store"
     val warehouse = s"$rootPath/examples/spark2/target/warehouse"
-    val metastoredb = s"$rootPath/examples/spark2/target/metastore_db"
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
@@ -49,7 +48,7 @@ object AlterTableExample {
       .master("local")
       .appName("AlterTableExample")
       .config("spark.sql.warehouse.dir", warehouse)
-      .getOrCreateCarbonSession(storeLocation, metastoredb)
+      .getOrCreateCarbonSession(storeLocation)
 
     spark.sparkContext.setLogLevel("WARN")
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonDataFrameExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonDataFrameExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonDataFrameExample.scala
index ac198d8..2450b49 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonDataFrameExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonDataFrameExample.scala
@@ -31,7 +31,6 @@ object CarbonDataFrameExample {
                             + "../../../..").getCanonicalPath
     val storeLocation = s"$rootPath/examples/spark2/target/store"
     val warehouse = s"$rootPath/examples/spark2/target/warehouse"
-    val metastoredb = s"$rootPath/examples/spark2/target"
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
index d8aca6b..6837c56 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonPartitionExample.scala
@@ -34,7 +34,6 @@ object CarbonPartitionExample {
                             + "../../../..").getCanonicalPath
     val storeLocation = s"$rootPath/examples/spark2/target/store"
     val warehouse = s"$rootPath/examples/spark2/target/warehouse"
-    val metastoredb = s"$rootPath/examples/spark2/target"
     val testData = s"$rootPath/integration/spark-common-test/src/test/resources/partition_data.csv"
 
     CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
index a48ed6a..7975859 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/ExampleUtils.scala
@@ -37,7 +37,6 @@ object ExampleUtils {
                             + "../../../..").getCanonicalPath
     val storeLocation = s"$rootPath/examples/spark2/target/store"
     val warehouse = s"$rootPath/examples/spark2/target/warehouse"
-    val metastoredb = s"$rootPath/examples/spark2/target"
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd HH:mm:ss")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index 88cbc36..21840e4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -66,7 +66,7 @@ class CarbonSession(@transient val sc: SparkContext,
   }
 
   if (existingSharedState.isEmpty) {
-    CarbonSession.initListeners
+    CarbonSession.initListeners()
   }
 
 }
@@ -76,9 +76,7 @@ object CarbonSession {
   implicit class CarbonBuilder(builder: Builder) {
 
     def getOrCreateCarbonSession(): SparkSession = {
-      getOrCreateCarbonSession(
-        null,
-        new File(CarbonCommonConstants.METASTORE_LOCATION_DEFAULT_VAL).getCanonicalPath)
+      getOrCreateCarbonSession(null, null)
     }
 
     def getOrCreateCarbonSession(storePath: String): SparkSession = {
@@ -94,17 +92,20 @@ object CarbonSession {
         getValue("options", builder).asInstanceOf[scala.collection.mutable.HashMap[String, String]]
       val userSuppliedContext: Option[SparkContext] =
         getValue("userSuppliedContext", builder).asInstanceOf[Option[SparkContext]]
-      val hadoopConf = new Configuration()
-      val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml")
-      if (configFile != null) {
-        hadoopConf.addResource(configFile)
-      }
-      if (options.get(CarbonCommonConstants.HIVE_CONNECTION_URL).isEmpty &&
-          hadoopConf.get(CarbonCommonConstants.HIVE_CONNECTION_URL) == null) {
-        val metaStorePathAbsolute = new File(metaStorePath).getCanonicalPath
-        val hiveMetaStoreDB = metaStorePathAbsolute + "/metastore_db"
-        options ++= Map[String, String]((CarbonCommonConstants.HIVE_CONNECTION_URL,
-          s"jdbc:derby:;databaseName=$hiveMetaStoreDB;create=true"))
+
+      if (metaStorePath != null) {
+        val hadoopConf = new Configuration()
+        val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml")
+        if (configFile != null) {
+          hadoopConf.addResource(configFile)
+        }
+        if (options.get(CarbonCommonConstants.HIVE_CONNECTION_URL).isEmpty &&
+            hadoopConf.get(CarbonCommonConstants.HIVE_CONNECTION_URL) == null) {
+          val metaStorePathAbsolute = new File(metaStorePath).getCanonicalPath
+          val hiveMetaStoreDB = metaStorePathAbsolute + "/metastore_db"
+          options ++= Map[String, String]((CarbonCommonConstants.HIVE_CONNECTION_URL,
+            s"jdbc:derby:;databaseName=$hiveMetaStoreDB;create=true"))
+        }
       }
 
       // Get the session from current thread's active session.
@@ -157,15 +158,16 @@ object CarbonSession {
           }
           sc
         }
+
+        session = new CarbonSession(sparkContext)
         val carbonProperties = CarbonProperties.getInstance()
         if (storePath != null) {
           carbonProperties.addProperty(CarbonCommonConstants.STORE_LOCATION, storePath)
           // In case if it is in carbon.properties for backward compatible
         } else if (carbonProperties.getProperty(CarbonCommonConstants.STORE_LOCATION) == null) {
           carbonProperties.addProperty(CarbonCommonConstants.STORE_LOCATION,
-            sparkContext.conf.get("spark.sql.warehouse.dir"))
+            session.sessionState.conf.warehousePath)
         }
-        session = new CarbonSession(sparkContext)
         options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) }
         SparkSession.setDefaultSession(session)
         CommonUtil.cleanInProgressSegments(
@@ -182,7 +184,7 @@ object CarbonSession {
         session.streams.addListener(new CarbonStreamingQueryListener(session))
       }
 
-      return session
+      session
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/dfc7442a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
index 22c7218..733744f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
@@ -39,7 +39,6 @@ object CarbonSQLCLIDriver {
 
   def init() {
     if (hiveContext == null) {
-      val sparkConf = new SparkConf(loadDefaults = true)
 
       import org.apache.spark.sql.CarbonSession._
 
@@ -50,7 +49,7 @@ object CarbonSQLCLIDriver {
           .master(System.getProperty("spark.master"))
           .appName("CarbonSQLCLIDriver")
           .config("spark.sql.warehouse.dir", warehouse)
-          .getOrCreateCarbonSession(storePath, storePath)
+          .getOrCreateCarbonSession(storePath)
 
       hiveContext = carbon.sqlContext
       hiveContext.conf.getAllConfs.toSeq.sorted.foreach { case (k, v) =>