Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[CARBONDATA-1099] Fixed bug for carbon-spark-shell in spark2 environment #964

Closed
wants to merge 7 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions bin/carbon-spark-shell
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,12 @@ export _SPARK_CMD_USAGE="Usage: ./bin/spark-shell [options]"

export FWDIR=$SPARK_HOME
export CARBON_SOURCE="$(cd "`dirname "$0"`"/..; pwd)"

ASSEMBLY_DIR="$CARBON_SOURCE/assembly/target/scala-2.10"
if [ -d "$CARBON_SOURCE/assembly/target/scala-2.11" ]; then
ASSEMBLY_DIR="$CARBON_SOURCE/assembly/target/scala-2.11"
fi

GREP_OPTIONS=
num_jars="$(ls -1 "$ASSEMBLY_DIR" | grep "^carbondata.*hadoop.*\.jar$" | wc -l)"
if [ "$num_jars" -eq "0" -a -z "$ASSEMBLY_DIR" ]; then
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,54 +19,66 @@ package org.apache.spark.repl

class CarbonSparkILoop extends SparkILoop {

override def initializeSpark() {
intp.beQuietDuring {
command("""
if(org.apache.spark.repl.carbon.Main.interp == null) {
org.apache.spark.repl.carbon.Main.main(Array[String]())
}
""")
command("val i1 = org.apache.spark.repl.carbon.Main.interp")
command("import i1._")
command("""
@transient val sc = {
val _sc = i1.createSparkContext()
println("Spark context available as sc.")
_sc
}
""")
command("import org.apache.spark.SparkContext._")
command("import org.apache.spark.sql.CarbonContext")
command("""
@transient val cc = {
val _cc = {
import java.io.File
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)
new CarbonContext(sc, storePath, store.getCanonicalPath)
}
println("Carbon context available as cc.")
_cc
}
""")
private def initOriginSpark(): Unit = {
processLine("""
@transient val spark = if (org.apache.spark.repl.Main.sparkSession != null) {
org.apache.spark.repl.Main.sparkSession
} else {
org.apache.spark.repl.Main.createSparkSession()
}
@transient val sc = {
val _sc = spark.sparkContext
if (_sc.getConf.getBoolean("spark.ui.reverseProxy", false)) {
val proxyUrl = _sc.getConf.get("spark.ui.reverseProxyUrl", null)
if (proxyUrl != null) {
println(s"Spark Context Web UI is available at " +
s"${proxyUrl}/proxy/${_sc.applicationId}")
} else {
println(s"Spark Context Web UI is available at Spark Master Public URL")
}
} else {
_sc.uiWebUrl.foreach {
webUrl => println(s"Spark context Web UI available at ${webUrl}")
}
}
println("Spark context available as 'sc' " +
s"(master = ${_sc.master}, app id = ${_sc.applicationId}).")
println("Spark session available as 'spark'.")
_sc
}
""")
processLine("import org.apache.spark.SparkContext._")
processLine("import spark.implicits._")
processLine("import spark.sql")
processLine("import org.apache.spark.sql.functions._")
}

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")
private def initCarbon(): Unit = {
processLine("""
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.CarbonSession._
@transient val carbon = {
val _carbon = {
import java.io.File
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)
SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession(storePath)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you do not specify the metastore path, it will generate carbon.metastore which is in same level with carbondata project.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

}
println("Carbon session available as carbon.")
_carbon
}
""")

command("import cc.implicits._")
command("import cc.sql")
command("import org.apache.spark.sql.functions._")
processLine("import carbon.implicits._")
processLine("import carbon.sql")
}
override def initializeSpark() {
intp.beQuietDuring {
initOriginSpark()
initCarbon()
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.repl.carbon

import org.apache.spark.repl.{CarbonSparkILoop, SparkILoop}

object Main {

private var _interp: SparkILoop = _

def interp: SparkILoop = _interp

def interp_=(i: SparkILoop) { _interp = i }

def main(args: Array[String]) {
_interp = new CarbonSparkILoop
org.apache.spark.repl.Main.doMain(args, _interp)
}
}