From 425c4ada4c24e338b45d0e9987071f05c5766fa5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 13 Jul 2017 17:06:24 +0800 Subject: [PATCH] [SPARK-19810][BUILD][CORE] Remove support for Scala 2.10 ## What changes were proposed in this pull request? - Remove Scala 2.10 build profiles and support - Replace some 2.10 support in scripts with commented placeholders for 2.12 later - Remove deprecated API calls from 2.10 support - Remove usages of deprecated context bounds where possible - Remove Scala 2.10 workarounds like ScalaReflectionLock - Other minor Scala warning fixes ## How was this patch tested? Existing tests Author: Sean Owen Closes #17150 from srowen/SPARK-19810. --- R/pkg/R/sparkR.R | 4 +- R/pkg/tests/fulltests/test_client.R | 4 +- bin/load-spark-env.cmd | 22 +- bin/load-spark-env.sh | 22 +- build/mvn | 6 +- .../scala/org/apache/spark/Accumulable.scala | 3 +- .../scala/org/apache/spark/SparkContext.scala | 15 +- .../spark/rdd/SequenceFileRDDFunctions.scala | 54 +- .../org/apache/spark/rpc/RpcTimeout.scala | 4 +- .../org/apache/spark/ui/JettyUtils.scala | 6 +- .../spark/util/logging/FileAppender.scala | 8 +- .../scala/org/apache/spark/FileSuite.scala | 4 +- .../spark/deploy/SparkSubmitUtilsSuite.scala | 6 +- .../spark/deploy/master/MasterSuite.scala | 3 +- .../apache/spark/executor/ExecutorSuite.scala | 1 + .../spark/rdd/LocalCheckpointSuite.scala | 1 + .../OutputCommitCoordinatorSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../scheduler/TaskSetBlacklistSuite.scala | 4 +- .../serializer/KryoSerializerSuite.scala | 4 +- .../spark/util/TimeStampedHashMapSuite.scala | 2 +- .../apache/spark/util/VersionUtilsSuite.scala | 8 +- .../util/collection/AppendOnlyMapSuite.scala | 4 +- .../util/collection/ExternalSorterSuite.scala | 14 +- .../util/collection/OpenHashMapSuite.scala | 5 +- dev/change-scala-version.sh | 4 +- dev/change-version-to-2.10.sh | 23 - dev/change-version-to-2.11.sh | 23 - dev/create-release/release-build.sh | 23 +- docs/building-spark.md | 8 - docs/index.md | 3 +- .../graphx/AggregateMessagesExample.scala | 2 +- .../spark/examples/mllib/AbstractParams.scala | 2 +- .../spark/examples/sql/SparkSQLExample.scala | 2 - .../spark/streaming/kafka/KafkaRDD.scala | 4 +- .../org/apache/spark/graphx/EdgeContext.scala | 2 +- .../launcher/AbstractCommandBuilder.java | 20 +- .../org/apache/spark/ml/linalg/Matrices.scala | 2 +- .../spark/ml/linalg/MatricesSuite.scala | 20 +- .../org/apache/spark/ml/tree/treeModels.scala | 2 +- .../apache/spark/mllib/linalg/Matrices.scala | 2 +- .../linalg/distributed/BlockMatrix.scala | 4 +- .../spark/ml/recommendation/ALSSuite.scala | 2 +- .../spark/mllib/clustering/LDASuite.scala | 2 +- .../spark/mllib/linalg/MatricesSuite.scala | 20 +- pom.xml | 84 +- project/SparkBuild.scala | 23 +- python/run-tests.py | 3 +- repl/pom.xml | 26 +- .../scala/org/apache/spark/repl/Main.scala | 37 - .../apache/spark/repl/SparkCommandLine.scala | 46 - .../apache/spark/repl/SparkExprTyper.scala | 114 -- .../org/apache/spark/repl/SparkHelper.scala | 39 - .../org/apache/spark/repl/SparkILoop.scala | 1145 ----------- .../apache/spark/repl/SparkILoopInit.scala | 168 -- .../org/apache/spark/repl/SparkIMain.scala | 1808 ----------------- .../org/apache/spark/repl/SparkImports.scala | 239 --- .../spark/repl/SparkJLineCompletion.scala | 403 ---- .../apache/spark/repl/SparkJLineReader.scala | 90 - .../spark/repl/SparkMemberHandlers.scala | 232 --- .../spark/repl/SparkRunnerSettings.scala | 31 - .../org/apache/spark/repl/ReplSuite.scala | 366 ---- .../execution/UnsafeExternalRowSorter.java | 4 +- .../spark/sql/catalyst/ScalaReflection.scala | 36 +- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../expressions/codegen/package.scala | 4 +- .../expressions/namedExpressions.scala | 2 +- .../apache/spark/sql/catalyst/package.scala | 6 - .../util/AbstractScalaRowIterator.scala | 30 - .../apache/spark/sql/types/BinaryType.scala | 3 +- .../apache/spark/sql/types/BooleanType.scala | 3 +- .../org/apache/spark/sql/types/ByteType.scala | 3 +- .../org/apache/spark/sql/types/DateType.scala | 4 +- .../apache/spark/sql/types/DecimalType.scala | 3 +- .../apache/spark/sql/types/DoubleType.scala | 3 +- .../apache/spark/sql/types/FloatType.scala | 3 +- .../spark/sql/types/HiveStringType.scala | 5 +- .../apache/spark/sql/types/IntegerType.scala | 4 +- .../org/apache/spark/sql/types/LongType.scala | 3 +- .../apache/spark/sql/types/ShortType.scala | 3 +- .../apache/spark/sql/types/StringType.scala | 3 +- .../spark/sql/types/TimestampType.scala | 4 +- .../optimizer/JoinOptimizationSuite.scala | 24 +- .../optimizer/OptimizeCodegenSuite.scala | 4 +- .../scala/org/apache/spark/sql/Column.scala | 2 +- .../execution/OptimizeMetadataOnlyQuery.scala | 8 +- .../spark/sql/execution/QueryExecution.scala | 4 +- .../execution/python/ExtractPythonUDFs.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 4 +- .../spark/sql/StatisticsCollectionSuite.scala | 20 +- .../org/apache/spark/sql/SubquerySuite.scala | 30 +- .../datasources/parquet/ParquetIOSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 60 +- .../sql/execution/ui/SQLListenerSuite.scala | 2 +- .../expressions/ReduceAggregatorSuite.scala | 14 +- .../StreamingQueryListenerSuite.scala | 2 - .../org/apache/spark/sql/hive/HiveUtils.scala | 4 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 8 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 2 + .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 101 files changed, 319 insertions(+), 5239 deletions(-) delete mode 100755 dev/change-version-to-2.10.sh delete mode 100755 dev/change-version-to-2.11.sh delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala delete mode 100644 repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala delete mode 100644 repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/AbstractScalaRowIterator.scala diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index f2d2620e54..81507ea718 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -113,7 +113,7 @@ sparkR.stop <- function() { #' list(spark.executor.memory="4g"), #' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), #' c("one.jar", "two.jar", "three.jar"), -#' c("com.databricks:spark-avro_2.10:2.0.1")) +#' c("com.databricks:spark-avro_2.11:2.0.1")) #'} #' @note sparkR.init since 1.4.0 sparkR.init <- function( @@ -357,7 +357,7 @@ sparkRHive.init <- function(jsc = NULL) { #' sparkR.session("yarn-client", "SparkR", "/home/spark", #' list(spark.executor.memory="4g"), #' c("one.jar", "two.jar", "three.jar"), -#' c("com.databricks:spark-avro_2.10:2.0.1")) +#' c("com.databricks:spark-avro_2.11:2.0.1")) #' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") #'} #' @note sparkR.session since 2.0.0 diff --git a/R/pkg/tests/fulltests/test_client.R b/R/pkg/tests/fulltests/test_client.R index 0cf25fe1db..de624b572c 100644 --- a/R/pkg/tests/fulltests/test_client.R +++ b/R/pkg/tests/fulltests/test_client.R @@ -37,7 +37,7 @@ test_that("multiple packages don't produce a warning", { test_that("sparkJars sparkPackages as character vectors", { args <- generateSparkSubmitArgs("", "", c("one.jar", "two.jar", "three.jar"), "", - c("com.databricks:spark-avro_2.10:2.0.1")) + c("com.databricks:spark-avro_2.11:2.0.1")) expect_match(args, "--jars one.jar,two.jar,three.jar") - expect_match(args, "--packages com.databricks:spark-avro_2.10:2.0.1") + expect_match(args, "--packages com.databricks:spark-avro_2.11:2.0.1") }) diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index 0977025c20..993aa31a4c 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -35,21 +35,21 @@ if [%SPARK_ENV_LOADED%] == [] ( rem Setting SPARK_SCALA_VERSION if not already set. -set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11" -set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.10" +rem set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11" +rem set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.12" if [%SPARK_SCALA_VERSION%] == [] ( - if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( - echo "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." - echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd." - exit 1 - ) - if exist %ASSEMBLY_DIR2% ( + rem if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( + rem echo "Presence of build for multiple Scala versions detected." + rem echo "Either clean one of them or, set SPARK_SCALA_VERSION=2.11 in spark-env.cmd." + rem exit 1 + rem ) + rem if exist %ASSEMBLY_DIR2% ( set SPARK_SCALA_VERSION=2.11 - ) else ( - set SPARK_SCALA_VERSION=2.10 - ) + rem ) else ( + rem set SPARK_SCALA_VERSION=2.12 + rem ) ) exit /b 0 diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 8a2f709960..9de62039c5 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -46,18 +46,18 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then - ASSEMBLY_DIR2="${SPARK_HOME}/assembly/target/scala-2.11" - ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.10" + #ASSEMBLY_DIR2="${SPARK_HOME}/assembly/target/scala-2.11" + #ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.12" - if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then - echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 - echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2 - exit 1 - fi + #if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then + # echo -e "Presence of build for multiple Scala versions detected." 1>&2 + # echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2 + # exit 1 + #fi - if [ -d "$ASSEMBLY_DIR2" ]; then + #if [ -d "$ASSEMBLY_DIR2" ]; then export SPARK_SCALA_VERSION="2.11" - else - export SPARK_SCALA_VERSION="2.10" - fi + #else + # export SPARK_SCALA_VERSION="2.12" + #fi fi diff --git a/build/mvn b/build/mvn index 1e393c331d..efa4f9364e 100755 --- a/build/mvn +++ b/build/mvn @@ -91,13 +91,13 @@ install_mvn() { # Install zinc under the build/ folder install_zinc() { - local zinc_path="zinc-0.3.11/bin/zinc" + local zinc_path="zinc-0.3.15/bin/zinc" [ ! -f "${_DIR}/${zinc_path}" ] && ZINC_INSTALL_FLAG=1 local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.typesafe.com} install_app \ - "${TYPESAFE_MIRROR}/zinc/0.3.11" \ - "zinc-0.3.11.tgz" \ + "${TYPESAFE_MIRROR}/zinc/0.3.15" \ + "zinc-0.3.15.tgz" \ "${zinc_path}" ZINC_BIN="${_DIR}/${zinc_path}" } diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala index 5532931e2a..3092074232 100644 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -201,7 +201,8 @@ trait AccumulableParam[R, T] extends Serializable { @deprecated("use AccumulatorV2", "2.0.0") private[spark] class -GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] +GrowableAccumulableParam[R : ClassTag, T] + (implicit rg: R => Growable[T] with TraversableOnce[T] with Serializable) extends AccumulableParam[R, T] { def addAccumulator(growable: R, elem: T): R = { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b2a26c51d4..e1ce66a547 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -183,8 +183,6 @@ class SparkContext(config: SparkConf) extends Logging { // log out Spark Version in Spark driver log logInfo(s"Running Spark version $SPARK_VERSION") - warnDeprecatedVersions() - /* ------------------------------------------------------------------------------------- * | Private variables. These variables keep the internal state of the context, and are | | not accessible by the outside world. They're mutable since we want to initialize all | @@ -349,13 +347,6 @@ class SparkContext(config: SparkConf) extends Logging { value } - private def warnDeprecatedVersions(): Unit = { - val javaVersion = System.getProperty("java.version").split("[+.\\-]+", 3) - if (scala.util.Properties.releaseVersion.exists(_.startsWith("2.10"))) { - logWarning("Support for Scala 2.10 is deprecated as of Spark 2.1.0") - } - } - /** Control our logLevel. This overrides any user-defined log settings. * @param logLevel The desired log level as a string. * Valid log levels include: ALL, DEBUG, ERROR, FATAL, INFO, OFF, TRACE, WARN @@ -1396,6 +1387,8 @@ class SparkContext(config: SparkConf) extends Logging { @deprecated("use AccumulatorV2", "2.0.0") def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { + // TODO the context bound (<%) above should be replaced with simple type bound and implicit + // conversion but is a breaking change. This should be fixed in Spark 3.x. val param = new GrowableAccumulableParam[R, T] val acc = new Accumulable(initialValue, param) cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) @@ -2605,9 +2598,9 @@ object SparkContext extends Logging { */ private[spark] val LEGACY_DRIVER_IDENTIFIER = "" - private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]) + private implicit def arrayToArrayWritable[T <: Writable : ClassTag](arr: Traversable[T]) : ArrayWritable = { - def anyToWritable[U <% Writable](u: U): Writable = u + def anyToWritable[U <: Writable](u: U): Writable = u new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]], arr.map(x => anyToWritable(x)).toArray) diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 86a332790f..02def89dd8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.rdd -import scala.reflect.{classTag, ClassTag} +import scala.reflect.ClassTag import org.apache.hadoop.io.Writable import org.apache.hadoop.io.compress.CompressionCodec @@ -39,40 +39,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag extends Logging with Serializable { - private val keyWritableClass = - if (_keyWritableClass == null) { - // pre 1.3.0, we need to use Reflection to get the Writable class - getWritableClass[K]() - } else { - _keyWritableClass - } - - private val valueWritableClass = - if (_valueWritableClass == null) { - // pre 1.3.0, we need to use Reflection to get the Writable class - getWritableClass[V]() - } else { - _valueWritableClass - } - - private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = { - val c = { - if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) { - classTag[T].runtimeClass - } else { - // We get the type of the Writable class by looking at the apply method which converts - // from T to Writable. Since we have two apply methods we filter out the one which - // is not of the form "java.lang.Object apply(java.lang.Object)" - implicitly[T => Writable].getClass.getDeclaredMethods().filter( - m => m.getReturnType().toString != "class java.lang.Object" && - m.getName() == "apply")(0).getReturnType - - } - // TODO: use something like WritableConverter to avoid reflection - } - c.asInstanceOf[Class[_ <: Writable]] - } - + // TODO the context bound (<%) above should be replaced with simple type bound and implicit + // conversion but is a breaking change. This should be fixed in Spark 3.x. /** * Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key @@ -90,24 +58,24 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag // valueWritableClass at the compile time. To implement that, we need to add type parameters to // SequenceFileRDDFunctions. however, SequenceFileRDDFunctions is a public class so it will be a // breaking change. - val convertKey = self.keyClass != keyWritableClass - val convertValue = self.valueClass != valueWritableClass + val convertKey = self.keyClass != _keyWritableClass + val convertValue = self.valueClass != _valueWritableClass - logInfo("Saving as sequence file of type (" + keyWritableClass.getSimpleName + "," + - valueWritableClass.getSimpleName + ")" ) + logInfo("Saving as sequence file of type " + + s"(${_keyWritableClass.getSimpleName},${_valueWritableClass.getSimpleName})" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyWritableClass, valueWritableClass, format, jobConf, codec) + self.saveAsHadoopFile(path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } else if (!convertKey && convertValue) { self.map(x => (x._1, anyToWritable(x._2))).saveAsHadoopFile( - path, keyWritableClass, valueWritableClass, format, jobConf, codec) + path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } else if (convertKey && !convertValue) { self.map(x => (anyToWritable(x._1), x._2)).saveAsHadoopFile( - path, keyWritableClass, valueWritableClass, format, jobConf, codec) + path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } else if (convertKey && convertValue) { self.map(x => (anyToWritable(x._1), anyToWritable(x._2))).saveAsHadoopFile( - path, keyWritableClass, valueWritableClass, format, jobConf, codec) + path, _keyWritableClass, _valueWritableClass, format, jobConf, codec) } } } diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala index 0557b7a3cc..3dc41f7f12 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala @@ -125,9 +125,9 @@ private[spark] object RpcTimeout { var foundProp: Option[(String, String)] = None while (itr.hasNext && foundProp.isEmpty) { val propKey = itr.next() - conf.getOption(propKey).foreach { prop => foundProp = Some(propKey, prop) } + conf.getOption(propKey).foreach { prop => foundProp = Some((propKey, prop)) } } - val finalProp = foundProp.getOrElse(timeoutPropList.head, defaultValue) + val finalProp = foundProp.getOrElse((timeoutPropList.head, defaultValue)) val timeout = { Utils.timeStringAsSeconds(finalProp._2).seconds } new RpcTimeout(timeout, finalProp._1) } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index b9371c7ad7..0fa96713ea 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -54,7 +54,7 @@ private[spark] object JettyUtils extends Logging { // implicit conversion from many types of functions to jetty Handlers. type Responder[T] = HttpServletRequest => T - class ServletParams[T <% AnyRef](val responder: Responder[T], + class ServletParams[T <: AnyRef](val responder: Responder[T], val contentType: String, val extractFn: T => String = (in: Any) => in.toString) {} @@ -68,7 +68,7 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = new ServletParams(responder, "text/plain") - def createServlet[T <% AnyRef]( + def createServlet[T <: AnyRef]( servletParams: ServletParams[T], securityMgr: SecurityManager, conf: SparkConf): HttpServlet = { @@ -113,7 +113,7 @@ private[spark] object JettyUtils extends Logging { } /** Create a context handler that responds to a request with the given path prefix */ - def createServletHandler[T <% AnyRef]( + def createServletHandler[T <: AnyRef]( path: String, servletParams: ServletParams[T], securityMgr: SecurityManager, diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 8a0cc709bc..2f9ad4c8cc 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -125,16 +125,16 @@ private[spark] object FileAppender extends Logging { val validatedParams: Option[(Long, String)] = rollingInterval match { case "daily" => logInfo(s"Rolling executor logs enabled for $file with daily rolling") - Some(24 * 60 * 60 * 1000L, "--yyyy-MM-dd") + Some((24 * 60 * 60 * 1000L, "--yyyy-MM-dd")) case "hourly" => logInfo(s"Rolling executor logs enabled for $file with hourly rolling") - Some(60 * 60 * 1000L, "--yyyy-MM-dd--HH") + Some((60 * 60 * 1000L, "--yyyy-MM-dd--HH")) case "minutely" => logInfo(s"Rolling executor logs enabled for $file with rolling every minute") - Some(60 * 1000L, "--yyyy-MM-dd--HH-mm") + Some((60 * 1000L, "--yyyy-MM-dd--HH-mm")) case IntParam(seconds) => logInfo(s"Rolling executor logs enabled for $file with rolling $seconds seconds") - Some(seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss") + Some((seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss")) case _ => logWarning(s"Illegal interval for rolling executor logs [$rollingInterval], " + s"rolling logs not enabled") diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 5be0121db5..02728180ac 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -113,11 +113,11 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { val normalFile = new File(normalDir, "part-00000") val normalContent = sc.sequenceFile[String, String](normalDir).collect - assert(normalContent === Array.fill(100)("abc", "abc")) + assert(normalContent === Array.fill(100)(("abc", "abc"))) val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension) val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect - assert(compressedContent === Array.fill(100)("abc", "abc")) + assert(compressedContent === Array.fill(100)(("abc", "abc"))) assert(compressedFile.length < normalFile.length) } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 57024786b9..88b77e5143 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -93,8 +93,8 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { test("add dependencies works correctly") { val md = SparkSubmitUtils.getModuleDescriptor - val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.10:0.1," + - "com.databricks:spark-avro_2.10:0.1") + val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.11:0.1," + + "com.databricks:spark-avro_2.11:0.1") SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default") assert(md.getDependencies.length === 2) @@ -196,7 +196,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { SparkSubmitUtils.buildIvySettings(None, None), isTest = true) assert(path === "", "should return empty path") - val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.10", "1.2.0") + val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.11", "1.2.0") IvyTestUtils.withRepository(main, None, None) { repo => val files = SparkSubmitUtils.resolveMavenCoordinates( coordinates + "," + main.toString, diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index a223212678..84b3a29b58 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -80,6 +80,7 @@ class MockWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) extend case Some(appId) => apps.remove(appId) master.send(UnregisterApplication(appId)) + case None => } driverIdToAppId.remove(driverId) } @@ -575,7 +576,7 @@ class MasterSuite extends SparkFunSuite override val rpcEnv: RpcEnv = master.rpcEnv override def receive: PartialFunction[Any, Unit] = { - case KillExecutor(_, appId, execId) => killedExecutors.add(appId, execId) + case KillExecutor(_, appId, execId) => killedExecutors.add((appId, execId)) case KillDriver(driverId) => killedDrivers.add(driverId) } }) diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index efcad14035..601dde6c63 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -25,6 +25,7 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import scala.collection.mutable.Map import scala.concurrent.duration._ +import scala.language.postfixOps import org.mockito.ArgumentCaptor import org.mockito.Matchers.{any, eq => meq} diff --git a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala index 9e204f5cc3..478f0690f8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import scala.concurrent.duration._ +import scala.language.postfixOps import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 1579b614ea..60b5955321 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -115,7 +115,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = { if (!hasDequeuedSpeculatedTask) { hasDequeuedSpeculatedTask = true - Some(0, TaskLocality.PROCESS_LOCAL) + Some((0, TaskLocality.PROCESS_LOCAL)) } else { None } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index f3d0bc1967..481603b403 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -299,7 +299,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") val d3 = d.map { i => w(i) -> (0 to (i % 5)) }.setName("shuffle input 2") val d4 = d2.cogroup(d3, numSlices).map { case (k, (v1, v2)) => - w(k) -> (v1.size, v2.size) + (w(k), (v1.size, v2.size)) } d4.setName("A Cogroup") d4.collectAsMap() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index 6b52c10b2c..f1392e9db6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -86,8 +86,8 @@ class TaskSetBlacklistSuite extends SparkFunSuite { Seq("exec1", "exec2").foreach { exec => assert( execToFailures(exec).taskToFailureCountAndFailureTime === Map( - 0 -> (1, 0), - 1 -> (1, 0) + 0 -> ((1, 0)), + 1 -> ((1, 0)) ) ) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 7c3922e47f..eaec098b8d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -276,7 +276,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("kryo with collect for specialized tuples") { - assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11)) + assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === ((1, 11))) } test("kryo with SerializableHyperLogLog") { @@ -475,7 +475,7 @@ class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSpar val deserializationStream = serInstance.deserializeStream(new ByteArrayInputStream(worldWorld)) assert(deserializationStream.readValue[Any]() === world) deserializationStream.close() - assert(serInstance.deserialize[Any](helloHello) === (hello, hello)) + assert(serInstance.deserialize[Any](helloHello) === ((hello, hello))) } } diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala index fd9add7690..dcae78900f 100644 --- a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -115,7 +115,7 @@ class TimeStampedHashMapSuite extends SparkFunSuite { testMap2("k1") = "v1" testMap2 --= keys assert(testMap2.size === 1) - assert(testMap2.iterator.toSeq.head === ("k1", "v1")) + assert(testMap2.iterator.toSeq.head === (("k1", "v1"))) // + val testMap3 = testMap2 + (("k0", "v0")) diff --git a/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala index aaf79ebd4f..b36d6be231 100644 --- a/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/VersionUtilsSuite.scala @@ -58,10 +58,10 @@ class VersionUtilsSuite extends SparkFunSuite { } test("Parse Spark major and minor versions") { - assert(majorMinorVersion("2.0") === (2, 0)) - assert(majorMinorVersion("12.10.11") === (12, 10)) - assert(majorMinorVersion("2.0.1-SNAPSHOT") === (2, 0)) - assert(majorMinorVersion("2.0.x") === (2, 0)) + assert(majorMinorVersion("2.0") === ((2, 0))) + assert(majorMinorVersion("12.10.11") === ((12, 10))) + assert(majorMinorVersion("2.0.1-SNAPSHOT") === ((2, 0))) + assert(majorMinorVersion("2.0.x") === ((2, 0))) withClue("majorMinorVersion parsing should fail for invalid major version number") { intercept[IllegalArgumentException] { majorMinorVersion("2z.0") diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index a2a6d70386..6b4e92879d 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -181,9 +181,9 @@ class AppendOnlyMapSuite extends SparkFunSuite { // Should be sorted by key assert(it.hasNext) var previous = it.next() - assert(previous == (null, "happy new year!")) + assert(previous == ((null, "happy new year!"))) previous = it.next() - assert(previous == ("1", "2014")) + assert(previous == (("1", "2014"))) while (it.hasNext) { val kv = it.next() assert(kv._1.toInt > previous._1.toInt) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 6bcc601e13..47173b89e9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -388,13 +388,13 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { sorter.insertAll(elements) assert(sorter.numSpills > 0, "sorter did not spill") val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList)) - assert(iter.next() === (0, Nil)) - assert(iter.next() === (1, List((1, 1)))) - assert(iter.next() === (2, (0 until 1000).map(x => (2, 2)).toList)) - assert(iter.next() === (3, Nil)) - assert(iter.next() === (4, Nil)) - assert(iter.next() === (5, List((5, 5)))) - assert(iter.next() === (6, Nil)) + assert(iter.next() === ((0, Nil))) + assert(iter.next() === ((1, List((1, 1))))) + assert(iter.next() === ((2, (0 until 1000).map(x => (2, 2)).toList))) + assert(iter.next() === ((3, Nil))) + assert(iter.next() === ((4, Nil))) + assert(iter.next() === ((5, List((5, 5))))) + assert(iter.next() === ((6, Nil))) sorter.stop() } diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 335ecb9320..08a3200288 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -75,7 +75,7 @@ class OpenHashMapSuite extends SparkFunSuite with Matchers { for ((k, v) <- map) { set.add((k, v)) } - val expected = (1 to 1000).map(x => (x.toString, x)) :+ (null.asInstanceOf[String], -1) + val expected = (1 to 1000).map(x => (x.toString, x)) :+ ((null.asInstanceOf[String], -1)) assert(set === expected.toSet) } @@ -103,7 +103,8 @@ class OpenHashMapSuite extends SparkFunSuite with Matchers { for ((k, v) <- map) { set.add((k, v)) } - val expected = (1 to 1000).map(_.toString).map(x => (x, x)) :+ (null.asInstanceOf[String], "-1") + val expected = + (1 to 1000).map(_.toString).map(x => (x, x)) :+ ((null.asInstanceOf[String], "-1")) assert(set === expected.toSet) } diff --git a/dev/change-scala-version.sh b/dev/change-scala-version.sh index d7975dfb64..022e68c279 100755 --- a/dev/change-scala-version.sh +++ b/dev/change-scala-version.sh @@ -19,7 +19,7 @@ set -e -VALID_VERSIONS=( 2.10 2.11 ) +VALID_VERSIONS=( 2.11 2.12 ) usage() { echo "Usage: $(basename $0) [-h|--help] @@ -45,7 +45,7 @@ check_scala_version() { check_scala_version "$TO_VERSION" if [ $TO_VERSION = "2.11" ]; then - FROM_VERSION="2.10" + FROM_VERSION="2.12" else FROM_VERSION="2.11" fi diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh deleted file mode 100755 index b718d94f84..0000000000 --- a/dev/change-version-to-2.10.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# This script exists for backwards compatibility. Use change-scala-version.sh instead. -echo "This script is deprecated. Please instead run: change-scala-version.sh 2.10" - -$(dirname $0)/change-scala-version.sh 2.10 diff --git a/dev/change-version-to-2.11.sh b/dev/change-version-to-2.11.sh deleted file mode 100755 index 93087959a3..0000000000 --- a/dev/change-version-to-2.11.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# This script exists for backwards compatibility. Use change-scala-version.sh instead. -echo "This script is deprecated. Please instead run: change-scala-version.sh 2.11" - -$(dirname $0)/change-scala-version.sh 2.11 diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index a72307a28a..9bf2899e34 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -155,10 +155,10 @@ if [[ "$1" == "package" ]]; then cd spark-$SPARK_VERSION-bin-$NAME - # TODO There should probably be a flag to make-distribution to allow 2.10 support - if [[ $FLAGS == *scala-2.10* ]]; then - ./dev/change-scala-version.sh 2.10 - fi + # TODO There should probably be a flag to make-distribution to allow 2.12 support + #if [[ $FLAGS == *scala-2.12* ]]; then + # ./dev/change-scala-version.sh 2.12 + #fi export ZINC_PORT=$ZINC_PORT echo "Creating distribution: $NAME ($FLAGS)" @@ -305,9 +305,9 @@ if [[ "$1" == "publish-snapshot" ]]; then export ZINC_PORT=$(python -S -c "import random; print random.randrange(3030,4030)") $MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES deploy - ./dev/change-scala-version.sh 2.10 - $MVN -DzincPort=$ZINC_PORT -Dscala-2.10 --settings $tmp_settings \ - -DskipTests $PUBLISH_PROFILES clean deploy + #./dev/change-scala-version.sh 2.12 + #$MVN -DzincPort=$ZINC_PORT -Pscala-2.12 --settings $tmp_settings \ + # -DskipTests $PUBLISH_PROFILES clean deploy # Clean-up Zinc nailgun process /usr/sbin/lsof -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill @@ -342,16 +342,13 @@ if [[ "$1" == "publish-release" ]]; then $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $PUBLISH_PROFILES clean install - ./dev/change-scala-version.sh 2.10 - - $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -Dscala-2.10 \ - -DskipTests $PUBLISH_PROFILES clean install + #./dev/change-scala-version.sh 2.12 + #$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -Pscala-2.12 \ + # -DskipTests $PUBLISH_PROFILES clean install # Clean-up Zinc nailgun process /usr/sbin/lsof -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill - ./dev/change-version-to-2.10.sh - pushd $tmp_repo/org/apache/spark # Remove any extra files generated during install diff --git a/docs/building-spark.md b/docs/building-spark.md index 815843c54a..69d83023b2 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -91,14 +91,6 @@ like ZooKeeper and Hadoop itself. ./build/mvn -Pmesos -DskipTests clean package -## Building for Scala 2.10 -To produce a Spark package compiled with Scala 2.10, use the `-Dscala-2.10` property: - - ./dev/change-scala-version.sh 2.10 - ./build/mvn -Pyarn -Dscala-2.10 -DskipTests clean package - -Note that support for Scala 2.10 is deprecated as of Spark 2.1.0 and may be removed in Spark 2.3.0. - ## Building submodules individually It's possible to build Spark sub-modules using the `mvn -pl` option. diff --git a/docs/index.md b/docs/index.md index 81ed4653b4..07b6b17101 100644 --- a/docs/index.md +++ b/docs/index.md @@ -31,8 +31,7 @@ uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scal ({{site.SCALA_BINARY_VERSION}}.x). Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0. - -Note that support for Scala 2.10 is deprecated as of Spark 2.1.0, and may be removed in Spark 2.3.0. +Support for Scala 2.10 was removed as of 2.3.0. # Running the Examples and Shell diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala index 8f8262db37..8441b5a9dd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/AggregateMessagesExample.scala @@ -52,7 +52,7 @@ object AggregateMessagesExample { triplet => { // Map Function if (triplet.srcAttr > triplet.dstAttr) { // Send message to destination vertex containing counter and age - triplet.sendToDst(1, triplet.srcAttr) + triplet.sendToDst((1, triplet.srcAttr)) } }, // Add counter and age diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala index ae6057758d..8985c8565c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala @@ -38,7 +38,7 @@ abstract class AbstractParams[T: TypeTag] { */ override def toString: String = { val tpe = tag.tpe - val allAccessors = tpe.declarations.collect { + val allAccessors = tpe.decls.collect { case m: MethodSymbol if m.isCaseAccessor => m } val mirror = runtimeMirror(getClass.getClassLoader) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index b9a612d96a..958361a668 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -29,8 +29,6 @@ import org.apache.spark.sql.types._ object SparkSQLExample { // $example on:create_ds$ - // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, - // you can use custom classes that implement the Product interface case class Person(name: String, age: Long) // $example off:create_ds$ diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 2b925774a2..5ea52b6ad3 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -259,8 +259,8 @@ object KafkaRDD { messageHandler: MessageAndMetadata[K, V] => R ): KafkaRDD[K, V, U, T, R] = { val leaders = untilOffsets.map { case (tp, lo) => - tp -> (lo.host, lo.port) - }.toMap + tp -> ((lo.host, lo.port)) + } val offsetRanges = fromOffsets.map { case (tp, fo) => val uo = untilOffsets(tp) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index 23430179f1..3b96a420b3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -63,5 +63,5 @@ object EdgeContext { * }}} */ def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]): Some[(VertexId, VertexId, VD, VD, ED)] = - Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) + Some((edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr)) } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 6c0c3ebcae..481ff20e42 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -229,17 +229,17 @@ abstract class AbstractCommandBuilder { return scala; } String sparkHome = getSparkHome(); - File scala210 = new File(sparkHome, "launcher/target/scala-2.10"); + //File scala212 = new File(sparkHome, "launcher/target/scala-2.12"); File scala211 = new File(sparkHome, "launcher/target/scala-2.11"); - checkState(!scala210.isDirectory() || !scala211.isDirectory(), - "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + - "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); - if (scala210.isDirectory()) { - return "2.10"; - } else { - checkState(scala211.isDirectory(), "Cannot find any build directories."); - return "2.11"; - } + //checkState(!scala210.isDirectory() || !scala211.isDirectory(), + // "Presence of build for multiple Scala versions detected.\n" + + // "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); + //if (scala212.isDirectory()) { + // return "2.12"; + //} else { + checkState(scala211.isDirectory(), "Cannot find any build directories."); + return "2.11"; + //} } String getSparkHome() { diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 07f3bc2728..66c5362441 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -856,7 +856,7 @@ object SparseMatrix { var prevRow = -1 var prevVal = 0.0 // Append a dummy entry to include the last one at the end of the loop. - (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) => + (sortedEntries.view :+ ((numRows, numCols, 1.0))).foreach { case (i, j, v) => if (v != 0) { if (i == prevRow && j == prevCol) { prevVal += v diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala index 9f82020868..7fb9034d65 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/MatricesSuite.scala @@ -633,22 +633,22 @@ class MatricesSuite extends SparkMLFunSuite { dnMap.put((i, j), value) } assert(dnMap.size === 6) - assert(dnMap(0, 0) === 1.0) - assert(dnMap(1, 0) === 2.0) - assert(dnMap(2, 0) === 0.0) - assert(dnMap(0, 1) === 0.0) - assert(dnMap(1, 1) === 4.0) - assert(dnMap(2, 1) === 5.0) + assert(dnMap((0, 0)) === 1.0) + assert(dnMap((1, 0)) === 2.0) + assert(dnMap((2, 0)) === 0.0) + assert(dnMap((0, 1)) === 0.0) + assert(dnMap((1, 1)) === 4.0) + assert(dnMap((2, 1)) === 5.0) val spMap = MutableMap[(Int, Int), Double]() sp.foreachActive { (i, j, value) => spMap.put((i, j), value) } assert(spMap.size === 4) - assert(spMap(0, 0) === 1.0) - assert(spMap(1, 0) === 2.0) - assert(spMap(1, 1) === 4.0) - assert(spMap(2, 1) === 5.0) + assert(spMap((0, 0)) === 1.0) + assert(spMap((1, 0)) === 2.0) + assert(spMap((1, 1)) === 4.0) + assert(spMap((2, 1)) === 5.0) } test("horzcat, vertcat, eye, speye") { diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index 0d6e9034e5..4aa4c3617e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -436,7 +436,7 @@ private[ml] object EnsembleModelReadWrite { val treesMetadataRDD: RDD[(Int, (Metadata, Double))] = sql.read.parquet(treesMetadataPath) .select("treeID", "metadata", "weights").as[(Int, String, Double)].rdd.map { case (treeID: Int, json: String, weights: Double) => - treeID -> (DefaultParamsReader.parseMetadata(json, treeClassName), weights) + treeID -> ((DefaultParamsReader.parseMetadata(json, treeClassName), weights)) } val treesMetadataWeights = treesMetadataRDD.sortByKey().values.collect() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 2b2b5fe49e..bf9b4cfe15 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -797,7 +797,7 @@ object SparseMatrix { var prevRow = -1 var prevVal = 0.0 // Append a dummy entry to include the last one at the end of the loop. - (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) => + (sortedEntries.view :+ ((numRows, numCols, 1.0))).foreach { case (i, j, v) => if (v != 0) { if (i == prevRow && j == prevCol) { prevVal += v diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 20d68a34bf..7caacd13b3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -275,7 +275,7 @@ class BlockMatrix @Since("1.3.0") ( val rows = blocks.flatMap { case ((blockRowIdx, blockColIdx), mat) => mat.rowIter.zipWithIndex.map { case (vector, rowIdx) => - blockRowIdx * rowsPerBlock + rowIdx -> (blockColIdx, vector.asBreeze) + blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector.asBreeze)) } }.groupByKey().map { case (rowIdx, vectors) => val numberNonZeroPerRow = vectors.map(_._2.activeSize).sum.toDouble / cols.toDouble @@ -286,7 +286,7 @@ class BlockMatrix @Since("1.3.0") ( BDV.zeros[Double](cols) } - vectors.foreach { case (blockColIdx: Int, vec: BV[Double]) => + vectors.foreach { case (blockColIdx: Int, vec: BV[_]) => val offset = colsPerBlock * blockColIdx wholeVector(offset until Math.min(cols, offset + colsPerBlock)) := vec } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 0a0fea255c..45d3f9b4c5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -898,7 +898,7 @@ class ALSStorageSuite // check final factor RDD default storage levels val defaultFactorRDDs = sc.getPersistentRDDs.collect { case (id, rdd) if rdd.name == "userFactors" || rdd.name == "itemFactors" => - rdd.name -> (id, rdd.getStorageLevel) + rdd.name -> ((id, rdd.getStorageLevel)) }.toMap defaultFactorRDDs.foreach { case (_, (id, level)) => assert(level == StorageLevel.MEMORY_AND_DISK) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 086bb211a9..8906e52fae 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -151,7 +151,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { // Check: topTopicAssignments // Make sure it assigns a topic to each term appearing in each doc. val topTopicAssignments: Map[Long, (Array[Int], Array[Int])] = - model.topicAssignments.collect().map(x => x._1 -> (x._2, x._3)).toMap + model.topicAssignments.collect().map(x => x._1 -> ((x._2, x._3))).toMap assert(topTopicAssignments.keys.max < tinyCorpus.length) tinyCorpus.foreach { case (docID: Long, doc: Vector) => if (topTopicAssignments.contains(docID)) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 93c00d8097..6736e7d3db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -241,22 +241,22 @@ class MatricesSuite extends SparkFunSuite { dnMap.put((i, j), value) } assert(dnMap.size === 6) - assert(dnMap(0, 0) === 1.0) - assert(dnMap(1, 0) === 2.0) - assert(dnMap(2, 0) === 0.0) - assert(dnMap(0, 1) === 0.0) - assert(dnMap(1, 1) === 4.0) - assert(dnMap(2, 1) === 5.0) + assert(dnMap((0, 0)) === 1.0) + assert(dnMap((1, 0)) === 2.0) + assert(dnMap((2, 0)) === 0.0) + assert(dnMap((0, 1)) === 0.0) + assert(dnMap((1, 1)) === 4.0) + assert(dnMap((2, 1)) === 5.0) val spMap = MutableMap[(Int, Int), Double]() sp.foreachActive { (i, j, value) => spMap.put((i, j), value) } assert(spMap.size === 4) - assert(spMap(0, 0) === 1.0) - assert(spMap(1, 0) === 2.0) - assert(spMap(1, 1) === 4.0) - assert(spMap(2, 1) === 5.0) + assert(spMap((0, 0)) === 1.0) + assert(spMap((1, 0)) === 2.0) + assert(spMap((1, 1)) === 4.0) + assert(spMap((2, 1)) === 5.0) } test("horzcat, vertcat, eye, speye") { diff --git a/pom.xml b/pom.xml index f124ba4500..bea2798cc8 100644 --- a/pom.xml +++ b/pom.xml @@ -498,7 +498,7 @@ org.slf4j jcl-over-slf4j ${slf4j.version} - + runtime log4j @@ -1859,9 +1859,9 @@ ${antlr4.version} - ${jline.groupid} + jline jline - ${jline.version} + 2.12.1 org.apache.commons @@ -1933,6 +1933,7 @@ --> org.jboss.netty org.codehaus.groovy + *:*_2.10 true @@ -1987,6 +1988,8 @@ -unchecked -deprecation -feature + -explaintypes + -Yno-adapted-args -Xms1024m @@ -2586,15 +2589,27 @@ - scala-2.10 + test-java-home - scala-2.10 + env.JAVA_HOME - 2.10.6 - 2.10 - ${scala.version} - org.scala-lang + ${env.JAVA_HOME} + + + + + + scala-2.11 + + + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 89b0c7a3ab..41f3a0451a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -87,19 +87,11 @@ object SparkBuild extends PomBuild { val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty override val profiles = { - val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match { + Properties.envOrNone("SBT_MAVEN_PROFILES") match { case None => Seq("sbt") case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq } - - if (System.getProperty("scala-2.10") == "") { - // To activate scala-2.10 profile, replace empty property value to non-empty value - // in the same way as Maven which handles -Dname as -Dname=true before executes build process. - // see: https://github.com/apache/maven/blob/maven-3.0.4/maven-embedder/src/main/java/org/apache/maven/cli/MavenCli.java#L1082 - System.setProperty("scala-2.10", "true") - } - profiles } Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { @@ -234,9 +226,7 @@ object SparkBuild extends PomBuild { }, javacJVMVersion := "1.8", - // SBT Scala 2.10 build still doesn't support Java 8, because scalac 2.10 doesn't, but, - // it also doesn't touch Java 8 code and it's OK to emit Java 7 bytecode in this case - scalacJVMVersion := (if (System.getProperty("scala-2.10") == "true") "1.7" else "1.8"), + scalacJVMVersion := "1.8", javacOptions in Compile ++= Seq( "-encoding", "UTF-8", @@ -477,7 +467,6 @@ object OldDeps { def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq( name := "old-deps", - scalaVersion := "2.10.5", libraryDependencies := allPreviousArtifactKeys.value.flatten ) } @@ -756,13 +745,7 @@ object CopyDependencies { object TestSettings { import BuildCommons._ - private val scalaBinaryVersion = - if (System.getProperty("scala-2.10") == "true") { - "2.10" - } else { - "2.11" - } - + private val scalaBinaryVersion = "2.11" lazy val settings = Seq ( // Fork new JVMs for tests and set Java options for those fork := true, diff --git a/python/run-tests.py b/python/run-tests.py index b2e50435bb..afd3d29a0f 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -54,7 +54,8 @@ FAILURE_REPORTING_LOCK = Lock() LOGGER = logging.getLogger() # Find out where the assembly jars are located. -for scala in ["2.11", "2.10"]: +# Later, add back 2.12 to this list: +for scala in ["2.11"]: build_dir = os.path.join(SPARK_HOME, "assembly", "target", "scala-" + scala) if os.path.isdir(build_dir): SPARK_DIST_CLASSPATH = os.path.join(build_dir, "jars", "*") diff --git a/repl/pom.xml b/repl/pom.xml index 6d133a3cff..51eb9b60dd 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -32,8 +32,8 @@ repl - scala-2.10/src/main/scala - scala-2.10/src/test/scala + scala-2.11/src/main/scala + scala-2.11/src/test/scala @@ -71,7 +71,7 @@ ${scala.version} - ${jline.groupid} + jline jline @@ -170,23 +170,17 @@ + + + diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala deleted file mode 100644 index fba321be91..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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 org.apache.spark.internal.Logging - -object Main extends Logging { - - initializeLogIfNecessary(true) - Signaling.cancelOnInterrupt() - - private var _interp: SparkILoop = _ - - def interp = _interp - - def interp_=(i: SparkILoop) { _interp = i } - - def main(args: Array[String]) { - _interp = new SparkILoop - _interp.process(args) - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala deleted file mode 100644 index be9b79021d..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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 scala.tools.nsc.{CompilerCommand, Settings} - -import org.apache.spark.annotation.DeveloperApi - -/** - * Command class enabling Spark-specific command line options (provided by - * org.apache.spark.repl.SparkRunnerSettings). - * - * @example new SparkCommandLine(Nil).settings - * - * @param args The list of command line arguments - * @param settings The underlying settings to associate with this set of - * command-line options - */ -@DeveloperApi -class SparkCommandLine(args: List[String], override val settings: Settings) - extends CompilerCommand(args, settings) { - def this(args: List[String], error: String => Unit) { - this(args, new SparkRunnerSettings(error)) - } - - def this(args: List[String]) { - // scalastyle:off println - this(args, str => Console.println("Error: " + str)) - // scalastyle:on println - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala deleted file mode 100644 index 2b5d56a895..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ /dev/null @@ -1,114 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Paul Phillips - */ - -package org.apache.spark.repl - -import scala.tools.nsc._ -import scala.tools.nsc.interpreter._ - -import scala.reflect.internal.util.BatchSourceFile -import scala.tools.nsc.ast.parser.Tokens.EOF - -import org.apache.spark.internal.Logging - -private[repl] trait SparkExprTyper extends Logging { - val repl: SparkIMain - - import repl._ - import global.{ reporter => _, Import => _, _ } - import definitions._ - import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name } - import naming.freshInternalVarName - - object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] { - def applyRule[T](code: String, rule: UnitParser => T): T = { - reporter.reset() - val scanner = newUnitParser(code) - val result = rule(scanner) - - if (!reporter.hasErrors) - scanner.accept(EOF) - - result - } - - def defns(code: String) = stmts(code) collect { case x: DefTree => x } - def expr(code: String) = applyRule(code, _.expr()) - def stmts(code: String) = applyRule(code, _.templateStats()) - def stmt(code: String) = stmts(code).last // guaranteed nonempty - } - - /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ - def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") { - var isIncomplete = false - reporter.withIncompleteHandler((_, _) => isIncomplete = true) { - val trees = codeParser.stmts(line) - if (reporter.hasErrors) { - Some(Nil) - } else if (isIncomplete) { - None - } else { - Some(trees) - } - } - } - // def parsesAsExpr(line: String) = { - // import codeParser._ - // (opt expr line).isDefined - // } - - def symbolOfLine(code: String): Symbol = { - def asExpr(): Symbol = { - val name = freshInternalVarName() - // Typing it with a lazy val would give us the right type, but runs - // into compiler bugs with things like existentials, so we compile it - // behind a def and strip the NullaryMethodType which wraps the expr. - val line = "def " + name + " = {\n" + code + "\n}" - - interpretSynthetic(line) match { - case IR.Success => - val sym0 = symbolOfTerm(name) - // drop NullaryMethodType - val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) - if (sym.info.typeSymbol eq UnitClass) NoSymbol else sym - case _ => NoSymbol - } - } - def asDefn(): Symbol = { - val old = repl.definedSymbolList.toSet - - interpretSynthetic(code) match { - case IR.Success => - repl.definedSymbolList filterNot old match { - case Nil => NoSymbol - case sym :: Nil => sym - case syms => NoSymbol.newOverloaded(NoPrefix, syms) - } - case _ => NoSymbol - } - } - beQuietDuring(asExpr()) orElse beQuietDuring(asDefn()) - } - - private var typeOfExpressionDepth = 0 - def typeOfExpression(expr: String, silent: Boolean = true): Type = { - if (typeOfExpressionDepth > 2) { - logDebug("Terminating typeOfExpression recursion for expression: " + expr) - return NoType - } - typeOfExpressionDepth += 1 - // Don't presently have a good way to suppress undesirable success output - // while letting errors through, so it is first trying it silently: if there - // is an error, and errors are desired, then it re-evaluates non-silently - // to induce the error message. - try beSilentDuring(symbolOfLine(expr).tpe) match { - case NoType if !silent => symbolOfLine(expr).tpe // generate error - case tpe => tpe - } - finally typeOfExpressionDepth -= 1 - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala deleted file mode 100644 index 955be17a73..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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 scala.tools.nsc - -import org.apache.spark.annotation.DeveloperApi - -// NOTE: Forced to be public (and in scala.tools.nsc package) to access the -// settings "explicitParentLoader" method - -/** - * Provides exposure for the explicitParentLoader method on settings instances. - */ -@DeveloperApi -object SparkHelper { - /** - * Retrieves the explicit parent loader for the provided settings. - * - * @param settings The settings whose explicit parent loader to retrieve - * - * @return The Optional classloader representing the explicit parent loader - */ - @DeveloperApi - def explicitParentLoader(settings: Settings) = settings.explicitParentLoader -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala deleted file mode 100644 index b7237a6ce8..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ /dev/null @@ -1,1145 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Alexander Spoon - */ - -package org.apache.spark.repl - - -import java.net.URL - -import scala.reflect.io.AbstractFile -import scala.tools.nsc._ -import scala.tools.nsc.backend.JavaPlatform -import scala.tools.nsc.interpreter._ -import scala.tools.nsc.interpreter.{Results => IR} -import Predef.{println => _, _} -import java.io.{BufferedReader, FileReader} -import java.net.URI -import java.util.concurrent.locks.ReentrantLock -import scala.sys.process.Process -import scala.tools.nsc.interpreter.session._ -import scala.util.Properties.{jdkHome, javaVersion} -import scala.tools.util.{Javap} -import scala.annotation.tailrec -import scala.collection.mutable.ListBuffer -import scala.concurrent.ops -import scala.tools.nsc.util._ -import scala.tools.nsc.interpreter._ -import scala.tools.nsc.io.{File, Directory} -import scala.reflect.NameTransformer._ -import scala.tools.nsc.util.ScalaClassLoader._ -import scala.tools.util._ -import scala.language.{implicitConversions, existentials, postfixOps} -import scala.reflect.{ClassTag, classTag} -import scala.tools.reflect.StdRuntimeTags._ - -import java.lang.{Class => jClass} -import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} - -import org.apache.spark.SparkConf -import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.util.Utils - -/** The Scala interactive shell. It provides a read-eval-print loop - * around the Interpreter class. - * After instantiation, clients should call the main() method. - * - * If no in0 is specified, then input will come from the console, and - * the class will attempt to provide input editing feature such as - * input history. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - * @version 1.2 - */ -@DeveloperApi -class SparkILoop( - private val in0: Option[BufferedReader], - protected val out: JPrintWriter, - val master: Option[String] -) extends AnyRef with LoopCommands with SparkILoopInit with Logging { - def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master)) - def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) - def this() = this(None, new JPrintWriter(Console.out, true), None) - - private var in: InteractiveReader = _ // the input stream from which commands come - - // NOTE: Exposed in package for testing - private[repl] var settings: Settings = _ - - private[repl] var intp: SparkIMain = _ - - @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp - @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i - - /** Having inherited the difficult "var-ness" of the repl instance, - * I'm trying to work around it by moving operations into a class from - * which it will appear a stable prefix. - */ - private def onIntp[T](f: SparkIMain => T): T = f(intp) - - class IMainOps[T <: SparkIMain](val intp: T) { - import intp._ - import global._ - - def printAfterTyper(msg: => String) = - intp.reporter printMessage afterTyper(msg) - - /** Strip NullaryMethodType artifacts. */ - private def replInfo(sym: Symbol) = { - sym.info match { - case NullaryMethodType(restpe) if sym.isAccessor => restpe - case info => info - } - } - def echoTypeStructure(sym: Symbol) = - printAfterTyper("" + deconstruct.show(replInfo(sym))) - - def echoTypeSignature(sym: Symbol, verbose: Boolean) = { - if (verbose) SparkILoop.this.echo("// Type signature") - printAfterTyper("" + replInfo(sym)) - - if (verbose) { - SparkILoop.this.echo("\n// Internal Type structure") - echoTypeStructure(sym) - } - } - } - implicit def stabilizeIMain(intp: SparkIMain) = new IMainOps[intp.type](intp) - - /** TODO - - * -n normalize - * -l label with case class parameter names - * -c complete - leave nothing out - */ - private def typeCommandInternal(expr: String, verbose: Boolean): Result = { - onIntp { intp => - val sym = intp.symbolOfLine(expr) - if (sym.exists) intp.echoTypeSignature(sym, verbose) - else "" - } - } - - // NOTE: Must be public for visibility - @DeveloperApi - var sparkContext: SparkContext = _ - - override def echoCommandMessage(msg: String) { - intp.reporter printMessage msg - } - - // def isAsync = !settings.Yreplsync.value - private[repl] def isAsync = false - // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) - private def history = in.history - - /** The context class loader at the time this object was created */ - protected val originalClassLoader = Utils.getContextOrSparkClassLoader - - // classpath entries added via :cp - private var addedClasspath: String = "" - - /** A reverse list of commands to replay if the user requests a :replay */ - private var replayCommandStack: List[String] = Nil - - /** A list of commands to replay if the user requests a :replay */ - private def replayCommands = replayCommandStack.reverse - - /** Record a command for replay should the user request a :replay */ - private def addReplay(cmd: String) = replayCommandStack ::= cmd - - private def savingReplayStack[T](body: => T): T = { - val saved = replayCommandStack - try body - finally replayCommandStack = saved - } - private def savingReader[T](body: => T): T = { - val saved = in - try body - finally in = saved - } - - - private def sparkCleanUp() { - echo("Stopping spark context.") - intp.beQuietDuring { - command("sc.stop()") - } - } - /** Close the interpreter and set the var to null. */ - private def closeInterpreter() { - if (intp ne null) { - sparkCleanUp() - intp.close() - intp = null - } - } - - class SparkILoopInterpreter extends SparkIMain(settings, out) { - outer => - - override private[repl] lazy val formatting = new Formatting { - def prompt = SparkILoop.this.prompt - } - override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) - } - - /** - * Constructs a new interpreter. - */ - protected def createInterpreter() { - require(settings != null) - - if (addedClasspath != "") settings.classpath.append(addedClasspath) - val addedJars = - if (Utils.isWindows) { - // Strip any URI scheme prefix so we can add the correct path to the classpath - // e.g. file:/C:/my/path.jar -> C:/my/path.jar - getAddedJars().map { jar => new URI(jar).getPath.stripPrefix("/") } - } else { - // We need new URI(jar).getPath here for the case that `jar` includes encoded white space (%20). - getAddedJars().map { jar => new URI(jar).getPath } - } - // work around for Scala bug - val totalClassPath = addedJars.foldLeft( - settings.classpath.value)((l, r) => ClassPath.join(l, r)) - this.settings.classpath.value = totalClassPath - - intp = new SparkILoopInterpreter - } - - /** print a friendly help message */ - private def helpCommand(line: String): Result = { - if (line == "") helpSummary() - else uniqueCommand(line) match { - case Some(lc) => echo("\n" + lc.longHelp) - case _ => ambiguousError(line) - } - } - private def helpSummary() = { - val usageWidth = commands map (_.usageMsg.length) max - val formatStr = "%-" + usageWidth + "s %s %s" - - echo("All commands can be abbreviated, e.g. :he instead of :help.") - echo("Those marked with a * have more detailed help, e.g. :help imports.\n") - - commands foreach { cmd => - val star = if (cmd.hasLongHelp) "*" else " " - echo(formatStr.format(cmd.usageMsg, star, cmd.help)) - } - } - private def ambiguousError(cmd: String): Result = { - matchingCommands(cmd) match { - case Nil => echo(cmd + ": no such command. Type :help for help.") - case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") - } - Result(true, None) - } - private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) - private def uniqueCommand(cmd: String): Option[LoopCommand] = { - // this lets us add commands willy-nilly and only requires enough command to disambiguate - matchingCommands(cmd) match { - case List(x) => Some(x) - // exact match OK even if otherwise appears ambiguous - case xs => xs find (_.name == cmd) - } - } - private var fallbackMode = false - - private def toggleFallbackMode() { - val old = fallbackMode - fallbackMode = !old - System.setProperty("spark.repl.fallback", fallbackMode.toString) - echo(s""" - |Switched ${if (old) "off" else "on"} fallback mode without restarting. - | If you have defined classes in the repl, it would - |be good to redefine them incase you plan to use them. If you still run - |into issues it would be good to restart the repl and turn on `:fallback` - |mode as first command. - """.stripMargin) - } - - /** Show the history */ - private lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { - override def usage = "[num]" - def defaultLines = 20 - - def apply(line: String): Result = { - if (history eq NoHistory) - return "No history available." - - val xs = words(line) - val current = history.index - val count = try xs.head.toInt catch { case _: Exception => defaultLines } - val lines = history.asStrings takeRight count - val offset = current - lines.size + 1 - - for ((line, index) <- lines.zipWithIndex) - echo("%3d %s".format(index + offset, line)) - } - } - - // When you know you are most likely breaking into the middle - // of a line being typed. This softens the blow. - private[repl] def echoAndRefresh(msg: String) = { - echo("\n" + msg) - in.redrawLine() - } - private[repl] def echo(msg: String) = { - out println msg - out.flush() - } - private def echoNoNL(msg: String) = { - out print msg - out.flush() - } - - /** Search the history */ - private def searchHistory(_cmdline: String) { - val cmdline = _cmdline.toLowerCase - val offset = history.index - history.size + 1 - - for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) - echo("%d %s".format(index + offset, line)) - } - - private var currentPrompt = Properties.shellPromptString - - /** - * Sets the prompt string used by the REPL. - * - * @param prompt The new prompt string - */ - @DeveloperApi - def setPrompt(prompt: String) = currentPrompt = prompt - - /** - * Represents the current prompt string used by the REPL. - * - * @return The current prompt string - */ - @DeveloperApi - def prompt = currentPrompt - - import LoopCommand.{ cmd, nullary } - - /** Standard commands */ - private lazy val standardCommands = List( - cmd("cp", "", "add a jar or directory to the classpath", addClasspath), - cmd("help", "[command]", "print this summary or command-specific help", helpCommand), - historyCommand, - cmd("h?", "", "search the history", searchHistory), - cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand), - cmd("implicits", "[-v]", "show the implicits in scope", implicitsCommand), - cmd("javap", "", "disassemble a file or class name", javapCommand), - cmd("load", "", "load and interpret a Scala file", loadCommand), - nullary("paste", "enter paste mode: all input up to ctrl-D compiled together", pasteCommand), -// nullary("power", "enable power user mode", powerCmd), - nullary("quit", "exit the repl", () => Result(false, None)), - nullary("replay", "reset execution and replay all previous commands", replay), - nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), - shCommand, - nullary("silent", "disable/enable automatic printing of results", verbosity), - nullary("fallback", """ - |disable/enable advanced repl changes, these fix some issues but may introduce others. - |This mode will be removed once these fixes stablize""".stripMargin, toggleFallbackMode), - cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), - nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) - ) - - /** Power user commands */ - private lazy val powerCommands: List[LoopCommand] = List( - // cmd("phase", "", "set the implicit phase for power commands", phaseCommand) - ) - - // private def dumpCommand(): Result = { - // echo("" + power) - // history.asStrings takeRight 30 foreach echo - // in.redrawLine() - // } - // private def valsCommand(): Result = power.valsDescription - - private val typeTransforms = List( - "scala.collection.immutable." -> "immutable.", - "scala.collection.mutable." -> "mutable.", - "scala.collection.generic." -> "generic.", - "java.lang." -> "jl.", - "scala.runtime." -> "runtime." - ) - - private def importsCommand(line: String): Result = { - val tokens = words(line) - val handlers = intp.languageWildcardHandlers ++ intp.importHandlers - val isVerbose = tokens contains "-v" - - handlers.filterNot(_.importedSymbols.isEmpty).zipWithIndex foreach { - case (handler, idx) => - val (types, terms) = handler.importedSymbols partition (_.name.isTypeName) - val imps = handler.implicitSymbols - val found = tokens filter (handler importsSymbolNamed _) - val typeMsg = if (types.isEmpty) "" else types.size + " types" - val termMsg = if (terms.isEmpty) "" else terms.size + " terms" - val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" - val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") - val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") - - intp.reporter.printMessage("%2d) %-30s %s%s".format( - idx + 1, - handler.importString, - statsMsg, - foundMsg - )) - } - } - - private def implicitsCommand(line: String): Result = onIntp { intp => - import intp._ - import global._ - - def p(x: Any) = intp.reporter.printMessage("" + x) - - // If an argument is given, only show a source with that - // in its name somewhere. - val args = line split "\\s+" - val filtered = intp.implicitSymbolsBySource filter { - case (source, syms) => - (args contains "-v") || { - if (line == "") (source.fullName.toString != "scala.Predef") - else (args exists (source.name.toString contains _)) - } - } - - if (filtered.isEmpty) - return "No implicits have been imported other than those in Predef." - - filtered foreach { - case (source, syms) => - p("/* " + syms.size + " implicit members imported from " + source.fullName + " */") - - // This groups the members by where the symbol is defined - val byOwner = syms groupBy (_.owner) - val sortedOwners = byOwner.toList sortBy { case (owner, _) => afterTyper(source.info.baseClasses indexOf owner) } - - sortedOwners foreach { - case (owner, members) => - // Within each owner, we cluster results based on the final result type - // if there are more than a couple, and sort each cluster based on name. - // This is really just trying to make the 100 or so implicits imported - // by default into something readable. - val memberGroups: List[List[Symbol]] = { - val groups = members groupBy (_.tpe.finalResultType) toList - val (big, small) = groups partition (_._2.size > 3) - val xss = ( - (big sortBy (_._1.toString) map (_._2)) :+ - (small flatMap (_._2)) - ) - - xss map (xs => xs sortBy (_.name.toString)) - } - - val ownerMessage = if (owner == source) " defined in " else " inherited from " - p(" /* " + members.size + ownerMessage + owner.fullName + " */") - - memberGroups foreach { group => - group foreach (s => p(" " + intp.symbolDefString(s))) - p("") - } - } - p("") - } - } - - private def findToolsJar() = { - val jdkPath = Directory(jdkHome) - val jar = jdkPath / "lib" / "tools.jar" toFile; - - if (jar isFile) - Some(jar) - else if (jdkPath.isDirectory) - jdkPath.deepFiles find (_.name == "tools.jar") - else None - } - private def addToolsJarToLoader() = { - val cl = findToolsJar match { - case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader) - case _ => intp.classLoader - } - if (Javap.isAvailable(cl)) { - logDebug(":javap available.") - cl - } - else { - logDebug(":javap unavailable: no tools.jar at " + jdkHome) - intp.classLoader - } - } - - private def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { - override def tryClass(path: String): Array[Byte] = { - val hd :: rest = path split '.' toList; - // If there are dots in the name, the first segment is the - // key to finding it. - if (rest.nonEmpty) { - intp optFlatName hd match { - case Some(flat) => - val clazz = flat :: rest mkString NAME_JOIN_STRING - val bytes = super.tryClass(clazz) - if (bytes.nonEmpty) bytes - else super.tryClass(clazz + MODULE_SUFFIX_STRING) - case _ => super.tryClass(path) - } - } - else { - // Look for Foo first, then Foo$, but if Foo$ is given explicitly, - // we have to drop the $ to find object Foo, then tack it back onto - // the end of the flattened name. - def className = intp flatName path - def moduleName = (intp flatName path.stripSuffix(MODULE_SUFFIX_STRING)) + MODULE_SUFFIX_STRING - - val bytes = super.tryClass(className) - if (bytes.nonEmpty) bytes - else super.tryClass(moduleName) - } - } - } - // private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap()) - private lazy val javap = - try newJavap() - catch { case _: Exception => null } - - // Still todo: modules. - private def typeCommand(line0: String): Result = { - line0.trim match { - case "" => ":type [-v] " - case s if s startsWith "-v " => typeCommandInternal(s stripPrefix "-v " trim, true) - case s => typeCommandInternal(s, false) - } - } - - private def warningsCommand(): Result = { - if (intp.lastWarnings.isEmpty) - "Can't find any cached warnings." - else - intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) } - } - - private def javapCommand(line: String): Result = { - if (javap == null) - ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome) - else if (javaVersion startsWith "1.7") - ":javap not yet working with java 1.7" - else if (line == "") - ":javap [-lcsvp] [path1 path2 ...]" - else - javap(words(line)) foreach { res => - if (res.isError) return "Failed: " + res.value - else res.show() - } - } - - private def wrapCommand(line: String): Result = { - def failMsg = "Argument to :wrap must be the name of a method with signature [T](=> T): T" - onIntp { intp => - import intp._ - import global._ - - words(line) match { - case Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => "Current execution wrapper: " + s - } - case "clear" :: Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." - } - case wrapper :: Nil => - intp.typeOfExpression(wrapper) match { - case PolyType(List(targ), MethodType(List(arg), restpe)) => - intp setExecutionWrapper intp.pathToTerm(wrapper) - "Set wrapper to '" + wrapper + "'" - case tp => - failMsg + "\nFound: " - } - case _ => failMsg - } - } - } - - private def pathToPhaseWrapper = intp.pathToTerm("$r") + ".phased.atCurrent" - // private def phaseCommand(name: String): Result = { - // val phased: Phased = power.phased - // import phased.NoPhaseName - - // if (name == "clear") { - // phased.set(NoPhaseName) - // intp.clearExecutionWrapper() - // "Cleared active phase." - // } - // else if (name == "") phased.get match { - // case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" - // case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) - // } - // else { - // val what = phased.parse(name) - // if (what.isEmpty || !phased.set(what)) - // "'" + name + "' does not appear to represent a valid phase." - // else { - // intp.setExecutionWrapper(pathToPhaseWrapper) - // val activeMessage = - // if (what.toString.length == name.length) "" + what - // else "%s (%s)".format(what, name) - - // "Active phase is now: " + activeMessage - // } - // } - // } - - /** - * Provides a list of available commands. - * - * @return The list of commands - */ - @DeveloperApi - def commands: List[LoopCommand] = standardCommands /*++ ( - if (isReplPower) powerCommands else Nil - )*/ - - private val replayQuestionMessage = - """|That entry seems to have slain the compiler. Shall I replay - |your session? I can re-run each line except the last one. - |[y/n] - """.trim.stripMargin - - private def crashRecovery(ex: Throwable): Boolean = { - echo(ex.toString) - ex match { - case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("\nUnrecoverable error.") - throw ex - case _ => - def fn(): Boolean = - try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) - catch { case _: RuntimeException => false } - - if (fn()) replay() - else echo("\nAbandoning crashed session.") - } - true - } - - /** The main read-eval-print loop for the repl. It calls - * command() for each line of input, and stops when - * command() returns false. - */ - private def loop() { - def readOneLine() = { - out.flush() - in readLine prompt - } - // return false if repl should exit - def processLine(line: String): Boolean = { - if (isAsync) { - if (!awaitInitialized()) return false - runThunks() - } - if (line eq null) false // assume null means EOF - else command(line) match { - case Result(false, _) => false - case Result(_, Some(finalLine)) => addReplay(finalLine) ; true - case _ => true - } - } - def innerLoop() { - val shouldContinue = try { - processLine(readOneLine()) - } catch {case t: Throwable => crashRecovery(t)} - if (shouldContinue) - innerLoop() - } - innerLoop() - } - - /** interpret all lines from a specified file */ - private def interpretAllFrom(file: File) { - savingReader { - savingReplayStack { - file applyReader { reader => - in = SimpleReader(reader, out, false) - echo("Loading " + file + "...") - loop() - } - } - } - } - - /** create a new interpreter and replay the given commands */ - private def replay() { - reset() - if (replayCommandStack.isEmpty) - echo("Nothing to replay.") - else for (cmd <- replayCommands) { - echo("Replaying: " + cmd) // flush because maybe cmd will have its own output - command(cmd) - echo("") - } - } - private def resetCommand() { - echo("Resetting repl state.") - if (replayCommandStack.nonEmpty) { - echo("Forgetting this session history:\n") - replayCommands foreach echo - echo("") - replayCommandStack = Nil - } - if (intp.namedDefinedTerms.nonEmpty) - echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", ")) - if (intp.definedTypes.nonEmpty) - echo("Forgetting defined types: " + intp.definedTypes.mkString(", ")) - - reset() - } - - private def reset() { - intp.reset() - // unleashAndSetPhase() - } - - /** fork a shell and run a command */ - private lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { - override def usage = "" - def apply(line: String): Result = line match { - case "" => showUsage() - case _ => - val toRun = classOf[ProcessResult].getName + "(" + string2codeQuoted(line) + ")" - intp interpret toRun - () - } - } - - private def withFile(filename: String)(action: File => Unit) { - val f = File(filename) - - if (f.exists) action(f) - else echo("That file does not exist") - } - - private def loadCommand(arg: String) = { - var shouldReplay: Option[String] = None - withFile(arg)(f => { - interpretAllFrom(f) - shouldReplay = Some(":load " + arg) - }) - Result(true, shouldReplay) - } - - private def addAllClasspath(args: Seq[String]): Unit = { - var added = false - var totalClasspath = "" - for (arg <- args) { - val f = File(arg).normalize - if (f.exists) { - added = true - addedClasspath = ClassPath.join(addedClasspath, f.path) - totalClasspath = ClassPath.join(settings.classpath.value, addedClasspath) - intp.addUrlsToClassPath(f.toURI.toURL) - sparkContext.addJar(f.toURI.toURL.getPath) - } - } - } - - private def addClasspath(arg: String): Unit = { - val f = File(arg).normalize - if (f.exists) { - addedClasspath = ClassPath.join(addedClasspath, f.path) - intp.addUrlsToClassPath(f.toURI.toURL) - sparkContext.addJar(f.toURI.toURL.getPath) - echo("Added '%s'. Your new classpath is:\n\"%s\"".format(f.path, intp.global.classPath.asClasspathString)) - } - else echo("The path '" + f + "' doesn't seem to exist.") - } - - - private def powerCmd(): Result = { - if (isReplPower) "Already in power mode." - else enablePowerMode(false) - } - - private[repl] def enablePowerMode(isDuringInit: Boolean) = { - // replProps.power setValue true - // unleashAndSetPhase() - // asyncEcho(isDuringInit, power.banner) - } - // private def unleashAndSetPhase() { -// if (isReplPower) { -// // power.unleash() -// // Set the phase to "typer" -// intp beSilentDuring phaseCommand("typer") -// } -// } - - private def asyncEcho(async: Boolean, msg: => String) { - if (async) asyncMessage(msg) - else echo(msg) - } - - private def verbosity() = { - // val old = intp.printResults - // intp.printResults = !old - // echo("Switched " + (if (old) "off" else "on") + " result printing.") - } - - /** - * Run one command submitted by the user. Two values are returned: - * (1) whether to keep running, (2) the line to record for replay, - * if any. - */ - private[repl] def command(line: String): Result = { - if (line startsWith ":") { - val cmd = line.tail takeWhile (x => !x.isWhitespace) - uniqueCommand(cmd) match { - case Some(lc) => lc(line.tail stripPrefix cmd dropWhile (_.isWhitespace)) - case _ => ambiguousError(cmd) - } - } - else if (intp.global == null) Result(false, None) // Notice failure to create compiler - else Result(true, interpretStartingWith(line)) - } - - private def readWhile(cond: String => Boolean) = { - Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) - } - - private def pasteCommand(): Result = { - echo("// Entering paste mode (ctrl-D to finish)\n") - val code = readWhile(_ => true) mkString "\n" - echo("\n// Exiting paste mode, now interpreting.\n") - intp interpret code - () - } - - private object paste extends Pasted { - val ContinueString = " | " - val PromptString = "scala> " - - def interpret(line: String): Unit = { - echo(line.trim) - intp interpret line - echo("") - } - - def transcript(start: String) = { - echo("\n// Detected repl transcript paste: ctrl-D to finish.\n") - apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim)) - } - } - import paste.{ ContinueString, PromptString } - - /** - * Interpret expressions starting with the first line. - * Read lines until a complete compilation unit is available - * or until a syntax error has been seen. If a full unit is - * read, go ahead and interpret it. Return the full string - * to be recorded for replay, if any. - */ - private def interpretStartingWith(code: String): Option[String] = { - // signal completion non-completion input has been received - in.completion.resetVerbosity() - - def reallyInterpret = { - val reallyResult = intp.interpret(code) - (reallyResult, reallyResult match { - case IR.Error => None - case IR.Success => Some(code) - case IR.Incomplete => - if (in.interactive && code.endsWith("\n\n")) { - echo("You typed two blank lines. Starting a new command.") - None - } - else in.readLine(ContinueString) match { - case null => - // we know compilation is going to fail since we're at EOF and the - // parser thinks the input is still incomplete, but since this is - // a file being read non-interactively we want to fail. So we send - // it straight to the compiler for the nice error message. - intp.compileString(code) - None - - case line => interpretStartingWith(code + "\n" + line) - } - }) - } - - /** Here we place ourselves between the user and the interpreter and examine - * the input they are ostensibly submitting. We intervene in several cases: - * - * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. - * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation - * on the previous result. - * 3) If the Completion object's execute returns Some(_), we inject that value - * and avoid the interpreter, as it's likely not valid scala code. - */ - if (code == "") None - else if (!paste.running && code.trim.startsWith(PromptString)) { - paste.transcript(code) - None - } - else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") { - interpretStartingWith(intp.mostRecentVar + code) - } - else if (code.trim startsWith "//") { - // line comment, do nothing - None - } - else - reallyInterpret._2 - } - - // runs :load `file` on any files passed via -i - private def loadFiles(settings: Settings) = settings match { - case settings: SparkRunnerSettings => - for (filename <- settings.loadfiles.value) { - val cmd = ":load " + filename - command(cmd) - addReplay(cmd) - echo("") - } - case _ => - } - - /** Tries to create a JLineReader, falling back to SimpleReader: - * unless settings or properties are such that it should start - * with SimpleReader. - */ - private def chooseReader(settings: Settings): InteractiveReader = { - if (settings.Xnojline.value || Properties.isEmacsShell) - SimpleReader() - else try new SparkJLineReader( - if (settings.noCompletion.value) NoCompletion - else new SparkJLineCompletion(intp) - ) - catch { - case ex @ (_: Exception | _: NoClassDefFoundError) => - echo("Failed to created SparkJLineReader: " + ex + "\nFalling back to SimpleReader.") - SimpleReader() - } - } - - private val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - private val m = u.runtimeMirror(Utils.getSparkClassLoader) - private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = - u.TypeTag[T]( - m, - new TypeCreator { - def apply[U <: ApiUniverse with Singleton](m: Mirror[U]): U # Type = - m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] - }) - - private def process(settings: Settings): Boolean = savingContextLoader { - this.settings = settings - createInterpreter() - - // sets in to some kind of reader depending on environmental cues - in = in0 match { - case Some(reader) => SimpleReader(reader, out, true) - case None => - // some post-initialization - chooseReader(settings) match { - case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x - case x => x - } - } - lazy val tagOfSparkIMain = tagOfStaticClass[org.apache.spark.repl.SparkIMain] - // Bind intp somewhere out of the regular namespace where - // we can get at it in generated code. - addThunk(intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfSparkIMain, classTag[SparkIMain]))) - addThunk({ - import scala.tools.nsc.io._ - import Properties.userHome - import scala.compat.Platform.EOL - val autorun = replProps.replAutorunCode.option flatMap (f => io.File(f).safeSlurp()) - if (autorun.isDefined) intp.quietRun(autorun.get) - }) - - addThunk(printWelcome()) - addThunk(initializeSpark()) - - // it is broken on startup; go ahead and exit - if (intp.reporter.hasErrors) - return false - - // This is about the illusion of snappiness. We call initialize() - // which spins off a separate thread, then print the prompt and try - // our best to look ready. The interlocking lazy vals tend to - // inter-deadlock, so we break the cycle with a single asynchronous - // message to an rpcEndpoint. - if (isAsync) { - intp initialize initializedCallback() - createAsyncListener() // listens for signal to run postInitialization - } - else { - intp.initializeSynchronous() - postInitialization() - } - // printWelcome() - - loadFiles(settings) - - try loop() - catch AbstractOrMissingHandler() - finally closeInterpreter() - - true - } - - // NOTE: Must be public for visibility - @DeveloperApi - def createSparkSession(): SparkSession = { - val execUri = System.getenv("SPARK_EXECUTOR_URI") - val jars = getAddedJars() - val conf = new SparkConf() - .setMaster(getMaster()) - .setJars(jars) - .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. - .set("spark.repl.class.outputDir", intp.outputDir.getAbsolutePath()) - if (execUri != null) { - conf.set("spark.executor.uri", execUri) - } - - val builder = SparkSession.builder.config(conf) - val sparkSession = if (SparkSession.hiveClassesArePresent) { - logInfo("Creating Spark session with Hive support") - builder.enableHiveSupport().getOrCreate() - } else { - logInfo("Creating Spark session") - builder.getOrCreate() - } - sparkContext = sparkSession.sparkContext - sparkSession - } - - private def getMaster(): String = { - val master = this.master match { - case Some(m) => m - case None => - val envMaster = sys.env.get("MASTER") - val propMaster = sys.props.get("spark.master") - propMaster.orElse(envMaster).getOrElse("local[*]") - } - master - } - - /** process command-line arguments and do as they request */ - def process(args: Array[String]): Boolean = { - val command = new SparkCommandLine(args.toList, msg => echo(msg)) - def neededHelp(): String = - (if (command.settings.help.value) command.usageMsg + "\n" else "") + - (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") - - // if they asked for no help and command is valid, we call the real main - neededHelp() match { - case "" => command.ok && process(command.settings) - case help => echoNoNL(help) ; true - } - } - - @deprecated("Use `process` instead", "2.9.0") - private def main(settings: Settings): Unit = process(settings) - - @DeveloperApi - def getAddedJars(): Array[String] = { - val conf = new SparkConf().setMaster(getMaster()) - val envJars = sys.env.get("ADD_JARS") - if (envJars.isDefined) { - logWarning("ADD_JARS environment variable is deprecated, use --jar spark submit argument instead") - } - val jars = { - val userJars = Utils.getUserJars(conf, isShell = true) - if (userJars.isEmpty) { - envJars.getOrElse("") - } else { - userJars.mkString(",") - } - } - Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) - } - -} - -object SparkILoop extends Logging { - implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp - private def echo(msg: String) = Console println msg - - // Designed primarily for use by test code: take a String with a - // bunch of code, and prints out a transcript of what it would look - // like if you'd just typed it into the repl. - private[repl] def runForTranscript(code: String, settings: Settings): String = { - import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - - stringFromStream { ostream => - Console.withOut(ostream) { - val output = new JPrintWriter(new OutputStreamWriter(ostream), true) { - override def write(str: String) = { - // completely skip continuation lines - if (str forall (ch => ch.isWhitespace || ch == '|')) () - // print a newline on empty scala prompts - else if ((str contains '\n') && (str.trim == "scala> ")) super.write("\n") - else super.write(str) - } - } - val input = new BufferedReader(new StringReader(code)) { - override def readLine(): String = { - val s = super.readLine() - // helping out by printing the line being interpreted. - if (s != null) - // scalastyle:off println - output.println(s) - // scalastyle:on println - s - } - } - val repl = new SparkILoop(input, output) - - if (settings.classpath.isDefault) - settings.classpath.value = sys.props("java.class.path") - - repl.getAddedJars().map(jar => new URI(jar).getPath).foreach(settings.classpath.append(_)) - - repl process settings - } - } - } - - /** Creates an interpreter loop with default settings and feeds - * the given code to it as input. - */ - private[repl] 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 JPrintWriter(new OutputStreamWriter(ostream), true) - val repl = new ILoop(input, output) - - if (sets.classpath.isDefault) - sets.classpath.value = sys.props("java.class.path") - - repl process sets - } - } - } - private[repl] def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala deleted file mode 100644 index 5f0d92bccd..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ /dev/null @@ -1,168 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Paul Phillips - */ - -package org.apache.spark.repl - -import scala.tools.nsc._ -import scala.tools.nsc.interpreter._ - -import scala.tools.nsc.util.stackTraceString - -import org.apache.spark.SPARK_VERSION - -/** - * Machinery for the asynchronous initialization of the repl. - */ -private[repl] trait SparkILoopInit { - self: SparkILoop => - - /** Print a welcome message */ - def printWelcome() { - echo("""Welcome to - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version %s - /_/ -""".format(SPARK_VERSION)) - import Properties._ - 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.") - } - - protected def asyncMessage(msg: String) { - if (isReplInfo || isReplPower) - echoAndRefresh(msg) - } - - private val initLock = new java.util.concurrent.locks.ReentrantLock() - private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized - private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized - private val initStart = System.nanoTime - - private def withLock[T](body: => T): T = { - initLock.lock() - try body - finally initLock.unlock() - } - // a condition used to ensure serial access to the compiler. - @volatile private var initIsComplete = false - @volatile private var initError: String = null - private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L) - - // the method to be called when the interpreter is initialized. - // Very important this method does nothing synchronous (i.e. do - // not try to use the interpreter) because until it returns, the - // repl's lazy val `global` is still locked. - protected def initializedCallback() = withLock(initCompilerCondition.signal()) - - // Spins off a thread which awaits a single message once the interpreter - // has been initialized. - protected def createAsyncListener() = { - io.spawn { - withLock(initCompilerCondition.await()) - asyncMessage("[info] compiler init time: " + elapsed() + " s.") - postInitialization() - } - } - - // called from main repl loop - protected def awaitInitialized(): Boolean = { - if (!initIsComplete) - withLock { while (!initIsComplete) initLoopCondition.await() } - if (initError != null) { - // scalastyle:off println - println(""" - |Failed to initialize the REPL due to an unexpected error. - |This is a bug, please, report it along with the error diagnostics printed below. - |%s.""".stripMargin.format(initError) - ) - // scalastyle:on println - false - } else true - } - // private def warningsThunks = List( - // () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _), - // ) - - protected def postInitThunks = List[Option[() => Unit]]( - Some(intp.setContextClassLoader _), - if (isReplPower) Some(() => enablePowerMode(true)) else None - ).flatten - // ++ ( - // warningsThunks - // ) - // called once after init condition is signalled - protected def postInitialization() { - try { - postInitThunks foreach (f => addThunk(f())) - runThunks() - } catch { - case ex: Throwable => - initError = stackTraceString(ex) - throw ex - } finally { - initIsComplete = true - - if (isAsync) { - asyncMessage("[info] total init time: " + elapsed() + " s.") - withLock(initLoopCondition.signal()) - } - } - } - - def initializeSpark() { - intp.beQuietDuring { - command(""" - @transient val spark = org.apache.spark.repl.Main.interp.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 - } - """) - command("import org.apache.spark.SparkContext._") - command("import spark.implicits._") - command("import spark.sql") - command("import org.apache.spark.sql.functions._") - } - } - - // code to be executed only after the interpreter is initialized - // and the lazy val `global` can be accessed without risk of deadlock. - private var pendingThunks: List[() => Unit] = Nil - protected def addThunk(body: => Unit) = synchronized { - pendingThunks :+= (() => body) - } - protected def runThunks(): Unit = synchronized { - if (pendingThunks.nonEmpty) - logDebug("Clearing " + pendingThunks.size + " thunks.") - - while (pendingThunks.nonEmpty) { - val thunk = pendingThunks.head - pendingThunks = pendingThunks.tail - thunk() - } - } -} diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala deleted file mode 100644 index 74a04d5a42..0000000000 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ /dev/null @@ -1,1808 +0,0 @@ -// scalastyle:off - -/* NSC -- new Scala compiler - * Copyright 2005-2013 LAMP/EPFL - * @author Martin Odersky - */ - -package org.apache.spark.repl - -import java.io.File - -import scala.tools.nsc._ -import scala.tools.nsc.backend.JavaPlatform -import scala.tools.nsc.interpreter._ - -import Predef.{ println => _, _ } -import scala.tools.nsc.util.{MergedClassPath, stringFromWriter, ScalaClassLoader, stackTraceString} -import scala.reflect.internal.util._ -import java.net.URL -import scala.sys.BooleanProp -import io.{AbstractFile, PlainFile, VirtualDirectory} - -import reporters._ -import symtab.Flags -import scala.reflect.internal.Names -import scala.tools.util.PathResolver -import ScalaClassLoader.URLClassLoader -import scala.tools.nsc.util.Exceptional.unwrap -import scala.collection.{ mutable, immutable } -import scala.util.control.Exception.{ ultimately } -import SparkIMain._ -import java.util.concurrent.Future -import typechecker.Analyzer -import scala.language.implicitConversions -import scala.reflect.runtime.{ universe => ru } -import scala.reflect.{ ClassTag, classTag } -import scala.tools.reflect.StdRuntimeTags._ -import scala.util.control.ControlThrowable - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils -import org.apache.spark.annotation.DeveloperApi - -// /** directory to save .class files to */ -// private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { -// private def pp(root: AbstractFile, indentLevel: Int) { -// val spaces = " " * indentLevel -// out.println(spaces + root.name) -// if (root.isDirectory) -// root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) -// } -// // print the contents hierarchically -// def show() = pp(this, 0) -// } - - /** An interpreter for Scala code. - * - * The main public entry points are compile(), interpret(), and bind(). - * The compile() method loads a complete Scala file. The interpret() method - * executes one line of Scala code at the request of the user. The bind() - * method binds an object to a variable that can then be used by later - * interpreted code. - * - * The overall approach is based on compiling the requested code and then - * using a Java classloader and Java reflection to run the code - * and access its results. - * - * In more detail, a single compiler instance is used - * to accumulate all successfully compiled or interpreted Scala code. To - * "interpret" a line of code, the compiler generates a fresh object that - * includes the line of code and which has public member(s) to export - * all variables defined by that code. To extract the result of an - * interpreted line to show the user, a second "result object" is created - * which imports the variables exported by the above object and then - * exports members called "$eval" and "$print". To accommodate user expressions - * that read from variables or methods defined in previous statements, "import" - * statements are used. - * - * This interpreter shares the strengths and weaknesses of using the - * full compiler-to-Java. The main strength is that interpreted code - * behaves exactly as does compiled code, including running at full speed. - * The main weakness is that redefining classes and methods is not handled - * properly, because rebinding at the Java level is technically difficult. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - */ - @DeveloperApi - class SparkIMain( - initialSettings: Settings, - val out: JPrintWriter, - propagateExceptions: Boolean = false) - extends SparkImports with Logging { imain => - - private val conf = new SparkConf() - - private val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - /** Local directory to save .class files too */ - private[repl] val outputDir = { - val rootDir = conf.getOption("spark.repl.classdir").getOrElse(Utils.getLocalDir(conf)) - Utils.createTempDir(root = rootDir, namePrefix = "repl") - } - if (SPARK_DEBUG_REPL) { - echo("Output directory: " + outputDir) - } - - /** - * Returns the path to the output directory containing all generated - * class files that will be served by the REPL class server. - */ - @DeveloperApi - lazy val getClassOutputDirectory = outputDir - - private val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles - /** Jetty server that will serve our classes to worker nodes */ - private var currentSettings: Settings = initialSettings - private var printResults = true // whether to print result lines - private var totalSilence = false // whether to print anything - private var _initializeComplete = false // compiler is initialized - private var _isInitialized: Future[Boolean] = null // set up initialization future - private var bindExceptions = true // whether to bind the lastException variable - private var _executionWrapper = "" // code to be wrapped around all lines - - /** We're going to go to some trouble to initialize the compiler asynchronously. - * It's critical that nothing call into it until it's been initialized or we will - * run into unrecoverable issues, but the perceived repl startup time goes - * through the roof if we wait for it. So we initialize it with a future and - * use a lazy val to ensure that any attempt to use the compiler object waits - * on the future. - */ - private var _classLoader: AbstractFileClassLoader = null // active classloader - private val _compiler: Global = newCompiler(settings, reporter) // our private compiler - - private trait ExposeAddUrl extends URLClassLoader { def addNewUrl(url: URL) = this.addURL(url) } - private var _runtimeClassLoader: URLClassLoader with ExposeAddUrl = null // wrapper exposing addURL - - private val nextReqId = { - var counter = 0 - () => { counter += 1 ; counter } - } - - private def compilerClasspath: Seq[URL] = ( - if (isInitializeComplete) global.classPath.asURLs - else new PathResolver(settings).result.asURLs // the compiler's classpath - ) - // NOTE: Exposed to repl package since accessed indirectly from SparkIMain - private[repl] def settings = currentSettings - private def mostRecentLine = prevRequestList match { - case Nil => "" - case req :: _ => req.originalLine - } - // Run the code body with the given boolean settings flipped to true. - private def withoutWarnings[T](body: => T): T = beQuietDuring { - val saved = settings.nowarn.value - if (!saved) - settings.nowarn.value = true - - try body - finally if (!saved) settings.nowarn.value = false - } - - /** construct an interpreter that reports to Console */ - def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) - def this() = this(new Settings()) - - private lazy val repllog: Logger = new Logger { - val out: JPrintWriter = imain.out - val isInfo: Boolean = BooleanProp keyExists "scala.repl.info" - val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug" - val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace" - } - private[repl] lazy val formatting: Formatting = new Formatting { - val prompt = Properties.shellPromptString - } - - // NOTE: Exposed to repl package since used by SparkExprTyper and SparkILoop - private[repl] lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) - - /** - * Determines if errors were reported (typically during compilation). - * - * @note This is not for runtime errors - * - * @return True if had errors, otherwise false - */ - @DeveloperApi - def isReportingErrors = reporter.hasErrors - - import formatting._ - import reporter.{ printMessage, withoutTruncating } - - // This exists mostly because using the reporter too early leads to deadlock. - private def echo(msg: String) { Console println msg } - private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }")) - private def _initialize() = { - try { - // todo. if this crashes, REPL will hang - new _compiler.Run() compileSources _initSources - _initializeComplete = true - true - } - catch AbstractOrMissingHandler() - } - private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" - - // argument is a thunk to execute after init is done - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def initialize(postInitSignal: => Unit) { - synchronized { - if (_isInitialized == null) { - _isInitialized = io.spawn { - try _initialize() - finally postInitSignal - } - } - } - } - - /** - * Initializes the underlying compiler/interpreter in a blocking fashion. - * - * @note Must be executed before using SparkIMain! - */ - @DeveloperApi - def initializeSynchronous(): Unit = { - if (!isInitializeComplete) { - _initialize() - assert(global != null, global) - } - } - private def isInitializeComplete = _initializeComplete - - /** the public, go through the future compiler */ - - /** - * The underlying compiler used to generate ASTs and execute code. - */ - @DeveloperApi - lazy val global: Global = { - if (isInitializeComplete) _compiler - else { - // If init hasn't been called yet you're on your own. - if (_isInitialized == null) { - logWarning("Warning: compiler accessed before init set up. Assuming no postInit code.") - initialize(()) - } - // // blocks until it is ; false means catastrophic failure - if (_isInitialized.get()) _compiler - else null - } - } - @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - private lazy val compiler: global.type = global - - import global._ - import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember} - import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass} - - private implicit class ReplTypeOps(tp: Type) { - def orElse(other: => Type): Type = if (tp ne NoType) tp else other - def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) - } - - // TODO: If we try to make naming a lazy val, we run into big time - // scalac unhappiness with what look like cycles. It has not been easy to - // reduce, but name resolution clearly takes different paths. - // NOTE: Exposed to repl package since used by SparkExprTyper - private[repl] object naming extends { - val global: imain.global.type = imain.global - } with Naming { - // make sure we don't overwrite their unwisely named res3 etc. - def freshUserTermName(): TermName = { - val name = newTermName(freshUserVarName()) - if (definedNameMap contains name) freshUserTermName() - else name - } - def isUserTermName(name: Name) = isUserVarName("" + name) - def isInternalTermName(name: Name) = isInternalVarName("" + name) - } - import naming._ - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] object deconstruct extends { - val global: imain.global.type = imain.global - } with StructuredTypeStrings - - // NOTE: Exposed to repl package since used by SparkImports - private[repl] lazy val memberHandlers = new { - val intp: imain.type = imain - } with SparkMemberHandlers - import memberHandlers._ - - /** - * Suppresses overwriting print results during the operation. - * - * @param body The block to execute - * @tparam T The return type of the block - * - * @return The result from executing the block - */ - @DeveloperApi - def beQuietDuring[T](body: => T): T = { - val saved = printResults - printResults = false - try body - finally printResults = saved - } - - /** - * Completely masks all output during the operation (minus JVM standard - * out and error). - * - * @param operation The block to execute - * @tparam T The return type of the block - * - * @return The result from executing the block - */ - @DeveloperApi - def beSilentDuring[T](operation: => T): T = { - val saved = totalSilence - totalSilence = true - try operation - finally totalSilence = saved - } - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { - case t: ControlThrowable => throw t - case t: Throwable => - logDebug(label + ": " + unwrap(t)) - logDebug(stackTraceString(unwrap(t))) - alt - } - /** takes AnyRef because it may be binding a Throwable or an Exceptional */ - - private def withLastExceptionLock[T](body: => T, alt: => T): T = { - assert(bindExceptions, "withLastExceptionLock called incorrectly.") - bindExceptions = false - - try beQuietDuring(body) - catch logAndDiscard("withLastExceptionLock", alt) - finally bindExceptions = true - } - - /** - * Contains the code (in string form) representing a wrapper around all - * code executed by this instance. - * - * @return The wrapper code as a string - */ - @DeveloperApi - def executionWrapper = _executionWrapper - - /** - * Sets the code to use as a wrapper around all code executed by this - * instance. - * - * @param code The wrapper code as a string - */ - @DeveloperApi - def setExecutionWrapper(code: String) = _executionWrapper = code - - /** - * Clears the code used as a wrapper around all code executed by - * this instance. - */ - @DeveloperApi - def clearExecutionWrapper() = _executionWrapper = "" - - /** interpreter settings */ - private lazy val isettings = new SparkISettings(this) - - /** - * Instantiates a new compiler used by SparkIMain. Overridable to provide - * own instance of a compiler. - * - * @param settings The settings to provide the compiler - * @param reporter The reporter to use for compiler output - * - * @return The compiler as a Global - */ - @DeveloperApi - protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { - settings.outputDirs setSingleOutput virtualDirectory - settings.exposeEmptyPackage.value = true - new Global(settings, reporter) with ReplGlobal { - override def toString: String = "" - } - } - - /** - * Adds any specified jars to the compile and runtime classpaths. - * - * @note Currently only supports jars, not directories - * @param urls The list of items to add to the compile and runtime classpaths - */ - @DeveloperApi - def addUrlsToClassPath(urls: URL*): Unit = { - new Run // Needed to force initialization of "something" to correctly load Scala classes from jars - urls.foreach(_runtimeClassLoader.addNewUrl) // Add jars/classes to runtime for execution - updateCompilerClassPath(urls: _*) // Add jars/classes to compile time for compiling - } - - private def updateCompilerClassPath(urls: URL*): Unit = { - require(!global.forMSIL) // Only support JavaPlatform - - val platform = global.platform.asInstanceOf[JavaPlatform] - - val newClassPath = mergeUrlsIntoClassPath(platform, urls: _*) - - // NOTE: Must use reflection until this is exposed/fixed upstream in Scala - val fieldSetter = platform.getClass.getMethods - .find(_.getName.endsWith("currentClassPath_$eq")).get - fieldSetter.invoke(platform, Some(newClassPath)) - - // Reload all jars specified into our compiler - global.invalidateClassPathEntries(urls.map(_.getPath): _*) - } - - private def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = { - // Collect our new jars/directories and add them to the existing set of classpaths - val allClassPaths = ( - platform.classPath.asInstanceOf[MergedClassPath[AbstractFile]].entries ++ - urls.map(url => { - platform.classPath.context.newClassPath( - if (url.getProtocol == "file") { - val f = new File(url.getPath) - if (f.isDirectory) - io.AbstractFile.getDirectory(f) - else - io.AbstractFile.getFile(f) - } else { - io.AbstractFile.getURL(url) - } - ) - }) - ).distinct - - // Combine all of our classpaths (old and new) into one merged classpath - new MergedClassPath(allClassPaths, platform.classPath.context) - } - - /** - * Represents the parent classloader used by this instance. Can be - * overridden to provide alternative classloader. - * - * @return The classloader used as the parent loader of this instance - */ - @DeveloperApi - protected def parentClassLoader: ClassLoader = - SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) - - /* A single class loader is used for all commands interpreted by this Interpreter. - It would also be possible to create a new class loader for each command - to interpret. The advantages of the current approach are: - - - Expressions are only evaluated one time. This is especially - significant for I/O, e.g. "val x = Console.readLine" - - The main disadvantage is: - - - Objects, classes, and methods cannot be rebound. Instead, definitions - shadow the old ones, and old code objects refer to the old - definitions. - */ - private def resetClassLoader() = { - logDebug("Setting new classloader: was " + _classLoader) - _classLoader = null - ensureClassLoader() - } - private final def ensureClassLoader() { - if (_classLoader == null) - _classLoader = makeClassLoader() - } - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def classLoader: AbstractFileClassLoader = { - ensureClassLoader() - _classLoader - } - private class TranslatingClassLoader(parent: ClassLoader) extends AbstractFileClassLoader(virtualDirectory, parent) { - /** Overridden here to try translating a simple name to the generated - * class name if the original attempt fails. This method is used by - * getResourceAsStream as well as findClass. - */ - override protected def findAbstractFile(name: String): AbstractFile = { - super.findAbstractFile(name) match { - // deadlocks on startup if we try to translate names too early - case null if isInitializeComplete => - generatedName(name) map (x => super.findAbstractFile(x)) orNull - case file => - file - } - } - } - private def makeClassLoader(): AbstractFileClassLoader = - new TranslatingClassLoader(parentClassLoader match { - case null => ScalaClassLoader fromURLs compilerClasspath - case p => - _runtimeClassLoader = new URLClassLoader(compilerClasspath, p) with ExposeAddUrl - _runtimeClassLoader - }) - - private def getInterpreterClassLoader() = classLoader - - // Set the current Java "context" class loader to this interpreter's class loader - // NOTE: Exposed to repl package since used by SparkILoopInit - private[repl] def setContextClassLoader() = classLoader.setAsContext() - - /** - * Returns the real name of a class based on its repl-defined name. - * - * ==Example== - * Given a simple repl-defined name, returns the real name of - * the class representing it, e.g. for "Bippy" it may return - * {{{ - * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy - * }}} - * - * @param simpleName The repl-defined name whose real name to retrieve - * - * @return Some real name if the simple name exists, else None - */ - @DeveloperApi - def generatedName(simpleName: String): Option[String] = { - if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING) - else optFlatName(simpleName) - } - - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def flatName(id: String) = optFlatName(id) getOrElse id - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) - - /** - * Retrieves all simple names contained in the current instance. - * - * @return A list of sorted names - */ - @DeveloperApi - def allDefinedNames = definedNameMap.keys.toList.sorted - - private def pathToType(id: String): String = pathToName(newTypeName(id)) - // NOTE: Exposed to repl package since used by SparkILoop - private[repl] def pathToTerm(id: String): String = pathToName(newTermName(id)) - - /** - * Retrieves the full code path to access the specified simple name - * content. - * - * @param name The simple name of the target whose path to determine - * - * @return The full path used to access the specified target (name) - */ - @DeveloperApi - def pathToName(name: Name): String = { - if (definedNameMap contains name) - definedNameMap(name) fullPath name - else name.toString - } - - /** Most recent tree handled which wasn't wholly synthetic. */ - private def mostRecentlyHandledTree: Option[Tree] = { - prevRequests.reverse foreach { req => - req.handlers.reverse foreach { - case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member) - case _ => () - } - } - None - } - - /** Stubs for work in progress. */ - private def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { - for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { - logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) - } - } - - private def handleTermRedefinition(name: TermName, old: Request, req: Request) = { - for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { - // Printing the types here has a tendency to cause assertion errors, like - // assertion failed: fatal: has owner value x, but a class owner is required - // so DBG is by-name now to keep it in the family. (It also traps the assertion error, - // but we don't want to unnecessarily risk hosing the compiler's internal state.) - logDebug("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) - } - } - - private def recordRequest(req: Request) { - if (req == null || referencedNameMap == null) - return - - prevRequests += req - req.referencedNames foreach (x => referencedNameMap(x) = req) - - // warning about serially defining companions. It'd be easy - // enough to just redefine them together but that may not always - // be what people want so I'm waiting until I can do it better. - for { - name <- req.definedNames filterNot (x => req.definedNames contains x.companionName) - oldReq <- definedNameMap get name.companionName - newSym <- req.definedSymbols get name - oldSym <- oldReq.definedSymbols get name.companionName - if Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule } - } { - afterTyper(replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.")) - replwarn("Companions must be defined together; you may wish to use :paste mode for this.") - } - - // Updating the defined name map - req.definedNames foreach { name => - if (definedNameMap contains name) { - if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) - else handleTermRedefinition(name.toTermName, definedNameMap(name), req) - } - definedNameMap(name) = req - } - } - - private def replwarn(msg: => String) { - if (!settings.nowarnings.value) - printMessage(msg) - } - - private def isParseable(line: String): Boolean = { - beSilentDuring { - try parse(line) match { - case Some(xs) => xs.nonEmpty // parses as-is - case None => true // incomplete - } - catch { case x: Exception => // crashed the compiler - replwarn("Exception in isParseable(\"" + line + "\"): " + x) - false - } - } - } - - private def compileSourcesKeepingRun(sources: SourceFile*) = { - val run = new Run() - reporter.reset() - run compileSources sources.toList - (!reporter.hasErrors, run) - } - - /** - * Compiles specified source files. - * - * @param sources The sequence of source files to compile - * - * @return True if successful, otherwise false - */ - @DeveloperApi - def compileSources(sources: SourceFile*): Boolean = - compileSourcesKeepingRun(sources: _*)._1 - - /** - * Compiles a string of code. - * - * @param code The string of code to compile - * - * @return True if successful, otherwise false - */ - @DeveloperApi - def compileString(code: String): Boolean = - compileSources(new BatchSourceFile("