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

[SPARK-30090][SHELL] Adapt Spark REPL to Scala 2.13 #28545

Closed
wants to merge 3 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
Original file line number Diff line number Diff line change
Expand Up @@ -495,7 +495,7 @@ final class ShuffleBlockFetcherIterator(
hostLocalDirManager.getHostLocalDirs(host, port, bmIds.map(_.executorId)) {
case Success(dirsByExecId) =>
fetchMultipleHostLocalBlocks(
hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains),
hostLocalBlocksWithMissingDirs.filterKeys(bmIds.contains).toMap,
dirsByExecId,
cached = false)

Expand Down
138 changes: 138 additions & 0 deletions repl/src/main/scala-2.13/org/apache/spark/repl/Main.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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

import java.io.File
import java.net.URI
import java.util.Locale

import scala.tools.nsc.GenericRunnerSettings

import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
import org.apache.spark.util.Utils

object Main extends Logging {

initializeLogIfNecessary(true)
Signaling.cancelOnInterrupt()

val conf = new SparkConf()
val rootDir =
conf.getOption("spark.repl.classdir").getOrElse(Utils.getLocalDir(conf))
val outputDir = Utils.createTempDir(root = rootDir, namePrefix = "repl")

var sparkContext: SparkContext = _
var sparkSession: SparkSession = _
// this is a public var because tests reset it.
var interp: SparkILoop = _

private var hasErrors = false
private var isShellSession = false

private def scalaOptionError(msg: String): Unit = {
hasErrors = true
// scalastyle:off println
Console.err.println(msg)
// scalastyle:on println
}

def main(args: Array[String]): Unit = {
isShellSession = true
doMain(args, new SparkILoop)
}

// Visible for testing
private[repl] def doMain(args: Array[String], _interp: SparkILoop): Unit = {
interp = _interp
val jars = Utils
.getLocalUserJarsForShell(conf)
// Remove file:///, file:// or file:/ scheme if exists for each jar
.map { x =>
if (x.startsWith("file:")) new File(new URI(x)).getPath else x
}
.mkString(File.pathSeparator)
val interpArguments = List(
"-Yrepl-class-based",
"-Yrepl-outdir",
s"${outputDir.getAbsolutePath}",
"-classpath",
jars
) ++ args.toList

val settings = new GenericRunnerSettings(scalaOptionError)
settings.processArguments(interpArguments, true)

if (!hasErrors) {
interp.run(settings) // Repl starts and goes in loop of R.E.P.L
Option(sparkContext).foreach(_.stop)
}
}

def createSparkSession(): SparkSession = {
try {
val execUri = System.getenv("SPARK_EXECUTOR_URI")
conf.setIfMissing("spark.app.name", "Spark shell")
// SparkContext will detect this configuration and register it with the RpcEnv's
// file server, setting spark.repl.class.uri to the actual URI for executors to
// use. This is sort of ugly but since executors are started as part of SparkContext
// initialization in certain cases, there's an initialization order issue that prevents
// this from being set after SparkContext is instantiated.
conf.set("spark.repl.class.outputDir", outputDir.getAbsolutePath())
if (execUri != null) {
conf.set("spark.executor.uri", execUri)
}
if (System.getenv("SPARK_HOME") != null) {
conf.setSparkHome(System.getenv("SPARK_HOME"))
}

val builder = SparkSession.builder.config(conf)
if (conf
.get(CATALOG_IMPLEMENTATION.key, "hive")
.toLowerCase(Locale.ROOT) == "hive") {
if (SparkSession.hiveClassesArePresent) {
// In the case that the property is not set at all, builder's config
// does not have this value set to 'hive' yet. The original default
// behavior is that when there are hive classes, we use hive catalog.
sparkSession = builder.enableHiveSupport().getOrCreate()
logInfo("Created Spark session with Hive support")
} else {
// Need to change it back to 'in-memory' if no hive classes are found
// in the case that the property is set to hive in spark-defaults.conf
builder.config(CATALOG_IMPLEMENTATION.key, "in-memory")
sparkSession = builder.getOrCreate()
logInfo("Created Spark session")
}
} else {
// In the case that the property is set but not to 'hive', the internal
// default is 'in-memory'. So the sparkSession will use in-memory catalog.
sparkSession = builder.getOrCreate()
logInfo("Created Spark session")
}
sparkContext = sparkSession.sparkContext
sparkSession
} catch {
case e: Exception if isShellSession =>
logError("Failed to initialize Spark session.", e)
sys.exit(1)
}
}

}
149 changes: 149 additions & 0 deletions repl/src/main/scala-2.13/org/apache/spark/repl/SparkILoop.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
/*
* 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

import java.io.{BufferedReader, PrintWriter}

// scalastyle:off println
import scala.Predef.{println => _, _}
import scala.tools.nsc.GenericRunnerSettings
import scala.tools.nsc.Settings
import scala.tools.nsc.interpreter.shell.{ILoop, ShellConfig}
import scala.tools.nsc.util.stringFromStream
import scala.util.Properties.{javaVersion, javaVmName, versionString}
// scalastyle:on println

/**
* A Spark-specific interactive shell.
*/
class SparkILoop(in0: BufferedReader, out: PrintWriter)
extends ILoop(ShellConfig(new GenericRunnerSettings(_ => ())), in0, out) {
def this() = this(null, new PrintWriter(Console.out, true))

val initializationCommands: Seq[String] = Seq(
"""
@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 ${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
}
""",
"import org.apache.spark.SparkContext._",
"import spark.implicits._",
"import spark.sql",
"import org.apache.spark.sql.functions._"
)

override protected def internalReplAutorunCode(): Seq[String] =
initializationCommands

def initializeSpark(): Unit = {
if (!intp.reporter.hasErrors) {
// `savingReplayStack` removes the commands from session history.
savingReplayStack {
initializationCommands.foreach(intp quietRun _)
}
} else {
throw new RuntimeException(
s"Scala $versionString interpreter encountered " +
"errors during initialization"
)
}
}

/** Print a welcome message */
override def printWelcome(): Unit = {
import org.apache.spark.SPARK_VERSION
echo("""Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
/___/ .__/\_,_/_/ /_/\_\ version %s
/_/
""".format(SPARK_VERSION))
val welcomeMsg = "Using Scala %s (%s, Java %s)".format(
versionString,
javaVmName,
javaVersion
)
echo(welcomeMsg)
echo("Type in expressions to have them evaluated.")
echo("Type :help for more information.")
}

/** Available commands */
override def commands: List[LoopCommand] = standardCommands

override def resetCommand(line: String): Unit = {
super.resetCommand(line)
initializeSpark()
echo(
"Note that after :reset, state of SparkSession and SparkContext is unchanged."
)
}

override def replay(): Unit = {
initializeSpark()
super.replay()
}
}

object SparkILoop {

/**
* Creates an interpreter loop with default settings and feeds
* the given code to it as input.
*/
def run(code: String, sets: Settings = new Settings): String = {
import java.io.{BufferedReader, StringReader, OutputStreamWriter}

stringFromStream { ostream =>
Console.withOut(ostream) {
val input = new BufferedReader(new StringReader(code))
val output = new PrintWriter(new OutputStreamWriter(ostream), true)
val repl = new SparkILoop(input, output)

if (sets.classpath.isDefault) {
sets.classpath.value = sys.props("java.class.path")
}
repl.run(sets)
}
}
}
def run(lines: List[String]): String = run(lines.map(_ + "\n").mkString)
}
58 changes: 58 additions & 0 deletions repl/src/test/scala-2.12/org/apache/spark/repl/Repl2Suite.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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

import java.io._
import java.nio.file.Files

import scala.tools.nsc.interpreter.SimpleReader

import org.apache.log4j.{Level, LogManager, PropertyConfigurator}
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.{SparkContext, SparkFunSuite}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION

class Repl2Suite extends SparkFunSuite with BeforeAndAfterAll {
test("propagation of local properties") {
// A mock ILoop that doesn't install the SIGINT handler.
class ILoop(out: PrintWriter) extends SparkILoop(None, out) {
settings = new scala.tools.nsc.Settings
settings.usejavacp.value = true
org.apache.spark.repl.Main.interp = this
in = SimpleReader()
}

val out = new StringWriter()
Main.interp = new ILoop(new PrintWriter(out))
Main.sparkContext = new SparkContext("local", "repl-test")
Main.interp.createInterpreter()

Main.sparkContext.setLocalProperty("someKey", "someValue")

// Make sure the value we set in the caller to interpret is propagated in the thread that
// interprets the command.
Main.interp.interpret("org.apache.spark.repl.Main.sparkContext.getLocalProperty(\"someKey\")")
assert(out.toString.contains("someValue"))

Main.sparkContext.stop()
System.clearProperty("spark.driver.port")
}
}
Loading