From fc8a2b6ee6f1041345f9ce9701fac496c3c3b1e6 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 12 May 2017 09:55:04 +0100 Subject: [PATCH] [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls ## What changes were proposed in this pull request? Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up. ## How was this patch tested? Existing tests. Author: Sean Owen Closes #17949 from srowen/SPARK-20554. --- .../spark/storage/DiskBlockManagerSuite.scala | 2 -- .../util/random/XORShiftRandomSuite.scala | 16 +++------- .../examples/ml/DecisionTreeExample.scala | 1 - .../apache/spark/examples/ml/GBTExample.scala | 1 - .../examples/ml/LinearRegressionExample.scala | 2 -- .../ml/LogisticRegressionExample.scala | 1 - .../examples/ml/RandomForestExample.scala | 1 - .../sql/execution/QueryExecutionSuite.scala | 31 +++++++++---------- .../sql/hive/client/HiveClientImpl.scala | 1 - .../hive/client/IsolatedClientLoader.scala | 1 - .../receiver/BlockGeneratorSuite.scala | 14 ++++----- 11 files changed, 25 insertions(+), 46 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index bbfd6df3b6..7859b0bba2 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.storage import java.io.{File, FileWriter} -import scala.language.reflectiveCalls - import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 83eba3690e..df3483830c 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.util.random -import scala.language.reflectiveCalls - import org.apache.commons.math3.stat.inference.ChiSquareTest import org.scalatest.Matchers @@ -27,26 +25,22 @@ import org.apache.spark.util.Utils.times class XORShiftRandomSuite extends SparkFunSuite with Matchers { - private def fixture = new { - val seed = 1L - val xorRand = new XORShiftRandom(seed) - val hundMil = 1e8.toInt - } - /* * This test is based on a chi-squared test for randomness. */ test ("XORShift generates valid random numbers") { - val f = fixture + val xorRand = new XORShiftRandom(1L) val numBins = 10 // create 10 bins val numRows = 5 // create 5 rows val bins = Array.ofDim[Long](numRows, numBins) // populate bins based on modulus of the random number for each row - for (r <- 0 to numRows-1) { - times(f.hundMil) {bins(r)(math.abs(f.xorRand.nextInt) % numBins) += 1} + for (r <- 0 until numRows) { + times(100000000) { + bins(r)(math.abs(xorRand.nextInt) % numBins) += 1 + } } /* diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index f736ceed44..b03701e491 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -21,7 +21,6 @@ package org.apache.spark.examples.ml import java.util.Locale import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala index ed598d0d7d..3bd8ff54c2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -21,7 +21,6 @@ package org.apache.spark.examples.ml import java.util.Locale import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala index 31ba180335..6903a1c298 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala @@ -18,8 +18,6 @@ // scalastyle:off println package org.apache.spark.examples.ml -import scala.language.reflectiveCalls - import scopt.OptionParser import org.apache.spark.examples.mllib.AbstractParams diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala index c67b53899c..bd6cc8cff2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.ml import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala index 8fd46c37e2..a735c218c0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -21,7 +21,6 @@ package org.apache.spark.examples.ml import java.util.Locale import scala.collection.mutable -import scala.language.reflectiveCalls import scopt.OptionParser diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 05637821f7..afccbe5cc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -16,39 +16,36 @@ */ package org.apache.spark.sql.execution -import java.util.Locale - -import scala.language.reflectiveCalls - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.test.SharedSQLContext class QueryExecutionSuite extends SharedSQLContext { test("toString() exception/error handling") { - val badRule = new SparkStrategy { - var mode: String = "" - override def apply(plan: LogicalPlan): Seq[SparkPlan] = - mode.toLowerCase(Locale.ROOT) match { - case "exception" => throw new AnalysisException(mode) - case "error" => throw new Error(mode) - case _ => Nil - } - } - spark.experimental.extraStrategies = badRule :: Nil + spark.experimental.extraStrategies = Seq( + new SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = Nil + }) def qe: QueryExecution = new QueryExecution(spark, OneRowRelation) // Nothing! - badRule.mode = "" assert(qe.toString.contains("OneRowRelation")) // Throw an AnalysisException - this should be captured. - badRule.mode = "exception" + spark.experimental.extraStrategies = Seq( + new SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = + throw new AnalysisException("exception") + }) assert(qe.toString.contains("org.apache.spark.sql.AnalysisException")) // Throw an Error - this should not be captured. - badRule.mode = "error" + spark.experimental.extraStrategies = Seq( + new SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = + throw new Error("error") + }) val error = intercept[Error](qe.toString) assert(error.getMessage.contains("error")) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 387ec4f967..74e15a5777 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -22,7 +22,6 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.language.reflectiveCalls import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index e95f9ea480..b8aa067cdb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -22,7 +22,6 @@ import java.lang.reflect.InvocationTargetException import java.net.{URL, URLClassLoader} import java.util -import scala.language.reflectiveCalls import scala.util.Try import org.apache.commons.io.{FileUtils, IOUtils} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala index b70383ecde..4f41b9d0a0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala @@ -21,7 +21,6 @@ import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.language.reflectiveCalls import org.scalatest.BeforeAndAfter import org.scalatest.Matchers._ @@ -202,21 +201,17 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter { test("block push errors are reported") { val listener = new TestBlockGeneratorListener { - @volatile var errorReported = false override def onPushBlock( blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { throw new SparkException("test") } - override def onError(message: String, throwable: Throwable): Unit = { - errorReported = true - } } blockGenerator = new BlockGenerator(listener, 0, conf) blockGenerator.start() - assert(listener.errorReported === false) + assert(listener.onErrorCalled === false) blockGenerator.addData(1) eventually(timeout(1 second), interval(10 milliseconds)) { - assert(listener.errorReported === true) + assert(listener.onErrorCalled === true) } blockGenerator.stop() } @@ -243,12 +238,15 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter { @volatile var onGenerateBlockCalled = false @volatile var onAddDataCalled = false @volatile var onPushBlockCalled = false + @volatile var onErrorCalled = false override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { pushedData.addAll(arrayBuffer.asJava) onPushBlockCalled = true } - override def onError(message: String, throwable: Throwable): Unit = {} + override def onError(message: String, throwable: Throwable): Unit = { + onErrorCalled = true + } override def onGenerateBlock(blockId: StreamBlockId): Unit = { onGenerateBlockCalled = true }