carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From chenliang...@apache.org
Subject [1/2] incubator-carbondata git commit: Fixed relative path issue from carbon-spark-shell
Date Fri, 22 Jul 2016 09:18:13 GMT
Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 06e8e95cc -> e84ff4270


Fixed relative path issue from carbon-spark-shell


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

Branch: refs/heads/master
Commit: ec97aa695c6d76b37cf406f8b0a14f7f01bf4e19
Parents: 06e8e95
Author: ravipesala <ravi.pesala@gmail.com>
Authored: Thu Jul 21 10:13:37 2016 +0530
Committer: chenliang613 <chenliang613@apache.org>
Committed: Fri Jul 22 17:15:15 2016 +0800

----------------------------------------------------------------------
 bin/carbon-spark-shell                             |  1 +
 bin/carbon-spark-sql                               |  1 +
 .../org/apache/spark/repl/CarbonSparkILoop.scala   | 16 ++++++++++++++--
 .../sql/execution/command/carbonTableSchema.scala  | 10 ++--------
 .../spark/sql/hive/cli/CarbonSQLCLIDriver.scala    |  4 ++--
 .../carbondata/spark/util/CarbonScalaUtil.scala    | 17 ++++++++++++++++-
 6 files changed, 36 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec97aa69/bin/carbon-spark-shell
----------------------------------------------------------------------
diff --git a/bin/carbon-spark-shell b/bin/carbon-spark-shell
index 1b7148f..78e350c 100755
--- a/bin/carbon-spark-shell
+++ b/bin/carbon-spark-shell
@@ -52,6 +52,7 @@ fi
 ASSEMBLY_JAR="${ASSEMBLY_DIR}/${ASSEMBLY_JARS}"
 export JAR="$ASSEMBLY_JAR"
 export SPARK_CLASSPATH=$SPARK_CLASSPATH:$JAR
+export CARBON_HOME=$CARBON_SOURCE
 
 
 # SPARK-4161: scala does not assume use of the java classpath,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec97aa69/bin/carbon-spark-sql
----------------------------------------------------------------------
diff --git a/bin/carbon-spark-sql b/bin/carbon-spark-sql
index dbf8883..43d6f1a 100755
--- a/bin/carbon-spark-sql
+++ b/bin/carbon-spark-sql
@@ -51,6 +51,7 @@ fi
 
 ASSEMBLY_JAR="${ASSEMBLY_DIR}/${ASSEMBLY_JARS}"
 export JAR="$ASSEMBLY_JAR"
+export CARBON_HOME=$CARBON_SOURCE
 
 function usage {
   if [ -n "$1" ]; then

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec97aa69/integration/spark/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
b/integration/spark/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
index aaed3d8..cee026d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/repl/CarbonSparkILoop.scala
@@ -41,7 +41,8 @@ class CarbonSparkILoop extends SparkILoop {
          @transient val cc = {
            val _cc = {
              import java.io.File
-             val store = new File("./carbonshellstore")
+             val path = System.getenv("CARBON_HOME") + "/bin/carbonshellstore"
+             val store = new File(path)
              store.mkdirs()
              val storePath = sc.getConf.getOption("spark.carbon.storepath")
                   .getOrElse(store.getCanonicalPath)
@@ -51,7 +52,18 @@ class CarbonSparkILoop extends SparkILoop {
            _cc
          }
               """)
-      command("""cc.setConf("carbon.kettle.home", "../processing/carbonplugins")""")
+
+      command("import org.apache.spark.sql.SQLContext")
+      command("""
+         @transient val sqlContext = {
+           val _sqlContext = new SQLContext(sc)
+           println("SQL context available as sqlContext.")
+           _sqlContext
+         }
+              """)
+      command("import sqlContext.implicits._")
+      command("import sqlContext.sql")
+
       command("import cc.implicits._")
       command("import cc.sql")
       command("import org.apache.spark.sql.functions._")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec97aa69/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 993e26a..d6cbaf4 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -797,10 +797,7 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel)
e
 
     val partitioner = relation.tableMeta.partitioner
 
-    var kettleHomePath = sqlContext.getConf("carbon.kettle.home", null)
-    if (null == kettleHomePath) {
-      kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home")
-    }
+    var kettleHomePath = CarbonScalaUtil.getKettleHomePath(sqlContext)
     if (kettleHomePath == null) {
       sys.error(s"carbon.kettle.home is not set")
     }
@@ -1083,10 +1080,7 @@ private[sql] case class LoadTable(
       storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
 
       val columinar = sqlContext.getConf("carbon.is.columnar.storage", "true").toBoolean
-      var kettleHomePath = sqlContext.getConf("carbon.kettle.home", null)
-      if (null == kettleHomePath) {
-        kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home")
-      }
+      var kettleHomePath = CarbonScalaUtil.getKettleHomePath(sqlContext)
       if (kettleHomePath == null) {
         sys.error(s"carbon.kettle.home is not set")
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec97aa69/integration/spark/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
b/integration/spark/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
index 0dafe2d..8fbf0cf 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/cli/CarbonSQLCLIDriver.scala
@@ -63,12 +63,12 @@ object CarbonSQLCLIDriver extends Logging {
 
       sparkContext = new SparkContext(sparkConf)
       sparkContext.addSparkListener(new StatsReportListener())
-      val store = new File("./carbonsqlclistore")
+      val path = System.getenv("CARBON_HOME") + "/bin/carbonsqlclistore"
+      val store = new File(path)
       store.mkdirs()
       hiveContext = new CarbonContext(sparkContext,
         maybeStorePath.getOrElse(store.getCanonicalPath),
         store.getCanonicalPath)
-      hiveContext.setConf("carbon.kettle.home", "../processing/carbonplugins")
 
       hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion)
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ec97aa69/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
index dff2fce..1693008 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
@@ -27,7 +27,7 @@ import org.carbondata.core.carbon.metadata.datatype.{DataType => CarbonDataType}
 import org.carbondata.core.carbon.metadata.encoder.Encoding
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.core.constants.CarbonCommonConstants
-import org.carbondata.core.util.CarbonUtil
+import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
 
 object CarbonScalaUtil {
   def convertSparkToCarbonDataType(
@@ -79,6 +79,21 @@ object CarbonScalaUtil {
     }
   }
 
+  def getKettleHomePath(sqlContext: SQLContext): String = {
+    val carbonHome = System.getenv("CARBON_HOME")
+    var kettleHomePath: String = null
+    if (carbonHome != null) {
+      kettleHomePath = System.getenv("CARBON_HOME") + "/processing/carbonplugins"
+    }
+    if (kettleHomePath == null) {
+      kettleHomePath = sqlContext.getConf("carbon.kettle.home", null)
+    }
+    if (null == kettleHomePath) {
+      kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home")
+    }
+    kettleHomePath
+  }
+
   object CarbonSparkUtil {
 
     def createSparkMeta(carbonTable: CarbonTable): CarbonMetaData = {


Mime
View raw message