From 0a8cc309211c62f8824d76618705c817edcf2424 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 1 Sep 2013 00:32:28 -0700 Subject: Move some classes to more appropriate packages: * RDD, *RDDFunctions -> org.apache.spark.rdd * Utils, ClosureCleaner, SizeEstimator -> org.apache.spark.util * JavaSerializer, KryoSerializer -> org.apache.spark.serializer --- .../main/scala/org/apache/spark/bagel/Bagel.scala | 2 +- .../main/scala/org/apache/spark/Accumulators.scala | 1 + .../main/scala/org/apache/spark/CacheManager.scala | 1 + .../scala/org/apache/spark/ClosureCleaner.scala | 231 ----- .../main/scala/org/apache/spark/Dependency.scala | 2 + .../org/apache/spark/DoubleRDDFunctions.scala | 78 -- .../scala/org/apache/spark/HttpFileServer.scala | 1 + .../main/scala/org/apache/spark/HttpServer.scala | 1 + .../scala/org/apache/spark/JavaSerializer.scala | 83 -- .../scala/org/apache/spark/KryoSerializer.scala | 156 ---- .../scala/org/apache/spark/MapOutputTracker.scala | 2 +- .../scala/org/apache/spark/PairRDDFunctions.scala | 703 --------------- .../main/scala/org/apache/spark/Partitioner.scala | 3 + core/src/main/scala/org/apache/spark/RDD.scala | 957 --------------------- .../scala/org/apache/spark/RDDCheckpointData.scala | 130 --- .../apache/spark/SequenceFileRDDFunctions.scala | 107 --- .../scala/org/apache/spark/SizeEstimator.scala | 283 ------ .../main/scala/org/apache/spark/SparkContext.scala | 8 +- .../src/main/scala/org/apache/spark/SparkEnv.scala | 6 +- core/src/main/scala/org/apache/spark/Utils.scala | 780 ----------------- .../org/apache/spark/api/java/JavaDoubleRDD.scala | 2 +- .../org/apache/spark/api/java/JavaPairRDD.scala | 2 +- .../scala/org/apache/spark/api/java/JavaRDD.scala | 1 + .../org/apache/spark/api/java/JavaRDDLike.scala | 6 +- .../apache/spark/api/java/JavaSparkContext.scala | 6 +- .../spark/api/python/PythonPartitioner.scala | 2 +- .../org/apache/spark/api/python/PythonRDD.scala | 2 + .../spark/broadcast/BitTorrentBroadcast.scala | 1 + .../org/apache/spark/broadcast/HttpBroadcast.scala | 4 +- .../org/apache/spark/broadcast/MultiTracker.scala | 1 + .../org/apache/spark/broadcast/TreeBroadcast.scala | 1 + .../org/apache/spark/deploy/DeployMessage.scala | 2 +- .../apache/spark/deploy/LocalSparkCluster.scala | 4 +- .../apache/spark/deploy/client/TestClient.scala | 4 +- .../org/apache/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/master/MasterArguments.scala | 3 +- .../apache/spark/deploy/master/WorkerInfo.scala | 2 +- .../spark/deploy/master/ui/ApplicationPage.scala | 2 +- .../apache/spark/deploy/master/ui/IndexPage.scala | 2 +- .../spark/deploy/master/ui/MasterWebUI.scala | 3 +- .../spark/deploy/worker/ExecutorRunner.scala | 3 +- .../org/apache/spark/deploy/worker/Worker.scala | 4 +- .../spark/deploy/worker/WorkerArguments.scala | 4 +- .../apache/spark/deploy/worker/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 3 +- .../scala/org/apache/spark/executor/Executor.scala | 1 + .../spark/executor/MesosExecutorBackend.scala | 3 +- .../spark/executor/StandaloneExecutorBackend.scala | 4 +- .../apache/spark/network/ConnectionManager.scala | 1 + .../apache/spark/network/ConnectionManagerId.scala | 2 +- core/src/main/scala/org/apache/spark/package.scala | 21 +- .../spark/partial/ApproximateActionListener.scala | 1 + .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 2 +- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../org/apache/spark/rdd/DoubleRDDFunctions.scala | 79 ++ .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- .../scala/org/apache/spark/rdd/FilteredRDD.scala | 2 +- .../scala/org/apache/spark/rdd/FlatMappedRDD.scala | 2 +- .../org/apache/spark/rdd/FlatMappedValuesRDD.scala | 2 +- .../scala/org/apache/spark/rdd/GlommedRDD.scala | 2 +- .../scala/org/apache/spark/rdd/HadoopRDD.scala | 8 +- .../main/scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 2 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 2 +- .../scala/org/apache/spark/rdd/MappedRDD.scala | 2 +- .../org/apache/spark/rdd/MappedValuesRDD.scala | 2 +- .../scala/org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 7 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 702 +++++++++++++++ .../apache/spark/rdd/ParallelCollectionRDD.scala | 2 + .../org/apache/spark/rdd/PartitionPruningRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 942 ++++++++++++++++++++ .../org/apache/spark/rdd/RDDCheckpointData.scala | 131 +++ .../scala/org/apache/spark/rdd/SampledRDD.scala | 2 +- .../spark/rdd/SequenceFileRDDFunctions.scala | 89 ++ .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/SubtractedRDD.scala | 1 - .../main/scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../org/apache/spark/rdd/ZippedPartitionsRDD.scala | 2 +- .../scala/org/apache/spark/rdd/ZippedRDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 1 + .../apache/spark/scheduler/DAGSchedulerEvent.scala | 1 + .../org/apache/spark/scheduler/JobLogger.scala | 1 + .../org/apache/spark/scheduler/ResultTask.scala | 7 +- .../apache/spark/scheduler/ShuffleMapTask.scala | 2 + .../org/apache/spark/scheduler/SparkListener.scala | 4 +- .../scala/org/apache/spark/scheduler/Stage.scala | 3 +- .../org/apache/spark/scheduler/TaskResult.scala | 3 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/cluster/SchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 3 +- .../cluster/StandaloneClusterMessage.scala | 3 +- .../cluster/StandaloneSchedulerBackend.scala | 3 +- .../apache/spark/scheduler/cluster/TaskInfo.scala | 2 +- .../spark/scheduler/local/LocalScheduler.scala | 1 + .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scheduler/mesos/MesosSchedulerBackend.scala | 3 +- .../apache/spark/serializer/JavaSerializer.scala | 82 ++ .../apache/spark/serializer/KryoSerializer.scala | 159 ++++ .../spark/storage/BlockFetcherIterator.scala | 2 +- .../org/apache/spark/storage/BlockManager.scala | 4 +- .../org/apache/spark/storage/BlockManagerId.scala | 2 +- .../spark/storage/BlockManagerMasterActor.scala | 3 +- .../apache/spark/storage/BlockManagerWorker.scala | 3 +- .../scala/org/apache/spark/storage/DiskStore.scala | 2 +- .../org/apache/spark/storage/MemoryStore.scala | 2 +- .../org/apache/spark/storage/StorageUtils.scala | 3 +- .../org/apache/spark/storage/ThreadingTest.scala | 2 +- .../main/scala/org/apache/spark/ui/SparkUI.scala | 3 +- .../org/apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../apache/spark/ui/jobs/JobProgressListener.scala | 2 +- .../org/apache/spark/ui/jobs/JobProgressUI.scala | 3 +- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 4 +- .../org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../org/apache/spark/ui/storage/IndexPage.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../org/apache/spark/util/ClosureCleaner.scala | 232 +++++ .../scala/org/apache/spark/util/MemoryParam.scala | 2 - .../org/apache/spark/util/SizeEstimator.scala | 284 ++++++ .../main/scala/org/apache/spark/util/Utils.scala | 781 +++++++++++++++++ .../scala/org/apache/spark/CheckpointSuite.scala | 1 + .../org/apache/spark/ClosureCleanerSuite.scala | 146 ---- .../test/scala/org/apache/spark/DriverSuite.scala | 1 + .../test/scala/org/apache/spark/FailureSuite.scala | 1 + .../org/apache/spark/KryoSerializerSuite.scala | 208 ----- .../org/apache/spark/PairRDDFunctionsSuite.scala | 299 ------- .../apache/spark/PartitionPruningRDDSuite.scala | 2 +- .../scala/org/apache/spark/PartitioningSuite.scala | 9 +- .../src/test/scala/org/apache/spark/RDDSuite.scala | 389 --------- .../test/scala/org/apache/spark/ShuffleSuite.scala | 3 +- .../org/apache/spark/SizeEstimatorSuite.scala | 164 ---- .../test/scala/org/apache/spark/SortingSuite.scala | 123 --- .../test/scala/org/apache/spark/UtilsSuite.scala | 139 --- .../apache/spark/rdd/PairRDDFunctionsSuite.scala | 300 +++++++ .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 391 +++++++++ .../scala/org/apache/spark/rdd/SortingSuite.scala | 125 +++ .../apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../apache/spark/scheduler/JobLoggerSuite.scala | 6 +- .../apache/spark/scheduler/TaskContextSuite.scala | 2 +- .../cluster/ClusterTaskSetManagerSuite.scala | 2 +- .../spark/serializer/KryoSerializerSuite.scala | 208 +++++ .../apache/spark/storage/BlockManagerSuite.scala | 8 +- .../apache/spark/util/ClosureCleanerSuite.scala | 147 ++++ .../org/apache/spark/util/SizeEstimatorSuite.scala | 164 ++++ .../scala/org/apache/spark/util/UtilsSuite.scala | 139 +++ docs/configuration.md | 10 +- docs/quick-start.md | 2 +- docs/scala-programming-guide.md | 2 +- docs/tuning.md | 10 +- .../spark/examples/bagel/PageRankUtils.scala | 1 + .../spark/examples/bagel/WikipediaPageRank.scala | 2 +- .../bagel/WikipediaPageRankStandalone.scala | 17 +- .../spark/streaming/examples/QueueStream.scala | 2 +- .../spark/streaming/examples/RawNetworkGrep.scala | 2 +- .../mllib/classification/ClassificationModel.scala | 2 +- .../mllib/classification/LogisticRegression.scala | 3 +- .../apache/spark/mllib/classification/SVM.scala | 3 +- .../org/apache/spark/mllib/clustering/KMeans.scala | 3 +- .../spark/mllib/clustering/KMeansModel.scala | 2 +- .../spark/mllib/optimization/GradientDescent.scala | 8 +- .../spark/mllib/optimization/Optimizer.scala | 2 +- .../apache/spark/mllib/recommendation/ALS.scala | 7 +- .../recommendation/MatrixFactorizationModel.scala | 2 +- .../regression/GeneralizedLinearAlgorithm.scala | 3 +- .../org/apache/spark/mllib/regression/Lasso.scala | 3 +- .../spark/mllib/regression/LinearRegression.scala | 3 +- .../spark/mllib/regression/RegressionModel.scala | 2 +- .../spark/mllib/regression/RidgeRegression.scala | 3 +- .../apache/spark/mllib/util/DataValidators.scala | 3 +- .../spark/mllib/util/KMeansDataGenerator.scala | 3 +- .../spark/mllib/util/LinearDataGenerator.scala | 3 +- .../util/LogisticRegressionDataGenerator.scala | 3 +- .../apache/spark/mllib/util/MFDataGenerator.scala | 3 +- .../org/apache/spark/mllib/util/MLUtils.scala | 3 +- .../apache/spark/mllib/util/SVMDataGenerator.scala | 5 +- python/pyspark/context.py | 4 +- .../scala/org/apache/spark/repl/SparkIMain.scala | 2 +- .../scala/org/apache/spark/streaming/DStream.scala | 3 +- .../org/apache/spark/streaming/Duration.scala | 2 +- .../spark/streaming/PairDStreamFunctions.scala | 9 +- .../apache/spark/streaming/StreamingContext.scala | 1 + .../spark/streaming/api/java/JavaDStream.scala | 2 +- .../spark/streaming/api/java/JavaDStreamLike.scala | 2 +- .../spark/streaming/api/java/JavaPairDStream.scala | 7 +- .../streaming/api/java/JavaStreamingContext.scala | 25 +- .../spark/streaming/dstream/CoGroupedDStream.scala | 3 +- .../streaming/dstream/ConstantInputDStream.scala | 2 +- .../spark/streaming/dstream/FileInputDStream.scala | 2 +- .../spark/streaming/dstream/FilteredDStream.scala | 2 +- .../streaming/dstream/FlatMapValuedDStream.scala | 2 +- .../streaming/dstream/FlatMappedDStream.scala | 2 +- .../streaming/dstream/FlumeInputDStream.scala | 15 +- .../spark/streaming/dstream/ForEachDStream.scala | 2 +- .../spark/streaming/dstream/GlommedDStream.scala | 2 +- .../streaming/dstream/MapPartitionedDStream.scala | 2 +- .../spark/streaming/dstream/MapValuedDStream.scala | 2 +- .../spark/streaming/dstream/MappedDStream.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 15 +- .../streaming/dstream/QueueInputDStream.scala | 2 +- .../streaming/dstream/ReducedWindowedDStream.scala | 2 +- .../spark/streaming/dstream/ShuffledDStream.scala | 3 +- .../spark/streaming/dstream/StateDStream.scala | 2 +- .../streaming/dstream/TransformedDStream.scala | 2 +- .../spark/streaming/dstream/UnionDStream.scala | 2 +- .../spark/streaming/dstream/WindowedDStream.scala | 2 +- .../spark/streaming/util/MasterFailureTest.scala | 3 +- .../spark/streaming/util/RawTextSender.scala | 3 +- .../org/apache/spark/streaming/TestSuiteBase.scala | 5 +- .../spark/tools/JavaAPICompletenessChecker.scala | 38 +- 210 files changed, 5290 insertions(+), 5246 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/ClosureCleaner.scala delete mode 100644 core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala delete mode 100644 core/src/main/scala/org/apache/spark/JavaSerializer.scala delete mode 100644 core/src/main/scala/org/apache/spark/KryoSerializer.scala delete mode 100644 core/src/main/scala/org/apache/spark/PairRDDFunctions.scala delete mode 100644 core/src/main/scala/org/apache/spark/RDD.scala delete mode 100644 core/src/main/scala/org/apache/spark/RDDCheckpointData.scala delete mode 100644 core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala delete mode 100644 core/src/main/scala/org/apache/spark/SizeEstimator.scala delete mode 100644 core/src/main/scala/org/apache/spark/Utils.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/RDD.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala create mode 100644 core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala create mode 100644 core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala create mode 100644 core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala create mode 100644 core/src/main/scala/org/apache/spark/util/SizeEstimator.scala create mode 100644 core/src/main/scala/org/apache/spark/util/Utils.scala delete mode 100644 core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/RDDSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/SortingSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/UtilsSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/UtilsSuite.scala diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index fec8737fcd..44e26bbb9e 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -19,7 +19,7 @@ package org.apache.spark.bagel import org.apache.spark._ import org.apache.spark.SparkContext._ - +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel object Bagel extends Logging { diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 5177ee58fa..6e922a612a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -21,6 +21,7 @@ import java.io._ import scala.collection.mutable.Map import scala.collection.generic.Growable +import org.apache.spark.serializer.JavaSerializer /** * A datatype that can be accumulated, i.e. has an commutative and associative "add" operation, diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 42e465b9d8..e299a106ee 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -19,6 +19,7 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.storage.{BlockManager, StorageLevel} +import org.apache.spark.rdd.RDD /** Spark class responsible for passing RDDs split contents to the BlockManager and making diff --git a/core/src/main/scala/org/apache/spark/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/ClosureCleaner.scala deleted file mode 100644 index 71d9e62d4f..0000000000 --- a/core/src/main/scala/org/apache/spark/ClosureCleaner.scala +++ /dev/null @@ -1,231 +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 - -import java.lang.reflect.Field - -import scala.collection.mutable.Map -import scala.collection.mutable.Set - -import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} -import org.objectweb.asm.Opcodes._ -import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} - -private[spark] object ClosureCleaner extends Logging { - // Get an ASM class reader for a given class from the JAR that loaded it - private def getClassReader(cls: Class[_]): ClassReader = { - // Copy data over, before delegating to ClassReader - else we can run out of open file handles. - val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" - val resourceStream = cls.getResourceAsStream(className) - // todo: Fixme - continuing with earlier behavior ... - if (resourceStream == null) return new ClassReader(resourceStream) - - val baos = new ByteArrayOutputStream(128) - Utils.copyStream(resourceStream, baos, true) - new ClassReader(new ByteArrayInputStream(baos.toByteArray)) - } - - // Check whether a class represents a Scala closure - private def isClosure(cls: Class[_]): Boolean = { - cls.getName.contains("$anonfun$") - } - - // Get a list of the classes of the outer objects of a given closure object, obj; - // the outer objects are defined as any closures that obj is nested within, plus - // possibly the class that the outermost closure is in, if any. We stop searching - // for outer objects beyond that because cloning the user's object is probably - // not a good idea (whereas we can clone closure objects just fine since we - // understand how all their fields are used). - private def getOuterClasses(obj: AnyRef): List[Class[_]] = { - for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { - f.setAccessible(true) - if (isClosure(f.getType)) { - return f.getType :: getOuterClasses(f.get(obj)) - } else { - return f.getType :: Nil // Stop at the first $outer that is not a closure - } - } - return Nil - } - - // Get a list of the outer objects for a given closure object. - private def getOuterObjects(obj: AnyRef): List[AnyRef] = { - for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { - f.setAccessible(true) - if (isClosure(f.getType)) { - return f.get(obj) :: getOuterObjects(f.get(obj)) - } else { - return f.get(obj) :: Nil // Stop at the first $outer that is not a closure - } - } - return Nil - } - - private def getInnerClasses(obj: AnyRef): List[Class[_]] = { - val seen = Set[Class[_]](obj.getClass) - var stack = List[Class[_]](obj.getClass) - while (!stack.isEmpty) { - val cr = getClassReader(stack.head) - stack = stack.tail - val set = Set[Class[_]]() - cr.accept(new InnerClosureFinder(set), 0) - for (cls <- set -- seen) { - seen += cls - stack = cls :: stack - } - } - return (seen - obj.getClass).toList - } - - private def createNullValue(cls: Class[_]): AnyRef = { - if (cls.isPrimitive) { - new java.lang.Byte(0: Byte) // Should be convertible to any primitive type - } else { - null - } - } - - def clean(func: AnyRef) { - // TODO: cache outerClasses / innerClasses / accessedFields - val outerClasses = getOuterClasses(func) - val innerClasses = getInnerClasses(func) - val outerObjects = getOuterObjects(func) - - val accessedFields = Map[Class[_], Set[String]]() - for (cls <- outerClasses) - accessedFields(cls) = Set[String]() - for (cls <- func.getClass :: innerClasses) - getClassReader(cls).accept(new FieldAccessFinder(accessedFields), 0) - //logInfo("accessedFields: " + accessedFields) - - val inInterpreter = { - try { - val interpClass = Class.forName("spark.repl.Main") - interpClass.getMethod("interp").invoke(null) != null - } catch { - case _: ClassNotFoundException => true - } - } - - var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse - var outer: AnyRef = null - if (outerPairs.size > 0 && !isClosure(outerPairs.head._1)) { - // The closure is ultimately nested inside a class; keep the object of that - // class without cloning it since we don't want to clone the user's objects. - outer = outerPairs.head._2 - outerPairs = outerPairs.tail - } - // Clone the closure objects themselves, nulling out any fields that are not - // used in the closure we're working on or any of its inner closures. - for ((cls, obj) <- outerPairs) { - outer = instantiateClass(cls, outer, inInterpreter) - for (fieldName <- accessedFields(cls)) { - val field = cls.getDeclaredField(fieldName) - field.setAccessible(true) - val value = field.get(obj) - //logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); - field.set(outer, value) - } - } - - if (outer != null) { - //logInfo("2: Setting $outer on " + func.getClass + " to " + outer); - val field = func.getClass.getDeclaredField("$outer") - field.setAccessible(true) - field.set(func, outer) - } - } - - private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { - //logInfo("Creating a " + cls + " with outer = " + outer) - if (!inInterpreter) { - // This is a bona fide closure class, whose constructor has no effects - // other than to set its fields, so use its constructor - val cons = cls.getConstructors()(0) - val params = cons.getParameterTypes.map(createNullValue).toArray - if (outer != null) - params(0) = outer // First param is always outer object - return cons.newInstance(params: _*).asInstanceOf[AnyRef] - } else { - // Use reflection to instantiate object without calling constructor - val rf = sun.reflect.ReflectionFactory.getReflectionFactory() - val parentCtor = classOf[java.lang.Object].getDeclaredConstructor() - val newCtor = rf.newConstructorForSerialization(cls, parentCtor) - val obj = newCtor.newInstance().asInstanceOf[AnyRef] - if (outer != null) { - //logInfo("3: Setting $outer on " + cls + " to " + outer); - val field = cls.getDeclaredField("$outer") - field.setAccessible(true) - field.set(obj, outer) - } - return obj - } - } -} - -private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { - override def visitMethod(access: Int, name: String, desc: String, - sig: String, exceptions: Array[String]): MethodVisitor = { - return new MethodVisitor(ASM4) { - override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { - if (op == GETFIELD) { - for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { - output(cl) += name - } - } - } - - override def visitMethodInsn(op: Int, owner: String, name: String, - desc: String) { - // Check for calls a getter method for a variable in an interpreter wrapper object. - // This means that the corresponding field will be accessed, so we should save it. - if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer")) { - for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { - output(cl) += name - } - } - } - } - } -} - -private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { - var myName: String = null - - override def visit(version: Int, access: Int, name: String, sig: String, - superName: String, interfaces: Array[String]) { - myName = name - } - - override def visitMethod(access: Int, name: String, desc: String, - sig: String, exceptions: Array[String]): MethodVisitor = { - return new MethodVisitor(ASM4) { - override def visitMethodInsn(op: Int, owner: String, name: String, - desc: String) { - val argTypes = Type.getArgumentTypes(desc) - if (op == INVOKESPECIAL && name == "" && argTypes.length > 0 - && argTypes(0).toString.startsWith("L") // is it an object? - && argTypes(0).getInternalName == myName) - output += Class.forName( - owner.replace('/', '.'), - false, - Thread.currentThread.getContextClassLoader) - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index cc3c2474a6..cc30105940 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.apache.spark.rdd.RDD + /** * Base class for dependencies. */ diff --git a/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala deleted file mode 100644 index dd344491b8..0000000000 --- a/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala +++ /dev/null @@ -1,78 +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 - -import org.apache.spark.partial.BoundedDouble -import org.apache.spark.partial.MeanEvaluator -import org.apache.spark.partial.PartialResult -import org.apache.spark.partial.SumEvaluator -import org.apache.spark.util.StatCounter - -/** - * Extra functions available on RDDs of Doubles through an implicit conversion. - * Import `spark.SparkContext._` at the top of your program to use these functions. - */ -class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { - /** Add up the elements in this RDD. */ - def sum(): Double = { - self.reduce(_ + _) - } - - /** - * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count - * of the RDD's elements in one operation. - */ - def stats(): StatCounter = { - self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b)) - } - - /** Compute the mean of this RDD's elements. */ - def mean(): Double = stats().mean - - /** Compute the variance of this RDD's elements. */ - def variance(): Double = stats().variance - - /** Compute the standard deviation of this RDD's elements. */ - def stdev(): Double = stats().stdev - - /** - * Compute the sample standard deviation of this RDD's elements (which corrects for bias in - * estimating the standard deviation by dividing by N-1 instead of N). - */ - def sampleStdev(): Double = stats().sampleStdev - - /** - * Compute the sample variance of this RDD's elements (which corrects for bias in - * estimating the variance by dividing by N-1 instead of N). - */ - def sampleVariance(): Double = stats().sampleVariance - - /** (Experimental) Approximate operation to return the mean within a timeout. */ - def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { - val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new MeanEvaluator(self.partitions.size, confidence) - self.context.runApproximateJob(self, processPartition, evaluator, timeout) - } - - /** (Experimental) Approximate operation to return the sum within a timeout. */ - def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { - val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) - val evaluator = new SumEvaluator(self.partitions.size, confidence) - self.context.runApproximateJob(self, processPartition, evaluator, timeout) - } -} diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 9b3a896648..ad1ee20045 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{File} import com.google.common.io.Files +import org.apache.spark.util.Utils private[spark] class HttpFileServer extends Logging { diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index db36c7c9dd..cdfc9dd54e 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -26,6 +26,7 @@ import org.eclipse.jetty.server.handler.DefaultHandler import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.server.handler.ResourceHandler import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.apache.spark.util.Utils /** * Exception type thrown by HttpServer when it is in the wrong state for an operation. diff --git a/core/src/main/scala/org/apache/spark/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/JavaSerializer.scala deleted file mode 100644 index f43396cb6b..0000000000 --- a/core/src/main/scala/org/apache/spark/JavaSerializer.scala +++ /dev/null @@ -1,83 +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 - -import java.io._ -import java.nio.ByteBuffer - -import serializer.{Serializer, SerializerInstance, DeserializationStream, SerializationStream} -import org.apache.spark.util.ByteBufferInputStream - -private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { - val objOut = new ObjectOutputStream(out) - def writeObject[T](t: T): SerializationStream = { objOut.writeObject(t); this } - def flush() { objOut.flush() } - def close() { objOut.close() } -} - -private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) -extends DeserializationStream { - val objIn = new ObjectInputStream(in) { - override def resolveClass(desc: ObjectStreamClass) = - Class.forName(desc.getName, false, loader) - } - - def readObject[T](): T = objIn.readObject().asInstanceOf[T] - def close() { objIn.close() } -} - -private[spark] class JavaSerializerInstance extends SerializerInstance { - def serialize[T](t: T): ByteBuffer = { - val bos = new ByteArrayOutputStream() - val out = serializeStream(bos) - out.writeObject(t) - out.close() - ByteBuffer.wrap(bos.toByteArray) - } - - def deserialize[T](bytes: ByteBuffer): T = { - val bis = new ByteBufferInputStream(bytes) - val in = deserializeStream(bis) - in.readObject().asInstanceOf[T] - } - - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { - val bis = new ByteBufferInputStream(bytes) - val in = deserializeStream(bis, loader) - in.readObject().asInstanceOf[T] - } - - def serializeStream(s: OutputStream): SerializationStream = { - new JavaSerializationStream(s) - } - - def deserializeStream(s: InputStream): DeserializationStream = { - new JavaDeserializationStream(s, Thread.currentThread.getContextClassLoader) - } - - def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { - new JavaDeserializationStream(s, loader) - } -} - -/** - * A Spark serializer that uses Java's built-in serialization. - */ -class JavaSerializer extends Serializer { - def newInstance(): SerializerInstance = new JavaSerializerInstance -} diff --git a/core/src/main/scala/org/apache/spark/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/KryoSerializer.scala deleted file mode 100644 index db86e6db43..0000000000 --- a/core/src/main/scala/org/apache/spark/KryoSerializer.scala +++ /dev/null @@ -1,156 +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 - -import java.io._ -import java.nio.ByteBuffer -import com.esotericsoftware.kryo.{Kryo, KryoException} -import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} -import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} -import com.twitter.chill.ScalaKryoInstantiator -import serializer.{SerializerInstance, DeserializationStream, SerializationStream} -import org.apache.spark.broadcast._ -import org.apache.spark.storage._ - -private[spark] -class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { - val output = new KryoOutput(outStream) - - def writeObject[T](t: T): SerializationStream = { - kryo.writeClassAndObject(output, t) - this - } - - def flush() { output.flush() } - def close() { output.close() } -} - -private[spark] -class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - val input = new KryoInput(inStream) - - def readObject[T](): T = { - try { - kryo.readClassAndObject(input).asInstanceOf[T] - } catch { - // DeserializationStream uses the EOF exception to indicate stopping condition. - case _: KryoException => throw new EOFException - } - } - - def close() { - // Kryo's Input automatically closes the input stream it is using. - input.close() - } -} - -private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - val kryo = ks.newKryo() - val output = ks.newKryoOutput() - val input = ks.newKryoInput() - - def serialize[T](t: T): ByteBuffer = { - output.clear() - kryo.writeClassAndObject(output, t) - ByteBuffer.wrap(output.toBytes) - } - - def deserialize[T](bytes: ByteBuffer): T = { - input.setBuffer(bytes.array) - kryo.readClassAndObject(input).asInstanceOf[T] - } - - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { - val oldClassLoader = kryo.getClassLoader - kryo.setClassLoader(loader) - input.setBuffer(bytes.array) - val obj = kryo.readClassAndObject(input).asInstanceOf[T] - kryo.setClassLoader(oldClassLoader) - obj - } - - def serializeStream(s: OutputStream): SerializationStream = { - new KryoSerializationStream(kryo, s) - } - - def deserializeStream(s: InputStream): DeserializationStream = { - new KryoDeserializationStream(kryo, s) - } -} - -/** - * Interface implemented by clients to register their classes with Kryo when using Kryo - * serialization. - */ -trait KryoRegistrator { - def registerClasses(kryo: Kryo) -} - -/** - * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]]. - */ -class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { - private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 - - def newKryoOutput() = new KryoOutput(bufferSize) - - def newKryoInput() = new KryoInput(bufferSize) - - def newKryo(): Kryo = { - val instantiator = new ScalaKryoInstantiator - val kryo = instantiator.newKryo() - val classLoader = Thread.currentThread.getContextClassLoader - - // Register some commonly used classes - val toRegister: Seq[AnyRef] = Seq( - ByteBuffer.allocate(1), - StorageLevel.MEMORY_ONLY, - PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY), - GotBlock("1", ByteBuffer.allocate(1)), - GetBlock("1") - ) - - for (obj <- toRegister) kryo.register(obj.getClass) - - // Allow sending SerializableWritable - kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) - kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) - - // Allow the user to register their own classes by setting spark.kryo.registrator - try { - Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => - logDebug("Running user registrator: " + regCls) - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] - reg.registerClasses(kryo) - } - } catch { - case _: Exception => println("Failed to register spark.kryo.registrator") - } - - kryo.setClassLoader(classLoader) - - // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops - kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) - - kryo - } - - def newInstance(): SerializerInstance = { - new KryoSerializerInstance(this) - } -} diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 0f422d910a..ae7cf2a893 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -32,7 +32,7 @@ import akka.util.Duration import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashMap} private[spark] sealed trait MapOutputTrackerMessage diff --git a/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala deleted file mode 100644 index d046e7c1a4..0000000000 --- a/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala +++ /dev/null @@ -1,703 +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 - -import java.nio.ByteBuffer -import java.util.{Date, HashMap => JHashMap} -import java.text.SimpleDateFormat - -import scala.collection.{mutable, Map} -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.io.SequenceFile.CompressionType -import org.apache.hadoop.mapred.FileOutputCommitter -import org.apache.hadoop.mapred.FileOutputFormat -import org.apache.hadoop.mapred.SparkHadoopWriter -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.OutputFormat - -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, - RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil} -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.partial.BoundedDouble -import org.apache.spark.partial.PartialResult -import org.apache.spark.rdd._ -import org.apache.spark.SparkContext._ -import org.apache.spark.Partitioner._ - -/** - * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. - * Import `spark.SparkContext._` at the top of your program to use these functions. - */ -class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) - extends Logging - with SparkHadoopMapReduceUtil - with Serializable { - - /** - * Generic function to combine the elements for each key using a custom set of aggregation - * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C - * Note that V and C can be different -- for example, one might group an RDD of type - * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: - * - * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) - * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) - * - `mergeCombiners`, to combine two C's into a single one. - * - * In addition, users can control the partitioning of the output RDD, and whether to perform - * map-side aggregation (if a mapper can produce multiple items with the same key). - */ - def combineByKey[C](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, - serializerClass: String = null): RDD[(K, C)] = { - if (getKeyClass().isArray) { - if (mapSideCombine) { - throw new SparkException("Cannot use map-side combining with array keys.") - } - if (partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") - } - } - val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) - if (self.partitioner == Some(partitioner)) { - self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) - } else if (mapSideCombine) { - val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) - val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) - .setSerializer(serializerClass) - partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = true) - } else { - // Don't apply map-side combiner. - // A sanity check to make sure mergeCombiners is not defined. - assert(mergeCombiners == null) - val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass) - values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) - } - } - - /** - * Simplified version of combineByKey that hash-partitions the output RDD. - */ - def combineByKey[C](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - numPartitions: Int): RDD[(K, C)] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { - // Serialize the zero value to a byte array so that we can get a new clone of it on each key - val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) - val zeroArray = new Array[Byte](zeroBuffer.limit) - zeroBuffer.get(zeroArray) - - // When deserializing, use a lazy val to create just one instance of the serializer per task - lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() - def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) - - combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { - foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { - foldByKey(zeroValue, defaultPartitioner(self))(func) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. - */ - def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = { - combineByKey[V]((v: V) => v, func, func, partitioner) - } - - /** - * Merge the values for each key using an associative reduce function, but return the results - * immediately to the master as a Map. This will also perform the merging locally on each mapper - * before sending results to a reducer, similarly to a "combiner" in MapReduce. - */ - def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { - - if (getKeyClass().isArray) { - throw new SparkException("reduceByKeyLocally() does not support array keys") - } - - def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { - val map = new JHashMap[K, V] - iter.foreach { case (k, v) => - val old = map.get(k) - map.put(k, if (old == null) v else func(old, v)) - } - Iterator(map) - } - - def mergeMaps(m1: JHashMap[K, V], m2: JHashMap[K, V]): JHashMap[K, V] = { - m2.foreach { case (k, v) => - val old = m1.get(k) - m1.put(k, if (old == null) v else func(old, v)) - } - m1 - } - - self.mapPartitions(reducePartition).reduce(mergeMaps) - } - - /** Alias for reduceByKeyLocally */ - def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func) - - /** Count the number of elements for each key, and return the result to the master as a Map. */ - def countByKey(): Map[K, Long] = self.map(_._1).countByValue() - - /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does - * not finish within a timeout. - */ - def countByKeyApprox(timeout: Long, confidence: Double = 0.95) - : PartialResult[Map[K, BoundedDouble]] = { - self.map(_._1).countByValueApprox(timeout, confidence) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. - */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { - reduceByKey(new HashPartitioner(numPartitions), func) - } - - /** - * Group the values for each key in the RDD into a single sequence. Allows controlling the - * partitioning of the resulting key-value pair RDD by passing a Partitioner. - */ - def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { - // groupByKey shouldn't use map side combine because map side combine does not - // reduce the amount of data shuffled and requires all map side data be inserted - // into a hash table, leading to more objects in the old gen. - def createCombiner(v: V) = ArrayBuffer(v) - def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - val bufs = combineByKey[ArrayBuffer[V]]( - createCombiner _, mergeValue _, null, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Seq[V])]] - } - - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numPartitions` partitions. - */ - def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { - groupByKey(new HashPartitioner(numPartitions)) - } - - /** - * Return a copy of the RDD partitioned using the specified partitioner. - */ - def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { - if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") - } - new ShuffledRDD[K, V, (K, V)](self, partitioner) - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. - */ - def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to - * partition the output RDD. - */ - def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) - } - } - } - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to - * partition the output RDD. - */ - def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) - : RDD[(K, (Option[V], W))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) - } - } - } - - /** - * Simplified version of combineByKey that hash-partitions the resulting RDD using the - * existing partitioner/parallelism level. - */ - def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) - : RDD[(K, C)] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ - * parallelism level. - */ - def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { - reduceByKey(defaultPartitioner(self), func) - } - - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the existing partitioner/parallelism level. - */ - def groupByKey(): RDD[(K, Seq[V])] = { - groupByKey(defaultPartitioner(self)) - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { - join(other, defaultPartitioner(self, other)) - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { - join(other, new HashPartitioner(numPartitions)) - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the existing partitioner/parallelism level. - */ - def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { - leftOuterJoin(other, defaultPartitioner(self, other)) - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numPartitions` partitions. - */ - def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { - leftOuterJoin(other, new HashPartitioner(numPartitions)) - } - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the existing partitioner/parallelism level. - */ - def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { - rightOuterJoin(other, defaultPartitioner(self, other)) - } - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD into the given number of partitions. - */ - def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { - rightOuterJoin(other, new HashPartitioner(numPartitions)) - } - - /** - * Return the key-value pairs in this RDD to the master as a Map. - */ - def collectAsMap(): Map[K, V] = { - val data = self.toArray() - val map = new mutable.HashMap[K, V] - map.sizeHint(data.length) - data.foreach { case (k, v) => map.put(k, v) } - map - } - - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - def mapValues[U](f: V => U): RDD[(K, U)] = { - val cleanF = self.context.clean(f) - new MappedValuesRDD(self, cleanF) - } - - /** - * Pass each value in the key-value pair RDD through a flatMap function without changing the - * keys; this also retains the original RDD's partitioning. - */ - def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { - val cleanF = self.context.clean(f) - new FlatMappedValuesRDD(self, cleanF) - } - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") - } - val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) - val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) - prfs.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) - } - } - - /** - * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this`, `other1` and `other2`. - */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") - } - val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) - val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) - prfs.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) - } - } - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { - cogroup(other, defaultPartitioner(self, other)) - } - - /** - * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this`, `other1` and `other2`. - */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - cogroup(other1, other2, defaultPartitioner(self, other1, other2)) - } - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { - cogroup(other, new HashPartitioner(numPartitions)) - } - - /** - * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - * tuple with the list of values for that key in `this`, `other1` and `other2`. - */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - cogroup(other1, other2, new HashPartitioner(numPartitions)) - } - - /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { - cogroup(other, defaultPartitioner(self, other)) - } - - /** Alias for cogroup. */ - def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - cogroup(other1, other2, defaultPartitioner(self, other1, other2)) - } - - /** - * Return an RDD with the pairs from `this` whose keys are not in `other`. - * - * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. - */ - def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] = - subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) - - /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = - subtractByKey(other, new HashPartitioner(numPartitions)) - - /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ - def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = - new SubtractedRDD[K, V, W](self, other, p) - - /** - * Return the list of values in the RDD for key `key`. This operation is done efficiently if the - * RDD has a known partitioner by only searching the partition that the key maps to. - */ - def lookup(key: K): Seq[V] = { - self.partitioner match { - case Some(p) => - val index = p.getPartition(key) - def process(it: Iterator[(K, V)]): Seq[V] = { - val buf = new ArrayBuffer[V] - for ((k, v) <- it if k == key) { - buf += v - } - buf - } - val res = self.context.runJob(self, process _, Array(index), false) - res(0) - case None => - self.filter(_._1 == key).map(_._2).collect() - } - } - - /** - * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class - * supporting the key and value types K and V in this RDD. - */ - def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) - } - - /** - * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class - * supporting the key and value types K and V in this RDD. Compress the result with the - * supplied codec. - */ - def saveAsHadoopFile[F <: OutputFormat[K, V]]( - path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) - } - - /** - * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` - * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. - */ - def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) - } - - /** - * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` - * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. - */ - def saveAsNewAPIHadoopFile( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[_ <: NewOutputFormat[_, _]], - conf: Configuration = self.context.hadoopConfiguration) { - val job = new NewAPIHadoopJob(conf) - job.setOutputKeyClass(keyClass) - job.setOutputValueClass(valueClass) - val wrappedConf = new SerializableWritable(job.getConfiguration) - NewFileOutputFormat.setOutputPath(job, new Path(path)) - val formatter = new SimpleDateFormat("yyyyMMddHHmm") - val jobtrackerID = formatter.format(new Date()) - val stageId = self.id - def writeShard(context: TaskContext, iter: Iterator[(K,V)]): Int = { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt - /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.splitId, attemptNumber) - val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) - val format = outputFormatClass.newInstance - val committer = format.getOutputCommitter(hadoopContext) - committer.setupTask(hadoopContext) - val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] - while (iter.hasNext) { - val (k, v) = iter.next() - writer.write(k, v) - } - writer.close(hadoopContext) - committer.commitTask(hadoopContext) - return 1 - } - val jobFormat = outputFormatClass.newInstance - /* apparently we need a TaskAttemptID to construct an OutputCommitter; - * however we're only going to use this local OutputCommitter for - * setupJob/commitJob, so we just use a dummy "map" task. - */ - val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0) - val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) - val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) - jobCommitter.setupJob(jobTaskContext) - val count = self.context.runJob(self, writeShard _).sum - jobCommitter.commitJob(jobTaskContext) - jobCommitter.cleanupJob(jobTaskContext) - } - - /** - * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class - * supporting the key and value types K and V in this RDD. Compress with the supplied codec. - */ - def saveAsHadoopFile( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[_ <: OutputFormat[_, _]], - codec: Class[_ <: CompressionCodec]) { - saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, - new JobConf(self.context.hadoopConfiguration), Some(codec)) - } - - /** - * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class - * supporting the key and value types K and V in this RDD. - */ - def saveAsHadoopFile( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[_ <: OutputFormat[_, _]], - conf: JobConf = new JobConf(self.context.hadoopConfiguration), - codec: Option[Class[_ <: CompressionCodec]] = None) { - conf.setOutputKeyClass(keyClass) - conf.setOutputValueClass(valueClass) - // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug - conf.set("mapred.output.format.class", outputFormatClass.getName) - for (c <- codec) { - conf.setCompressMapOutput(true) - conf.set("mapred.output.compress", "true") - conf.setMapOutputCompressorClass(c) - conf.set("mapred.output.compression.codec", c.getCanonicalName) - conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) - } - conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath(conf, SparkHadoopWriter.createPathFromString(path, conf)) - saveAsHadoopDataset(conf) - } - - /** - * Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for - * that storage system. The JobConf should set an OutputFormat and any output paths required - * (e.g. a table name to write to) in the same way as it would be configured for a Hadoop - * MapReduce job. - */ - def saveAsHadoopDataset(conf: JobConf) { - val outputFormatClass = conf.getOutputFormat - val keyClass = conf.getOutputKeyClass - val valueClass = conf.getOutputValueClass - if (outputFormatClass == null) { - throw new SparkException("Output format class not set") - } - if (keyClass == null) { - throw new SparkException("Output key class not set") - } - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - - logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")") - - val writer = new SparkHadoopWriter(conf) - writer.preSetup() - - def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt - - writer.setup(context.stageId, context.splitId, attemptNumber) - writer.open() - - var count = 0 - while(iter.hasNext) { - val record = iter.next() - count += 1 - writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) - } - - writer.close() - writer.commit() - } - - self.context.runJob(self, writeToFile _) - writer.commitJob() - writer.cleanup() - } - - /** - * Return an RDD with the keys of each tuple. - */ - def keys: RDD[K] = self.map(_._1) - - /** - * Return an RDD with the values of each tuple. - */ - def values: RDD[V] = self.map(_._2) - - private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure - - private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure -} - - -private[spark] object Manifests { - val seqSeqManifest = classManifest[Seq[Seq[_]]] -} diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 4dce2607b0..0e2c987a59 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -17,6 +17,9 @@ package org.apache.spark +import org.apache.spark.util.Utils +import org.apache.spark.rdd.RDD + /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. diff --git a/core/src/main/scala/org/apache/spark/RDD.scala b/core/src/main/scala/org/apache/spark/RDD.scala deleted file mode 100644 index 0d1f07f76c..0000000000 --- a/core/src/main/scala/org/apache/spark/RDD.scala +++ /dev/null @@ -1,957 +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 - -import java.util.Random - -import scala.collection.Map -import scala.collection.JavaConversions.mapAsScalaMap -import scala.collection.mutable.ArrayBuffer - -import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.TextOutputFormat - -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} - -import org.apache.spark.Partitioner._ -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.partial.BoundedDouble -import org.apache.spark.partial.CountEvaluator -import org.apache.spark.partial.GroupedCountEvaluator -import org.apache.spark.partial.PartialResult -import org.apache.spark.rdd.CoalescedRDD -import org.apache.spark.rdd.CartesianRDD -import org.apache.spark.rdd.FilteredRDD -import org.apache.spark.rdd.FlatMappedRDD -import org.apache.spark.rdd.GlommedRDD -import org.apache.spark.rdd.MappedRDD -import org.apache.spark.rdd.MapPartitionsRDD -import org.apache.spark.rdd.MapPartitionsWithIndexRDD -import org.apache.spark.rdd.PipedRDD -import org.apache.spark.rdd.SampledRDD -import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.rdd.ZippedRDD -import org.apache.spark.rdd.ZippedPartitionsRDD2 -import org.apache.spark.rdd.ZippedPartitionsRDD3 -import org.apache.spark.rdd.ZippedPartitionsRDD4 -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.BoundedPriorityQueue - -import SparkContext._ - -/** - * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, - * partitioned collection of elements that can be operated on in parallel. This class contains the - * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition, - * [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such - * as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations available only on - * RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations available on RDDs - * that can be saved as SequenceFiles. These operations are automatically available on any RDD of - * the right type (e.g. RDD[(Int, Int)] through implicit conversions when you - * `import org.apache.spark.SparkContext._`. - * - * Internally, each RDD is characterized by five main properties: - * - * - A list of partitions - * - A function for computing each split - * - A list of dependencies on other RDDs - * - Optionally, a Partitioner for key-value RDDs (e.g. to say that the RDD is hash-partitioned) - * - Optionally, a list of preferred locations to compute each split on (e.g. block locations for - * an HDFS file) - * - * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD - * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for - * reading data from a new storage system) by overriding these functions. Please refer to the - * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details - * on RDD internals. - */ -abstract class RDD[T: ClassManifest]( - @transient private var sc: SparkContext, - @transient private var deps: Seq[Dependency[_]] - ) extends Serializable with Logging { - - /** Construct an RDD with just a one-to-one dependency on one parent */ - def this(@transient oneParent: RDD[_]) = - this(oneParent.context , List(new OneToOneDependency(oneParent))) - - // ======================================================================= - // Methods that should be implemented by subclasses of RDD - // ======================================================================= - - /** Implemented by subclasses to compute a given partition. */ - def compute(split: Partition, context: TaskContext): Iterator[T] - - /** - * Implemented by subclasses to return the set of partitions in this RDD. This method will only - * be called once, so it is safe to implement a time-consuming computation in it. - */ - protected def getPartitions: Array[Partition] - - /** - * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only - * be called once, so it is safe to implement a time-consuming computation in it. - */ - protected def getDependencies: Seq[Dependency[_]] = deps - - /** Optionally overridden by subclasses to specify placement preferences. */ - protected def getPreferredLocations(split: Partition): Seq[String] = Nil - - /** Optionally overridden by subclasses to specify how they are partitioned. */ - val partitioner: Option[Partitioner] = None - - // ======================================================================= - // Methods and fields available on all RDDs - // ======================================================================= - - /** The SparkContext that created this RDD. */ - def sparkContext: SparkContext = sc - - /** A unique ID for this RDD (within its SparkContext). */ - val id: Int = sc.newRddId() - - /** A friendly name for this RDD */ - var name: String = null - - /** Assign a name to this RDD */ - def setName(_name: String) = { - name = _name - this - } - - /** User-defined generator of this RDD*/ - var generator = Utils.getCallSiteInfo.firstUserClass - - /** Reset generator*/ - def setGenerator(_generator: String) = { - generator = _generator - } - - /** - * Set this RDD's storage level to persist its values across operations after the first time - * it is computed. This can only be used to assign a new storage level if the RDD does not - * have a storage level set yet.. - */ - def persist(newLevel: StorageLevel): RDD[T] = { - // TODO: Handle changes of StorageLevel - if (storageLevel != StorageLevel.NONE && newLevel != storageLevel) { - throw new UnsupportedOperationException( - "Cannot change storage level of an RDD after it was already assigned a level") - } - storageLevel = newLevel - // Register the RDD with the SparkContext - sc.persistentRdds(id) = this - this - } - - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) - - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def cache(): RDD[T] = persist() - - /** - * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. - * - * @param blocking Whether to block until all blocks are deleted. - * @return This RDD. - */ - def unpersist(blocking: Boolean = true): RDD[T] = { - logInfo("Removing RDD " + id + " from persistence list") - sc.env.blockManager.master.removeRdd(id, blocking) - sc.persistentRdds.remove(id) - storageLevel = StorageLevel.NONE - this - } - - /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ - def getStorageLevel = storageLevel - - // Our dependencies and partitions will be gotten by calling subclass's methods below, and will - // be overwritten when we're checkpointed - private var dependencies_ : Seq[Dependency[_]] = null - @transient private var partitions_ : Array[Partition] = null - - /** An Option holding our checkpoint RDD, if we are checkpointed */ - private def checkpointRDD: Option[RDD[T]] = checkpointData.flatMap(_.checkpointRDD) - - /** - * Get the list of dependencies of this RDD, taking into account whether the - * RDD is checkpointed or not. - */ - final def dependencies: Seq[Dependency[_]] = { - checkpointRDD.map(r => List(new OneToOneDependency(r))).getOrElse { - if (dependencies_ == null) { - dependencies_ = getDependencies - } - dependencies_ - } - } - - /** - * Get the array of partitions of this RDD, taking into account whether the - * RDD is checkpointed or not. - */ - final def partitions: Array[Partition] = { - checkpointRDD.map(_.partitions).getOrElse { - if (partitions_ == null) { - partitions_ = getPartitions - } - partitions_ - } - } - - /** - * Get the preferred locations of a partition (as hostnames), taking into account whether the - * RDD is checkpointed. - */ - final def preferredLocations(split: Partition): Seq[String] = { - checkpointRDD.map(_.getPreferredLocations(split)).getOrElse { - getPreferredLocations(split) - } - } - - /** - * Internal method to this RDD; will read from cache if applicable, or otherwise compute it. - * This should ''not'' be called by users directly, but is available for implementors of custom - * subclasses of RDD. - */ - final def iterator(split: Partition, context: TaskContext): Iterator[T] = { - if (storageLevel != StorageLevel.NONE) { - SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) - } else { - computeOrReadCheckpoint(split, context) - } - } - - /** - * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. - */ - private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = { - if (isCheckpointed) { - firstParent[T].iterator(split, context) - } else { - compute(split, context) - } - } - - // Transformations (return a new RDD) - - /** - * Return a new RDD by applying a function to all elements of this RDD. - */ - def map[U: ClassManifest](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) - - /** - * Return a new RDD by first applying a function to all elements of this - * RDD, and then flattening the results. - */ - def flatMap[U: ClassManifest](f: T => TraversableOnce[U]): RDD[U] = - new FlatMappedRDD(this, sc.clean(f)) - - /** - * Return a new RDD containing only the elements that satisfy a predicate. - */ - def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f)) - - /** - * Return a new RDD containing the distinct elements in this RDD. - */ - def distinct(numPartitions: Int): RDD[T] = - map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) - - def distinct(): RDD[T] = distinct(partitions.size) - - /** - * Return a new RDD that is reduced into `numPartitions` partitions. - */ - def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { - if (shuffle) { - // include a shuffle step so that our upstream tasks are still distributed - new CoalescedRDD( - new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), - new HashPartitioner(numPartitions)), - numPartitions).keys - } else { - new CoalescedRDD(this, numPartitions) - } - } - - /** - * Return a sampled subset of this RDD. - */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = - new SampledRDD(this, withReplacement, fraction, seed) - - def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { - var fraction = 0.0 - var total = 0 - val multiplier = 3.0 - val initialCount = this.count() - var maxSelected = 0 - - if (num < 0) { - throw new IllegalArgumentException("Negative number of elements requested") - } - - if (initialCount > Integer.MAX_VALUE - 1) { - maxSelected = Integer.MAX_VALUE - 1 - } else { - maxSelected = initialCount.toInt - } - - if (num > initialCount && !withReplacement) { - total = maxSelected - fraction = multiplier * (maxSelected + 1) / initialCount - } else { - fraction = multiplier * (num + 1) / initialCount - total = num - } - - val rand = new Random(seed) - var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() - - // If the first sample didn't turn out large enough, keep trying to take samples; - // this shouldn't happen often because we use a big multiplier for thei initial size - while (samples.length < total) { - samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() - } - - Utils.randomizeInPlace(samples, rand).take(total) - } - - /** - * Return the union of this RDD and another one. Any identical elements will appear multiple - * times (use `.distinct()` to eliminate them). - */ - def union(other: RDD[T]): RDD[T] = new UnionRDD(sc, Array(this, other)) - - /** - * Return the union of this RDD and another one. Any identical elements will appear multiple - * times (use `.distinct()` to eliminate them). - */ - def ++(other: RDD[T]): RDD[T] = this.union(other) - - /** - * Return an RDD created by coalescing all elements within each partition into an array. - */ - def glom(): RDD[Array[T]] = new GlommedRDD(this) - - /** - * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of - * elements (a, b) where a is in `this` and b is in `other`. - */ - def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) - - /** - * Return an RDD of grouped items. - */ - def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = - groupBy[K](f, defaultPartitioner(this)) - - /** - * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements - * mapping to that key. - */ - def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = - groupBy(f, new HashPartitioner(numPartitions)) - - /** - * Return an RDD of grouped items. - */ - def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { - val cleanF = sc.clean(f) - this.map(t => (cleanF(t), t)).groupByKey(p) - } - - /** - * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: String): RDD[String] = new PipedRDD(this, command) - - /** - * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: String, env: Map[String, String]): RDD[String] = - new PipedRDD(this, command, env) - - - /** - * Return an RDD created by piping elements to a forked external process. - * The print behavior can be customized by providing two functions. - * - * @param command command to run in forked process. - * @param env environment variables to set. - * @param printPipeContext Before piping elements, this function is called as an oppotunity - * to pipe context data. Print line function (like out.println) will be - * passed as printPipeContext's parameter. - * @param printRDDElement Use this function to customize how to pipe elements. This function - * will be called with each RDD element as the 1st parameter, and the - * print line function (like out.println()) as the 2nd parameter. - * An example of pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the elements: - * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = - * for (e <- record._2){f(e)} - * @return the result RDD - */ - def pipe( - command: Seq[String], - env: Map[String, String] = Map(), - printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = - new PipedRDD(this, command, env, - if (printPipeContext ne null) sc.clean(printPipeContext) else null, - if (printRDDElement ne null) sc.clean(printRDDElement) else null) - - /** - * Return a new RDD by applying a function to each partition of this RDD. - */ - def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U], - preservesPartitioning: Boolean = false): RDD[U] = - new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) - - /** - * Return a new RDD by applying a function to each partition of this RDD, while tracking the index - * of the original partition. - */ - def mapPartitionsWithIndex[U: ClassManifest]( - f: (Int, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean = false): RDD[U] = - new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) - - /** - * Return a new RDD by applying a function to each partition of this RDD, while tracking the index - * of the original partition. - */ - @deprecated("use mapPartitionsWithIndex", "0.7.0") - def mapPartitionsWithSplit[U: ClassManifest]( - f: (Int, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean = false): RDD[U] = - new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) - - /** - * Maps f over this RDD, where f takes an additional parameter of type A. This - * additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - def mapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) - (f:(T, A) => U): RDD[U] = { - def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { - val a = constructA(index) - iter.map(t => f(t, a)) - } - new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning) - } - - /** - * FlatMaps f over this RDD, where f takes an additional parameter of type A. This - * additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - def flatMapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) - (f:(T, A) => Seq[U]): RDD[U] = { - def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { - val a = constructA(index) - iter.flatMap(t => f(t, a)) - } - new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning) - } - - /** - * Applies f to each element of this RDD, where f takes an additional parameter of type A. - * This additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - def foreachWith[A: ClassManifest](constructA: Int => A) - (f:(T, A) => Unit) { - def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { - val a = constructA(index) - iter.map(t => {f(t, a); t}) - } - (new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true)).foreach(_ => {}) - } - - /** - * Filters this RDD with p, where p takes an additional parameter of type A. This - * additional parameter is produced by constructA, which is called in each - * partition with the index of that partition. - */ - def filterWith[A: ClassManifest](constructA: Int => A) - (p:(T, A) => Boolean): RDD[T] = { - def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { - val a = constructA(index) - iter.filter(t => p(t, a)) - } - new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true) - } - - /** - * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, - * second element in each RDD, etc. Assumes that the two RDDs have the *same number of - * partitions* and the *same number of elements in each partition* (e.g. one was made through - * a map on the other). - */ - def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) - - /** - * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by - * applying a function to the zipped partitions. Assumes that all the RDDs have the - * *same number of partitions*, but does *not* require them to have the same number - * of elements in each partition. - */ - def zipPartitions[B: ClassManifest, V: ClassManifest] - (rdd2: RDD[B]) - (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) - - def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] - (rdd2: RDD[B], rdd3: RDD[C]) - (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) - - def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] - (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D]) - (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = - new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) - - - // Actions (launch a job to return a value to the user program) - - /** - * Applies a function f to all elements of this RDD. - */ - def foreach(f: T => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) - } - - /** - * Applies a function f to each partition of this RDD. - */ - def foreachPartition(f: Iterator[T] => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) - } - - /** - * Return an array that contains all of the elements in this RDD. - */ - def collect(): Array[T] = { - val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray) - Array.concat(results: _*) - } - - /** - * Return an array that contains all of the elements in this RDD. - */ - def toArray(): Array[T] = collect() - - /** - * Return an RDD that contains all matching values by applying `f`. - */ - def collect[U: ClassManifest](f: PartialFunction[T, U]): RDD[U] = { - filter(f.isDefinedAt).map(f) - } - - /** - * Return an RDD with the elements from `this` that are not in `other`. - * - * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. - */ - def subtract(other: RDD[T]): RDD[T] = - subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) - - /** - * Return an RDD with the elements from `this` that are not in `other`. - */ - def subtract(other: RDD[T], numPartitions: Int): RDD[T] = - subtract(other, new HashPartitioner(numPartitions)) - - /** - * Return an RDD with the elements from `this` that are not in `other`. - */ - def subtract(other: RDD[T], p: Partitioner): RDD[T] = { - if (partitioner == Some(p)) { - // Our partitioner knows how to handle T (which, since we have a partitioner, is - // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples - val p2 = new Partitioner() { - override def numPartitions = p.numPartitions - override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) - } - // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies - // anyway, and when calling .keys, will not have a partitioner set, even though - // the SubtractedRDD will, thanks to p2's de-tupled partitioning, already be - // partitioned by the right/real keys (e.g. p). - this.map(x => (x, null)).subtractByKey(other.map((_, null)), p2).keys - } else { - this.map(x => (x, null)).subtractByKey(other.map((_, null)), p).keys - } - } - - /** - * Reduces the elements of this RDD using the specified commutative and associative binary operator. - */ - def reduce(f: (T, T) => T): T = { - val cleanF = sc.clean(f) - val reducePartition: Iterator[T] => Option[T] = iter => { - if (iter.hasNext) { - Some(iter.reduceLeft(cleanF)) - } else { - None - } - } - var jobResult: Option[T] = None - val mergeResult = (index: Int, taskResult: Option[T]) => { - if (taskResult != None) { - jobResult = jobResult match { - case Some(value) => Some(f(value, taskResult.get)) - case None => taskResult - } - } - } - sc.runJob(this, reducePartition, mergeResult) - // Get the final result out of our Option, or throw an exception if the RDD was empty - jobResult.getOrElse(throw new UnsupportedOperationException("empty collection")) - } - - /** - * Aggregate the elements of each partition, and then the results for all the partitions, using a - * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to - * modify t1 and return it as its result value to avoid object allocation; however, it should not - * modify t2. - */ - def fold(zeroValue: T)(op: (T, T) => T): T = { - // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) - val cleanOp = sc.clean(op) - val foldPartition = (iter: Iterator[T]) => iter.fold(zeroValue)(cleanOp) - val mergeResult = (index: Int, taskResult: T) => jobResult = op(jobResult, taskResult) - sc.runJob(this, foldPartition, mergeResult) - jobResult - } - - /** - * Aggregate the elements of each partition, and then the results for all the partitions, using - * given combine functions and a neutral "zero value". This function can return a different result - * type, U, than the type of this RDD, T. Thus, we need one operation for merging a T into an U - * and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are - * allowed to modify and return their first argument instead of creating a new U to avoid memory - * allocation. - */ - def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { - // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) - val cleanSeqOp = sc.clean(seqOp) - val cleanCombOp = sc.clean(combOp) - val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) - val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) - sc.runJob(this, aggregatePartition, mergeResult) - jobResult - } - - /** - * Return the number of elements in the RDD. - */ - def count(): Long = { - sc.runJob(this, (iter: Iterator[T]) => { - var result = 0L - while (iter.hasNext) { - result += 1L - iter.next() - } - result - }).sum - } - - /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result - * within a timeout, even if not all tasks have finished. - */ - def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { - val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => - var result = 0L - while (iter.hasNext) { - result += 1L - iter.next() - } - result - } - val evaluator = new CountEvaluator(partitions.size, confidence) - sc.runApproximateJob(this, countElements, evaluator, timeout) - } - - /** - * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final - * combine step happens locally on the master, equivalent to running a single reduce task. - */ - def countByValue(): Map[T, Long] = { - if (elementClassManifest.erasure.isArray) { - throw new SparkException("countByValue() does not support arrays") - } - // TODO: This should perhaps be distributed by default. - def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = { - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) - } - Iterator(map) - } - def mergeMaps(m1: OLMap[T], m2: OLMap[T]): OLMap[T] = { - val iter = m2.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue) - } - return m1 - } - val myResult = mapPartitions(countPartition).reduce(mergeMaps) - myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map - } - - /** - * (Experimental) Approximate version of countByValue(). - */ - def countByValueApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[Map[T, BoundedDouble]] = { - if (elementClassManifest.erasure.isArray) { - throw new SparkException("countByValueApprox() does not support arrays") - } - val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) - } - map - } - val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence) - sc.runApproximateJob(this, countPartition, evaluator, timeout) - } - - /** - * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so - * it will be slow if a lot of partitions are required. In that case, use collect() to get the - * whole RDD instead. - */ - def take(num: Int): Array[T] = { - if (num == 0) { - return new Array[T](0) - } - val buf = new ArrayBuffer[T] - var p = 0 - while (buf.size < num && p < partitions.size) { - val left = num - buf.size - val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true) - buf ++= res(0) - if (buf.size == num) - return buf.toArray - p += 1 - } - return buf.toArray - } - - /** - * Return the first element in this RDD. - */ - def first(): T = take(1) match { - case Array(t) => t - case _ => throw new UnsupportedOperationException("empty collection") - } - - /** - * Returns the top K elements from this RDD as defined by - * the specified implicit Ordering[T]. - * @param num the number of top elements to return - * @param ord the implicit ordering for T - * @return an array of top elements - */ - def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { - mapPartitions { items => - val queue = new BoundedPriorityQueue[T](num) - queue ++= items - Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord.reverse) - } - - /** - * Returns the first K elements from this RDD as defined by - * the specified implicit Ordering[T] and maintains the - * ordering. - * @param num the number of top elements to return - * @param ord the implicit ordering for T - * @return an array of top elements - */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) - - /** - * Save this RDD as a text file, using string representations of elements. - */ - def saveAsTextFile(path: String) { - this.map(x => (NullWritable.get(), new Text(x.toString))) - .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) - } - - /** - * Save this RDD as a compressed text file, using string representations of elements. - */ - def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { - this.map(x => (NullWritable.get(), new Text(x.toString))) - .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) - } - - /** - * Save this RDD as a SequenceFile of serialized objects. - */ - def saveAsObjectFile(path: String) { - this.mapPartitions(iter => iter.grouped(10).map(_.toArray)) - .map(x => (NullWritable.get(), new BytesWritable(Utils.serialize(x)))) - .saveAsSequenceFile(path) - } - - /** - * Creates tuples of the elements in this RDD by applying `f`. - */ - def keyBy[K](f: T => K): RDD[(K, T)] = { - map(x => (f(x), x)) - } - - /** A private method for tests, to look at the contents of each partition */ - private[spark] def collectPartitions(): Array[Array[T]] = { - sc.runJob(this, (iter: Iterator[T]) => iter.toArray) - } - - /** - * Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint - * directory set with SparkContext.setCheckpointDir() and all references to its parent - * RDDs will be removed. This function must be called before any job has been - * executed on this RDD. It is strongly recommended that this RDD is persisted in - * memory, otherwise saving it on a file will require recomputation. - */ - def checkpoint() { - if (context.checkpointDir.isEmpty) { - throw new Exception("Checkpoint directory has not been set in the SparkContext") - } else if (checkpointData.isEmpty) { - checkpointData = Some(new RDDCheckpointData(this)) - checkpointData.get.markForCheckpoint() - } - } - - /** - * Return whether this RDD has been checkpointed or not - */ - def isCheckpointed: Boolean = { - checkpointData.map(_.isCheckpointed).getOrElse(false) - } - - /** - * Gets the name of the file to which this RDD was checkpointed - */ - def getCheckpointFile: Option[String] = { - checkpointData.flatMap(_.getCheckpointFile) - } - - // ======================================================================= - // Other internal methods and fields - // ======================================================================= - - private var storageLevel: StorageLevel = StorageLevel.NONE - - /** Record user function generating this RDD. */ - private[spark] val origin = Utils.formatSparkCallSite - - private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] - - private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None - - /** Returns the first parent RDD */ - protected[spark] def firstParent[U: ClassManifest] = { - dependencies.head.rdd.asInstanceOf[RDD[U]] - } - - /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ - def context = sc - - // Avoid handling doCheckpoint multiple times to prevent excessive recursion - private var doCheckpointCalled = false - - /** - * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler - * after a job using this RDD has completed (therefore the RDD has been materialized and - * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. - */ - private[spark] def doCheckpoint() { - if (!doCheckpointCalled) { - doCheckpointCalled = true - if (checkpointData.isDefined) { - checkpointData.get.doCheckpoint() - } else { - dependencies.foreach(_.rdd.doCheckpoint()) - } - } - } - - /** - * Changes the dependencies of this RDD from its original parents to a new RDD (`newRDD`) - * created from the checkpoint file, and forget its old dependencies and partitions. - */ - private[spark] def markCheckpointed(checkpointRDD: RDD[_]) { - clearDependencies() - partitions_ = null - deps = null // Forget the constructor argument for dependencies too - } - - /** - * Clears the dependencies of this RDD. This method must ensure that all references - * to the original parent RDDs is removed to enable the parent RDDs to be garbage - * collected. Subclasses of RDD may override this method for implementing their own cleaning - * logic. See [[org.apache.spark.rdd.UnionRDD]] for an example. - */ - protected def clearDependencies() { - dependencies_ = null - } - - /** A description of this RDD and its recursive dependencies for debugging. */ - def toDebugString: String = { - def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { - Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++ - rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) - } - debugString(this).mkString("\n") - } - - override def toString: String = "%s%s[%d] at %s".format( - Option(name).map(_ + " ").getOrElse(""), - getClass.getSimpleName, - id, - origin) - - def toJavaRDD() : JavaRDD[T] = { - new JavaRDD(this)(elementClassManifest) - } - -} diff --git a/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala deleted file mode 100644 index 0334de6924..0000000000 --- a/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala +++ /dev/null @@ -1,130 +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 - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.conf.Configuration -import rdd.{CheckpointRDD, CoalescedRDD} -import scheduler.{ResultTask, ShuffleMapTask} - -/** - * Enumeration to manage state transitions of an RDD through checkpointing - * [ Initialized --> marked for checkpointing --> checkpointing in progress --> checkpointed ] - */ -private[spark] object CheckpointState extends Enumeration { - type CheckpointState = Value - val Initialized, MarkedForCheckpoint, CheckpointingInProgress, Checkpointed = Value -} - -/** - * This class contains all the information related to RDD checkpointing. Each instance of this class - * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as, - * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations - * of the checkpointed RDD. - */ -private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) - extends Logging with Serializable { - - import CheckpointState._ - - // The checkpoint state of the associated RDD. - var cpState = Initialized - - // The file to which the associated RDD has been checkpointed to - @transient var cpFile: Option[String] = None - - // The CheckpointRDD created from the checkpoint file, that is, the new parent the associated RDD. - var cpRDD: Option[RDD[T]] = None - - // Mark the RDD for checkpointing - def markForCheckpoint() { - RDDCheckpointData.synchronized { - if (cpState == Initialized) cpState = MarkedForCheckpoint - } - } - - // Is the RDD already checkpointed - def isCheckpointed: Boolean = { - RDDCheckpointData.synchronized { cpState == Checkpointed } - } - - // Get the file to which this RDD was checkpointed to as an Option - def getCheckpointFile: Option[String] = { - RDDCheckpointData.synchronized { cpFile } - } - - // Do the checkpointing of the RDD. Called after the first job using that RDD is over. - def doCheckpoint() { - // If it is marked for checkpointing AND checkpointing is not already in progress, - // then set it to be in progress, else return - RDDCheckpointData.synchronized { - if (cpState == MarkedForCheckpoint) { - cpState = CheckpointingInProgress - } else { - return - } - } - - // Create the output path for the checkpoint - val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) - val fs = path.getFileSystem(new Configuration()) - if (!fs.mkdirs(path)) { - throw new SparkException("Failed to create checkpoint path " + path) - } - - // Save to file, and reload it as an RDD - rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString) _) - val newRDD = new CheckpointRDD[T](rdd.context, path.toString) - - // Change the dependencies and partitions of the RDD - RDDCheckpointData.synchronized { - cpFile = Some(path.toString) - cpRDD = Some(newRDD) - rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions - cpState = Checkpointed - RDDCheckpointData.clearTaskCaches() - logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id) - } - } - - // Get preferred location of a split after checkpointing - def getPreferredLocations(split: Partition): Seq[String] = { - RDDCheckpointData.synchronized { - cpRDD.get.preferredLocations(split) - } - } - - def getPartitions: Array[Partition] = { - RDDCheckpointData.synchronized { - cpRDD.get.partitions - } - } - - def checkpointRDD: Option[RDD[T]] = { - RDDCheckpointData.synchronized { - cpRDD - } - } -} - -private[spark] object RDDCheckpointData { - def clearTaskCaches() { - ShuffleMapTask.clearCache() - ResultTask.clearCache() - } -} diff --git a/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala deleted file mode 100644 index d58fb4e4bc..0000000000 --- a/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala +++ /dev/null @@ -1,107 +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 - -import java.io.EOFException -import java.net.URL -import java.io.ObjectInputStream -import java.util.concurrent.atomic.AtomicLong -import java.util.HashSet -import java.util.Random -import java.util.Date - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.Map -import scala.collection.mutable.HashMap - -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.OutputFormat -import org.apache.hadoop.mapred.TextOutputFormat -import org.apache.hadoop.mapred.SequenceFileOutputFormat -import org.apache.hadoop.mapred.OutputCommitter -import org.apache.hadoop.mapred.FileOutputCommitter -import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.io.Writable -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.Text - -import org.apache.spark.SparkContext._ - -/** - * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, - * through an implicit conversion. Note that this can't be part of PairRDDFunctions because - * we need more implicit parameters to convert our keys and values to Writable. - * - * Users should import `spark.SparkContext._` at the top of their program to use these functions. - */ -class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest]( - self: RDD[(K, V)]) - extends Logging - with Serializable { - - private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = { - val c = { - if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { - classManifest[T].erasure - } 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]] - } - - /** - * Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key - * and value types. If the key or value are Writable, then we use their classes directly; - * otherwise we map primitive types such as Int and Double to IntWritable, DoubleWritable, etc, - * byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported - * file system. - */ - def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) { - def anyToWritable[U <% Writable](u: U): Writable = u - - val keyClass = getWritableClass[K] - val valueClass = getWritableClass[V] - val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) - val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) - - logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) - val format = classOf[SequenceFileOutputFormat[Writable, Writable]] - val jobConf = new JobConf(self.context.hadoopConfiguration) - if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyClass, valueClass, format, jobConf, codec) - } else if (!convertKey && convertValue) { - self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile( - path, keyClass, valueClass, format, jobConf, codec) - } else if (convertKey && !convertValue) { - self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile( - path, keyClass, valueClass, format, jobConf, codec) - } else if (convertKey && convertValue) { - self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile( - path, keyClass, valueClass, format, jobConf, codec) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/SizeEstimator.scala deleted file mode 100644 index 4bfc837710..0000000000 --- a/core/src/main/scala/org/apache/spark/SizeEstimator.scala +++ /dev/null @@ -1,283 +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 - -import java.lang.reflect.Field -import java.lang.reflect.Modifier -import java.lang.reflect.{Array => JArray} -import java.util.IdentityHashMap -import java.util.concurrent.ConcurrentHashMap -import java.util.Random - -import javax.management.MBeanServer -import java.lang.management.ManagementFactory - -import scala.collection.mutable.ArrayBuffer - -import it.unimi.dsi.fastutil.ints.IntOpenHashSet - -/** - * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in - * memory-aware caches. - * - * Based on the following JavaWorld article: - * http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html - */ -private[spark] object SizeEstimator extends Logging { - - // Sizes of primitive types - private val BYTE_SIZE = 1 - private val BOOLEAN_SIZE = 1 - private val CHAR_SIZE = 2 - private val SHORT_SIZE = 2 - private val INT_SIZE = 4 - private val LONG_SIZE = 8 - private val FLOAT_SIZE = 4 - private val DOUBLE_SIZE = 8 - - // Alignment boundary for objects - // TODO: Is this arch dependent ? - private val ALIGN_SIZE = 8 - - // A cache of ClassInfo objects for each class - private val classInfos = new ConcurrentHashMap[Class[_], ClassInfo] - - // Object and pointer sizes are arch dependent - private var is64bit = false - - // Size of an object reference - // Based on https://wikis.oracle.com/display/HotSpotInternals/CompressedOops - private var isCompressedOops = false - private var pointerSize = 4 - - // Minimum size of a java.lang.Object - private var objectSize = 8 - - initialize() - - // Sets object size, pointer size based on architecture and CompressedOops settings - // from the JVM. - private def initialize() { - is64bit = System.getProperty("os.arch").contains("64") - isCompressedOops = getIsCompressedOops - - objectSize = if (!is64bit) 8 else { - if(!isCompressedOops) { - 16 - } else { - 12 - } - } - pointerSize = if (is64bit && !isCompressedOops) 8 else 4 - classInfos.clear() - classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil)) - } - - private def getIsCompressedOops : Boolean = { - if (System.getProperty("spark.test.useCompressedOops") != null) { - return System.getProperty("spark.test.useCompressedOops").toBoolean - } - - try { - val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic" - val server = ManagementFactory.getPlatformMBeanServer() - - // NOTE: This should throw an exception in non-Sun JVMs - val hotSpotMBeanClass = Class.forName("com.sun.management.HotSpotDiagnosticMXBean") - val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption", - Class.forName("java.lang.String")) - - val bean = ManagementFactory.newPlatformMXBeanProxy(server, - hotSpotMBeanName, hotSpotMBeanClass) - // TODO: We could use reflection on the VMOption returned ? - return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") - } catch { - case e: Exception => { - // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB - val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) - val guessInWords = if (guess) "yes" else "not" - logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords) - return guess - } - } - } - - /** - * The state of an ongoing size estimation. Contains a stack of objects to visit as well as an - * IdentityHashMap of visited objects, and provides utility methods for enqueueing new objects - * to visit. - */ - private class SearchState(val visited: IdentityHashMap[AnyRef, AnyRef]) { - val stack = new ArrayBuffer[AnyRef] - var size = 0L - - def enqueue(obj: AnyRef) { - if (obj != null && !visited.containsKey(obj)) { - visited.put(obj, null) - stack += obj - } - } - - def isFinished(): Boolean = stack.isEmpty - - def dequeue(): AnyRef = { - val elem = stack.last - stack.trimEnd(1) - return elem - } - } - - /** - * Cached information about each class. We remember two things: the "shell size" of the class - * (size of all non-static fields plus the java.lang.Object size), and any fields that are - * pointers to objects. - */ - private class ClassInfo( - val shellSize: Long, - val pointerFields: List[Field]) {} - - def estimate(obj: AnyRef): Long = estimate(obj, new IdentityHashMap[AnyRef, AnyRef]) - - private def estimate(obj: AnyRef, visited: IdentityHashMap[AnyRef, AnyRef]): Long = { - val state = new SearchState(visited) - state.enqueue(obj) - while (!state.isFinished) { - visitSingleObject(state.dequeue(), state) - } - return state.size - } - - private def visitSingleObject(obj: AnyRef, state: SearchState) { - val cls = obj.getClass - if (cls.isArray) { - visitArray(obj, cls, state) - } else if (obj.isInstanceOf[ClassLoader] || obj.isInstanceOf[Class[_]]) { - // Hadoop JobConfs created in the interpreter have a ClassLoader, which greatly confuses - // the size estimator since it references the whole REPL. Do nothing in this case. In - // general all ClassLoaders and Classes will be shared between objects anyway. - } else { - val classInfo = getClassInfo(cls) - state.size += classInfo.shellSize - for (field <- classInfo.pointerFields) { - state.enqueue(field.get(obj)) - } - } - } - - // Estimat the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. - private val ARRAY_SIZE_FOR_SAMPLING = 200 - private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING - - private def visitArray(array: AnyRef, cls: Class[_], state: SearchState) { - val length = JArray.getLength(array) - val elementClass = cls.getComponentType - - // Arrays have object header and length field which is an integer - var arrSize: Long = alignSize(objectSize + INT_SIZE) - - if (elementClass.isPrimitive) { - arrSize += alignSize(length * primitiveSize(elementClass)) - state.size += arrSize - } else { - arrSize += alignSize(length * pointerSize) - state.size += arrSize - - if (length <= ARRAY_SIZE_FOR_SAMPLING) { - for (i <- 0 until length) { - state.enqueue(JArray.get(array, i)) - } - } else { - // Estimate the size of a large array by sampling elements without replacement. - var size = 0.0 - val rand = new Random(42) - val drawn = new IntOpenHashSet(ARRAY_SAMPLE_SIZE) - for (i <- 0 until ARRAY_SAMPLE_SIZE) { - var index = 0 - do { - index = rand.nextInt(length) - } while (drawn.contains(index)) - drawn.add(index) - val elem = JArray.get(array, index) - size += SizeEstimator.estimate(elem, state.visited) - } - state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong - } - } - } - - private def primitiveSize(cls: Class[_]): Long = { - if (cls == classOf[Byte]) - BYTE_SIZE - else if (cls == classOf[Boolean]) - BOOLEAN_SIZE - else if (cls == classOf[Char]) - CHAR_SIZE - else if (cls == classOf[Short]) - SHORT_SIZE - else if (cls == classOf[Int]) - INT_SIZE - else if (cls == classOf[Long]) - LONG_SIZE - else if (cls == classOf[Float]) - FLOAT_SIZE - else if (cls == classOf[Double]) - DOUBLE_SIZE - else throw new IllegalArgumentException( - "Non-primitive class " + cls + " passed to primitiveSize()") - } - - /** - * Get or compute the ClassInfo for a given class. - */ - private def getClassInfo(cls: Class[_]): ClassInfo = { - // Check whether we've already cached a ClassInfo for this class - val info = classInfos.get(cls) - if (info != null) { - return info - } - - val parent = getClassInfo(cls.getSuperclass) - var shellSize = parent.shellSize - var pointerFields = parent.pointerFields - - for (field <- cls.getDeclaredFields) { - if (!Modifier.isStatic(field.getModifiers)) { - val fieldClass = field.getType - if (fieldClass.isPrimitive) { - shellSize += primitiveSize(fieldClass) - } else { - field.setAccessible(true) // Enable future get()'s on this field - shellSize += pointerSize - pointerFields = field :: pointerFields - } - } - } - - shellSize = alignSize(shellSize) - - // Create and cache a new ClassInfo - val newInfo = new ClassInfo(shellSize, pointerFields) - classInfos.put(cls, newInfo) - return newInfo - } - - private def alignSize(size: Long): Long = { - val rem = size % ALIGN_SIZE - return if (rem == 0) size else (size + ALIGN_SIZE - rem) - } -} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1207b242bc..faf0c2362a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -54,17 +54,15 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.deploy.LocalSparkCluster import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, - OrderedRDDFunctions} +import org.apache.spark.rdd._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, SchedulingMode} import org.apache.spark.scheduler.local.LocalScheduler import org.apache.spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} +import org.apache.spark.storage.{StorageUtils, BlockManagerSource} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} -import scala.Some +import org.apache.spark.util.{ClosureCleaner, Utils, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.RDDInfo import org.apache.spark.storage.StorageStatus diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6e6fe5df6b..478e5a0aaf 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster} import org.apache.spark.network.ConnectionManager import org.apache.spark.serializer.{Serializer, SerializerManager} -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.api.python.PythonWorkerFactory @@ -155,10 +155,10 @@ object SparkEnv extends Logging { val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - System.getProperty("spark.serializer", "org.apache.spark.JavaSerializer")) + System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) val closureSerializer = serializerManager.get( - System.getProperty("spark.closure.serializer", "org.apache.spark.JavaSerializer")) + System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { diff --git a/core/src/main/scala/org/apache/spark/Utils.scala b/core/src/main/scala/org/apache/spark/Utils.scala deleted file mode 100644 index 1e17deb010..0000000000 --- a/core/src/main/scala/org/apache/spark/Utils.scala +++ /dev/null @@ -1,780 +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 - -import java.io._ -import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} -import java.util.{Locale, Random, UUID} -import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import java.util.regex.Pattern - -import scala.collection.Map -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.collection.JavaConversions._ -import scala.io.Source - -import com.google.common.io.Files -import com.google.common.util.concurrent.ThreadFactoryBuilder - -import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} - -import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import org.apache.spark.deploy.SparkHadoopUtil -import java.nio.ByteBuffer - - -/** - * Various utility methods used by Spark. - */ -private object Utils extends Logging { - - /** Serialize an object using Java serialization */ - def serialize[T](o: T): Array[Byte] = { - val bos = new ByteArrayOutputStream() - val oos = new ObjectOutputStream(bos) - oos.writeObject(o) - oos.close() - return bos.toByteArray - } - - /** Deserialize an object using Java serialization */ - def deserialize[T](bytes: Array[Byte]): T = { - val bis = new ByteArrayInputStream(bytes) - val ois = new ObjectInputStream(bis) - return ois.readObject.asInstanceOf[T] - } - - /** Deserialize an object using Java serialization and the given ClassLoader */ - def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = { - val bis = new ByteArrayInputStream(bytes) - val ois = new ObjectInputStream(bis) { - override def resolveClass(desc: ObjectStreamClass) = - Class.forName(desc.getName, false, loader) - } - return ois.readObject.asInstanceOf[T] - } - - /** Serialize via nested stream using specific serializer */ - def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)(f: SerializationStream => Unit) = { - val osWrapper = ser.serializeStream(new OutputStream { - def write(b: Int) = os.write(b) - - override def write(b: Array[Byte], off: Int, len: Int) = os.write(b, off, len) - }) - try { - f(osWrapper) - } finally { - osWrapper.close() - } - } - - /** Deserialize via nested stream using specific serializer */ - def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)(f: DeserializationStream => Unit) = { - val isWrapper = ser.deserializeStream(new InputStream { - def read(): Int = is.read() - - override def read(b: Array[Byte], off: Int, len: Int): Int = is.read(b, off, len) - }) - try { - f(isWrapper) - } finally { - isWrapper.close() - } - } - - /** - * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. - */ - def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = { - if (bb.hasArray) { - out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) - } else { - val bbval = new Array[Byte](bb.remaining()) - bb.get(bbval) - out.write(bbval) - } - } - - def isAlpha(c: Char): Boolean = { - (c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z') - } - - /** Split a string into words at non-alphabetic characters */ - def splitWords(s: String): Seq[String] = { - val buf = new ArrayBuffer[String] - var i = 0 - while (i < s.length) { - var j = i - while (j < s.length && isAlpha(s.charAt(j))) { - j += 1 - } - if (j > i) { - buf += s.substring(i, j) - } - i = j - while (i < s.length && !isAlpha(s.charAt(i))) { - i += 1 - } - } - return buf - } - - private val shutdownDeletePaths = new collection.mutable.HashSet[String]() - - // Register the path to be deleted via shutdown hook - def registerShutdownDeleteDir(file: File) { - val absolutePath = file.getAbsolutePath() - shutdownDeletePaths.synchronized { - shutdownDeletePaths += absolutePath - } - } - - // Is the path already registered to be deleted via a shutdown hook ? - def hasShutdownDeleteDir(file: File): Boolean = { - val absolutePath = file.getAbsolutePath() - shutdownDeletePaths.synchronized { - shutdownDeletePaths.contains(absolutePath) - } - } - - // Note: if file is child of some registered path, while not equal to it, then return true; - // else false. This is to ensure that two shutdown hooks do not try to delete each others - // paths - resulting in IOException and incomplete cleanup. - def hasRootAsShutdownDeleteDir(file: File): Boolean = { - val absolutePath = file.getAbsolutePath() - val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => - !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined - } - if (retval) { - logInfo("path = " + file + ", already present as root for deletion.") - } - retval - } - - /** Create a temporary directory inside the given parent directory */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { - var attempts = 0 - val maxAttempts = 10 - var dir: File = null - while (dir == null) { - attempts += 1 - if (attempts > maxAttempts) { - throw new IOException("Failed to create a temp directory (under " + root + ") after " + - maxAttempts + " attempts!") - } - try { - dir = new File(root, "spark-" + UUID.randomUUID.toString) - if (dir.exists() || !dir.mkdirs()) { - dir = null - } - } catch { case e: IOException => ; } - } - - registerShutdownDeleteDir(dir) - - // Add a shutdown hook to delete the temp dir when the JVM exits - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dir " + dir) { - override def run() { - // Attempt to delete if some patch which is parent of this is not already registered. - if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir) - } - }) - dir - } - - /** Copy all data from an InputStream to an OutputStream */ - def copyStream(in: InputStream, - out: OutputStream, - closeStreams: Boolean = false) - { - val buf = new Array[Byte](8192) - var n = 0 - while (n != -1) { - n = in.read(buf) - if (n != -1) { - out.write(buf, 0, n) - } - } - if (closeStreams) { - in.close() - out.close() - } - } - - /** - * Download a file requested by the executor. Supports fetching the file in a variety of ways, - * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. - * - * Throws SparkException if the target file already exists and has different contents than - * the requested file. - */ - def fetchFile(url: String, targetDir: File) { - val filename = url.split("/").last - val tempDir = getLocalDir - val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) - val targetFile = new File(targetDir, filename) - val uri = new URI(url) - uri.getScheme match { - case "http" | "https" | "ftp" => - logInfo("Fetching " + url + " to " + tempFile) - val in = new URL(url).openStream() - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - tempFile.delete() - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } else { - Files.move(tempFile, targetFile) - } - case "file" | null => - // In the case of a local file, copy the local file to the target directory. - // Note the difference between uri vs url. - val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) - if (targetFile.exists) { - // If the target file already exists, warn the user if - if (!Files.equal(sourceFile, targetFile)) { - throw new SparkException( - "File " + targetFile + " exists and does not match contents of" + " " + url) - } else { - // Do nothing if the file contents are the same, i.e. this file has been copied - // previously. - logInfo(sourceFile.getAbsolutePath + " has been previously copied to " - + targetFile.getAbsolutePath) - } - } else { - // The file does not exist in the target directory. Copy it there. - logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) - Files.copy(sourceFile, targetFile) - } - case _ => - // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others - val env = SparkEnv.get - val uri = new URI(url) - val conf = env.hadoop.newConfiguration() - val fs = FileSystem.get(uri, conf) - val in = fs.open(new Path(uri)) - val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, true) - if (targetFile.exists && !Files.equal(tempFile, targetFile)) { - tempFile.delete() - throw new SparkException("File " + targetFile + " exists and does not match contents of" + - " " + url) - } else { - Files.move(tempFile, targetFile) - } - } - // Decompress the file if it's a .tar or .tar.gz - if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xzf", filename), targetDir) - } else if (filename.endsWith(".tar")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xf", filename), targetDir) - } - // Make the file executable - That's necessary for scripts - FileUtil.chmod(targetFile.getAbsolutePath, "a+x") - } - - /** - * Get a temporary directory using Spark's spark.local.dir property, if set. This will always - * return a single directory, even though the spark.local.dir property might be a list of - * multiple paths. - */ - def getLocalDir: String = { - System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) - } - - /** - * Shuffle the elements of a collection into a random order, returning the - * result in a new collection. Unlike scala.util.Random.shuffle, this method - * uses a local random number generator, avoiding inter-thread contention. - */ - def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = { - randomizeInPlace(seq.toArray) - } - - /** - * Shuffle the elements of an array into a random order, modifying the - * original array. Returns the original array. - */ - def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = { - for (i <- (arr.length - 1) to 1 by -1) { - val j = rand.nextInt(i) - val tmp = arr(j) - arr(j) = arr(i) - arr(i) = tmp - } - arr - } - - /** - * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). - * Note, this is typically not used from within core spark. - */ - lazy val localIpAddress: String = findLocalIpAddress() - lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress) - - private def findLocalIpAddress(): String = { - val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") - if (defaultIpOverride != null) { - defaultIpOverride - } else { - val address = InetAddress.getLocalHost - if (address.isLoopbackAddress) { - // Address resolves to something like 127.0.1.1, which happens on Debian; try to find - // a better address using the local network interfaces - for (ni <- NetworkInterface.getNetworkInterfaces) { - for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && - !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) { - // We've found an address that looks reasonable! - logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + - " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress + - " instead (on interface " + ni.getName + ")") - logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") - return addr.getHostAddress - } - } - logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + - " a loopback address: " + address.getHostAddress + ", but we couldn't find any" + - " external IP address!") - logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") - } - address.getHostAddress - } - } - - private var customHostname: Option[String] = None - - /** - * Allow setting a custom host name because when we run on Mesos we need to use the same - * hostname it reports to the master. - */ - def setCustomHostname(hostname: String) { - // DEBUG code - Utils.checkHost(hostname) - customHostname = Some(hostname) - } - - /** - * Get the local machine's hostname. - */ - def localHostName(): String = { - customHostname.getOrElse(localIpAddressHostname) - } - - def getAddressHostName(address: String): String = { - InetAddress.getByName(address).getHostName - } - - def localHostPort(): String = { - val retval = System.getProperty("spark.hostPort", null) - if (retval == null) { - logErrorWithStack("spark.hostPort not set but invoking localHostPort") - return localHostName() - } - - retval - } - - def checkHost(host: String, message: String = "") { - assert(host.indexOf(':') == -1, message) - } - - def checkHostPort(hostPort: String, message: String = "") { - assert(hostPort.indexOf(':') != -1, message) - } - - // Used by DEBUG code : remove when all testing done - def logErrorWithStack(msg: String) { - try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } - } - - // Typically, this will be of order of number of nodes in cluster - // If not, we should change it to LRUCache or something. - private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() - - def parseHostPort(hostPort: String): (String, Int) = { - { - // Check cache first. - var cached = hostPortParseResults.get(hostPort) - if (cached != null) return cached - } - - val indx: Int = hostPort.lastIndexOf(':') - // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... - // but then hadoop does not support ipv6 right now. - // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 - if (-1 == indx) { - val retval = (hostPort, 0) - hostPortParseResults.put(hostPort, retval) - return retval - } - - val retval = (hostPort.substring(0, indx).trim(), hostPort.substring(indx + 1).trim().toInt) - hostPortParseResults.putIfAbsent(hostPort, retval) - hostPortParseResults.get(hostPort) - } - - private[spark] val daemonThreadFactory: ThreadFactory = - new ThreadFactoryBuilder().setDaemon(true).build() - - /** - * Wrapper over newCachedThreadPool. - */ - def newDaemonCachedThreadPool(): ThreadPoolExecutor = - Executors.newCachedThreadPool(daemonThreadFactory).asInstanceOf[ThreadPoolExecutor] - - /** - * Return the string to tell how long has passed in seconds. The passing parameter should be in - * millisecond. - */ - def getUsedTimeMs(startTimeMs: Long): String = { - return " " + (System.currentTimeMillis - startTimeMs) + " ms" - } - - /** - * Wrapper over newFixedThreadPool. - */ - def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = - Executors.newFixedThreadPool(nThreads, daemonThreadFactory).asInstanceOf[ThreadPoolExecutor] - - /** - * Delete a file or directory and its contents recursively. - */ - def deleteRecursively(file: File) { - if (file.isDirectory) { - for (child <- file.listFiles()) { - deleteRecursively(child) - } - } - if (!file.delete()) { - throw new IOException("Failed to delete: " + file) - } - } - - /** - * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. - * This is used to figure out how much memory to claim from Mesos based on the SPARK_MEM - * environment variable. - */ - def memoryStringToMb(str: String): Int = { - val lower = str.toLowerCase - if (lower.endsWith("k")) { - (lower.substring(0, lower.length-1).toLong / 1024).toInt - } else if (lower.endsWith("m")) { - lower.substring(0, lower.length-1).toInt - } else if (lower.endsWith("g")) { - lower.substring(0, lower.length-1).toInt * 1024 - } else if (lower.endsWith("t")) { - lower.substring(0, lower.length-1).toInt * 1024 * 1024 - } else {// no suffix, so it's just a number in bytes - (lower.toLong / 1024 / 1024).toInt - } - } - - /** - * Convert a quantity in bytes to a human-readable string such as "4.0 MB". - */ - def bytesToString(size: Long): String = { - val TB = 1L << 40 - val GB = 1L << 30 - val MB = 1L << 20 - val KB = 1L << 10 - - val (value, unit) = { - if (size >= 2*TB) { - (size.asInstanceOf[Double] / TB, "TB") - } else if (size >= 2*GB) { - (size.asInstanceOf[Double] / GB, "GB") - } else if (size >= 2*MB) { - (size.asInstanceOf[Double] / MB, "MB") - } else if (size >= 2*KB) { - (size.asInstanceOf[Double] / KB, "KB") - } else { - (size.asInstanceOf[Double], "B") - } - } - "%.1f %s".formatLocal(Locale.US, value, unit) - } - - /** - * Returns a human-readable string representing a duration such as "35ms" - */ - def msDurationToString(ms: Long): String = { - val second = 1000 - val minute = 60 * second - val hour = 60 * minute - - ms match { - case t if t < second => - "%d ms".format(t) - case t if t < minute => - "%.1f s".format(t.toFloat / second) - case t if t < hour => - "%.1f m".format(t.toFloat / minute) - case t => - "%.2f h".format(t.toFloat / hour) - } - } - - /** - * Convert a quantity in megabytes to a human-readable string such as "4.0 MB". - */ - def megabytesToString(megabytes: Long): String = { - bytesToString(megabytes * 1024L * 1024L) - } - - /** - * Execute a command in the given working directory, throwing an exception if it completes - * with an exit code other than 0. - */ - def execute(command: Seq[String], workingDir: File) { - val process = new ProcessBuilder(command: _*) - .directory(workingDir) - .redirectErrorStream(true) - .start() - new Thread("read stdout for " + command(0)) { - override def run() { - for (line <- Source.fromInputStream(process.getInputStream).getLines) { - System.err.println(line) - } - } - }.start() - val exitCode = process.waitFor() - if (exitCode != 0) { - throw new SparkException("Process " + command + " exited with code " + exitCode) - } - } - - /** - * Execute a command in the current working directory, throwing an exception if it completes - * with an exit code other than 0. - */ - def execute(command: Seq[String]) { - execute(command, new File(".")) - } - - /** - * Execute a command and get its output, throwing an exception if it yields a code other than 0. - */ - def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), - extraEnvironment: Map[String, String] = Map.empty): String = { - val builder = new ProcessBuilder(command: _*) - .directory(workingDir) - val environment = builder.environment() - for ((key, value) <- extraEnvironment) { - environment.put(key, value) - } - val process = builder.start() - new Thread("read stderr for " + command(0)) { - override def run() { - for (line <- Source.fromInputStream(process.getErrorStream).getLines) { - System.err.println(line) - } - } - }.start() - val output = new StringBuffer - val stdoutThread = new Thread("read stdout for " + command(0)) { - override def run() { - for (line <- Source.fromInputStream(process.getInputStream).getLines) { - output.append(line) - } - } - } - stdoutThread.start() - val exitCode = process.waitFor() - stdoutThread.join() // Wait for it to finish reading output - if (exitCode != 0) { - throw new SparkException("Process " + command + " exited with code " + exitCode) - } - output.toString - } - - /** - * A regular expression to match classes of the "core" Spark API that we want to skip when - * finding the call site of a method. - */ - private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r - - private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, - val firstUserLine: Int, val firstUserClass: String) - - /** - * When called inside a class in the spark package, returns the name of the user code class - * (outside the spark package) that called into Spark, as well as which Spark method they called. - * This is used, for example, to tell users where in their code each RDD got created. - */ - def getCallSiteInfo: CallSiteInfo = { - val trace = Thread.currentThread.getStackTrace().filter( el => - (!el.getMethodName.contains("getStackTrace"))) - - // Keep crawling up the stack trace until we find the first function not inside of the spark - // package. We track the last (shallowest) contiguous Spark method. This might be an RDD - // transformation, a SparkContext function (such as parallelize), or anything else that leads - // to instantiation of an RDD. We also track the first (deepest) user method, file, and line. - var lastSparkMethod = "" - var firstUserFile = "" - var firstUserLine = 0 - var finished = false - var firstUserClass = "" - - for (el <- trace) { - if (!finished) { - if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) { - lastSparkMethod = if (el.getMethodName == "") { - // Spark method is a constructor; get its class name - el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) - } else { - el.getMethodName - } - } - else { - firstUserLine = el.getLineNumber - firstUserFile = el.getFileName - firstUserClass = el.getClassName - finished = true - } - } - } - new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) - } - - def formatSparkCallSite = { - val callSiteInfo = getCallSiteInfo - "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, - callSiteInfo.firstUserLine) - } - - /** Return a string containing part of a file from byte 'start' to 'end'. */ - def offsetBytes(path: String, start: Long, end: Long): String = { - val file = new File(path) - val length = file.length() - val effectiveEnd = math.min(length, end) - val effectiveStart = math.max(0, start) - val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) - val stream = new FileInputStream(file) - - stream.skip(effectiveStart) - stream.read(buff) - stream.close() - Source.fromBytes(buff).mkString - } - - /** - * Clone an object using a Spark serializer. - */ - def clone[T](value: T, serializer: SerializerInstance): T = { - serializer.deserialize[T](serializer.serialize(value)) - } - - /** - * Detect whether this thread might be executing a shutdown hook. Will always return true if - * the current thread is a running a shutdown hook but may spuriously return true otherwise (e.g. - * if System.exit was just called by a concurrent thread). - * - * Currently, this detects whether the JVM is shutting down by Runtime#addShutdownHook throwing - * an IllegalStateException. - */ - def inShutdown(): Boolean = { - try { - val hook = new Thread { - override def run() {} - } - Runtime.getRuntime.addShutdownHook(hook) - Runtime.getRuntime.removeShutdownHook(hook) - } catch { - case ise: IllegalStateException => return true - } - return false - } - - def isSpace(c: Char): Boolean = { - " \t\r\n".indexOf(c) != -1 - } - - /** - * Split a string of potentially quoted arguments from the command line the way that a shell - * would do it to determine arguments to a command. For example, if the string is 'a "b c" d', - * then it would be parsed as three arguments: 'a', 'b c' and 'd'. - */ - def splitCommandString(s: String): Seq[String] = { - val buf = new ArrayBuffer[String] - var inWord = false - var inSingleQuote = false - var inDoubleQuote = false - var curWord = new StringBuilder - def endWord() { - buf += curWord.toString - curWord.clear() - } - var i = 0 - while (i < s.length) { - var nextChar = s.charAt(i) - if (inDoubleQuote) { - if (nextChar == '"') { - inDoubleQuote = false - } else if (nextChar == '\\') { - if (i < s.length - 1) { - // Append the next character directly, because only " and \ may be escaped in - // double quotes after the shell's own expansion - curWord.append(s.charAt(i + 1)) - i += 1 - } - } else { - curWord.append(nextChar) - } - } else if (inSingleQuote) { - if (nextChar == '\'') { - inSingleQuote = false - } else { - curWord.append(nextChar) - } - // Backslashes are not treated specially in single quotes - } else if (nextChar == '"') { - inWord = true - inDoubleQuote = true - } else if (nextChar == '\'') { - inWord = true - inSingleQuote = true - } else if (!isSpace(nextChar)) { - curWord.append(nextChar) - inWord = true - } else if (inWord && isSpace(nextChar)) { - endWord() - inWord = false - } - i += 1 - } - if (inWord || inDoubleQuote || inSingleQuote) { - endWord() - } - return buf - } - - /* Calculates 'x' modulo 'mod', takes to consideration sign of x, - * i.e. if 'x' is negative, than 'x' % 'mod' is negative too - * so function return (x % mod) + mod in that case. - */ - def nonNegativeMod(x: Int, mod: Int): Int = { - val rawMod = x % mod - rawMod + (if (rawMod < 0) mod else 0) - } -} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index cb25ff728e..5fd1fab580 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.util.StatCounter diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 09da35aee6..a6518abf45 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -33,12 +33,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.HashPartitioner import org.apache.spark.Partitioner import org.apache.spark.Partitioner._ -import org.apache.spark.RDD import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.api.java.function.{Function2 => JFunction2} import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.PartialResult +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.OrderedRDDFunctions import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 68cfcf5999..eec58abdd6 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.java import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 1ad8514980..7e6e691f11 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -21,13 +21,15 @@ import java.util.{List => JList, Comparator} import scala.Tuple2 import scala.collection.JavaConversions._ +import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.spark.{SparkContext, Partition, RDD, TaskContext} + +import org.apache.spark.{SparkContext, Partition, TaskContext} +import org.apache.spark.rdd.RDD import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import org.apache.spark.partial.{PartialResult, BoundedDouble} import org.apache.spark.storage.StorageLevel -import com.google.common.base.Optional trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 618a7b3bf7..8869e072bf 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -26,13 +26,13 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} +import com.google.common.base.Optional -import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext} +import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, SparkContext} import org.apache.spark.SparkContext.IntAccumulatorParam import org.apache.spark.SparkContext.DoubleAccumulatorParam import org.apache.spark.broadcast.Broadcast - -import com.google.common.base.Optional +import org.apache.spark.rdd.RDD /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index eea63d5a4e..b090c6edf3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -18,8 +18,8 @@ package org.apache.spark.api.python import org.apache.spark.Partitioner -import org.apache.spark.Utils import java.util.Arrays +import org.apache.spark.util.Utils /** * A [[org.apache.spark.Partitioner]] that performs handling of byte arrays, for use by the Python API. diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 621f0fe8ee..ccd3833964 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -26,7 +26,9 @@ import scala.collection.JavaConversions._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PipedRDD +import org.apache.spark.util.Utils private[spark] class PythonRDD[T: ClassManifest]( diff --git a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala index 99e86237fc..93e7815ab5 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala @@ -27,6 +27,7 @@ import scala.math import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 7a52ff0769..9db26ae6de 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -23,10 +23,10 @@ import java.net.URL import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{HttpServer, Logging, SparkEnv, Utils} +import org.apache.spark.{HttpServer, Logging, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{MetadataCleaner, TimeStampedHashSet} +import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashSet} private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) diff --git a/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala index 10b910df87..21ec94659e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala @@ -24,6 +24,7 @@ import java.util.Random import scala.collection.mutable.Map import org.apache.spark._ +import org.apache.spark.util.Utils private object MultiTracker extends Logging { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala index b5a4ccc0ee..80c97ca073 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala @@ -26,6 +26,7 @@ import scala.math import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 4dc6ada2d1..c31619db27 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -19,10 +19,10 @@ package org.apache.spark.deploy import scala.collection.immutable.List -import org.apache.spark.Utils import org.apache.spark.deploy.ExecutorState.ExecutorState import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo} import org.apache.spark.deploy.worker.ExecutorRunner +import org.apache.spark.util.Utils private[deploy] sealed trait DeployMessage extends Serializable diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index af5a4110b0..78e3747ad8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -21,8 +21,8 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.master.Master -import org.apache.spark.util.AkkaUtils -import org.apache.spark.{Logging, Utils} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.{Logging} import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 0322029fbd..d5e9a0e095 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.client -import org.apache.spark.util.AkkaUtils -import org.apache.spark.{Logging, Utils} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.{Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 869b2b2646..7cf0a7754f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -27,12 +27,12 @@ import akka.actor.Terminated import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} import akka.util.duration._ -import org.apache.spark.{Logging, SparkException, Utils} +import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index c86cca278d..9d89b455fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -17,8 +17,7 @@ package org.apache.spark.deploy.master -import org.apache.spark.util.IntParam -import org.apache.spark.Utils +import org.apache.spark.util.{Utils, IntParam} /** * Command-line parser for the master. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 285e07a823..6219f11f2a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import akka.actor.ActorRef import scala.collection.mutable -import org.apache.spark.Utils +import org.apache.spark.util.Utils private[spark] class WorkerInfo( val id: String, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 6435c7f917..f4e574d15d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMaste import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.UIUtils -import org.apache.spark.Utils +import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 58d3863009..d7a57229b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -27,12 +27,12 @@ import akka.util.duration._ import net.liftweb.json.JsonAST.JValue -import org.apache.spark.Utils import org.apache.spark.deploy.DeployWebUI import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo} import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 47b1e521f5..f4df729e87 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -23,10 +23,11 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging, Utils} +import org.apache.spark.{Logging} import org.apache.spark.deploy.master.Master import org.apache.spark.ui.JettyUtils import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils /** * Web UI server for the standalone master. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 01ce4a6dea..e3dc30eefc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -25,9 +25,10 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.{Utils, Logging} +import org.apache.spark.{Logging} import org.apache.spark.deploy.{ExecutorState, ApplicationDescription} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.util.Utils /** * Manages the execution of one executor process. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 86e8e7543b..09530beb3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -27,13 +27,13 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import akka.util.duration._ -import org.apache.spark.{Logging, Utils} +import org.apache.spark.{Logging} import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} private[spark] class Worker( diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 6d91223413..0ae89a864f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -17,9 +17,7 @@ package org.apache.spark.deploy.worker -import org.apache.spark.util.IntParam -import org.apache.spark.util.MemoryParam -import org.apache.spark.Utils +import org.apache.spark.util.{Utils, IntParam, MemoryParam} import java.lang.management.ManagementFactory /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 6192c2324b..d2d3617498 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -27,11 +27,11 @@ import akka.util.duration._ import net.liftweb.json.JsonAST.JValue -import org.apache.spark.Utils import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.worker.ExecutorRunner import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils private[spark] class IndexPage(parent: WorkerWebUI) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index bb8165ac09..95d6007f3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -26,10 +26,11 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.{Utils, Logging} +import org.apache.spark.{Logging} import org.apache.spark.ui.JettyUtils import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils /** * Web UI server for the standalone worker. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5446a3fca9..d365804994 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -27,6 +27,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.scheduler._ import org.apache.spark._ +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 410a94df6b..da62091980 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -22,8 +22,9 @@ import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNa import org.apache.mesos.Protos.{TaskState => MesosTaskState, TaskStatus => MesosTaskStatus, _} import org.apache.spark.TaskState.TaskState import com.google.protobuf.ByteString -import org.apache.spark.{Utils, Logging} +import org.apache.spark.{Logging} import org.apache.spark.TaskState +import org.apache.spark.util.Utils private[spark] class MesosExecutorBackend extends MesosExecutor diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index 65801f75b7..7839023868 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -22,10 +22,10 @@ import java.nio.ByteBuffer import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import org.apache.spark.{Logging, Utils, SparkEnv} +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} private[spark] class StandaloneExecutorBackend( diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 9e2233c07b..e15a839c4e 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -34,6 +34,7 @@ import scala.collection.mutable.ArrayBuffer import akka.dispatch.{Await, Promise, ExecutionContext, Future} import akka.util.Duration import akka.util.duration._ +import org.apache.spark.util.Utils private[spark] class ConnectionManager(port: Int) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala index 0839c011b8..50dd9bc2d1 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala @@ -19,7 +19,7 @@ package org.apache.spark.network import java.net.InetSocketAddress -import org.apache.spark.Utils +import org.apache.spark.util.Utils private[spark] case class ConnectionManagerId(host: String, port: Int) { diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 1126480689..c0ec527339 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -1,3 +1,5 @@ +import org.apache.spark.rdd.{SequenceFileRDDFunctions, DoubleRDDFunctions, PairRDDFunctions} + /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,16 +18,17 @@ */ /** - * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to Spark, while - * [[org.apache.spark.RDD]] is the data type representing a distributed collection, and provides most - * parallel operations. + * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to + * Spark, while [[org.apache.spark.rdd.RDD]] is the data type representing a distributed collection, + * and provides most parallel operations. * - * In addition, [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value - * pairs, such as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations - * available only on RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations - * available on RDDs that can be saved as SequenceFiles. These operations are automatically - * available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit conversions when - * you `import org.apache.spark.SparkContext._`. + * In addition, [[org.apache.spark.rdd.PairRDDFunctions]] contains operations available only on RDDs + * of key-value pairs, such as `groupByKey` and `join`; [[org.apache.spark.rdd.DoubleRDDFunctions]] + * contains operations available only on RDDs of Doubles; and + * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can + * be saved as SequenceFiles. These operations are automatically available on any RDD of the right + * type (e.g. RDD[(Int, Int)] through implicit conversions when you + * `import org.apache.spark.SparkContext._`. */ package object spark { // For package docs only diff --git a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala index c5d51bee50..d71069444a 100644 --- a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.partial import org.apache.spark._ import org.apache.spark.scheduler.JobListener +import org.apache.spark.rdd.RDD /** * A JobListener for an approximate single-result action, such as count() or non-parallel reduce(). diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 4bb01efa86..bca6956a18 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.BlockManager private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index dcc35e8d0e..0187256a8e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext} +import org.apache.spark.{Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala new file mode 100644 index 0000000000..a4bec41752 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -0,0 +1,79 @@ +/* + * 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.rdd + +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.MeanEvaluator +import org.apache.spark.partial.PartialResult +import org.apache.spark.partial.SumEvaluator +import org.apache.spark.util.StatCounter +import org.apache.spark.{TaskContext, Logging} + +/** + * Extra functions available on RDDs of Doubles through an implicit conversion. + * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. + */ +class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { + /** Add up the elements in this RDD. */ + def sum(): Double = { + self.reduce(_ + _) + } + + /** + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count + * of the RDD's elements in one operation. + */ + def stats(): StatCounter = { + self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b)) + } + + /** Compute the mean of this RDD's elements. */ + def mean(): Double = stats().mean + + /** Compute the variance of this RDD's elements. */ + def variance(): Double = stats().variance + + /** Compute the standard deviation of this RDD's elements. */ + def stdev(): Double = stats().stdev + + /** + * Compute the sample standard deviation of this RDD's elements (which corrects for bias in + * estimating the standard deviation by dividing by N-1 instead of N). + */ + def sampleStdev(): Double = stats().sampleStdev + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = stats().sampleVariance + + /** (Experimental) Approximate operation to return the mean within a timeout. */ + def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { + val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) + val evaluator = new MeanEvaluator(self.partitions.size, confidence) + self.context.runApproximateJob(self, processPartition, evaluator, timeout) + } + + /** (Experimental) Approximate operation to return the sum within a timeout. */ + def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { + val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) + val evaluator = new SumEvaluator(self.partitions.size, confidence) + self.context.runApproximateJob(self, processPartition, evaluator, timeout) + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index 24ce4abbc4..c8900d1a93 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} /** diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala index 4df8ceb58b..5312dc0b59 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{OneToOneDependency, RDD, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, Partition, TaskContext} private[spark] class FilteredRDD[T: ClassManifest]( prev: RDD[T], diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala index 2bf7653af1..cbdf6d84c0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala index e544720b05..82000bac09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{TaskContext, Partition, RDD} +import org.apache.spark.{TaskContext, Partition} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala index 2ce94199f2..829545d7b0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev) { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 08e6154bb9..2cb6734e41 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -18,21 +18,15 @@ package org.apache.spark.rdd import java.io.EOFException -import java.util.NoSuchElementException -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.FileInputFormat import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils -import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} +import org.apache.spark.{Logging, Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext} import org.apache.spark.util.NextIterator import org.apache.hadoop.conf.{Configuration, Configurable} diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 3db460b3ce..aca0146884 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.sql.{Connection, ResultSet} -import org.apache.spark.{Logging, Partition, RDD, SparkContext, TaskContext} +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 13009d3e17..203179c4ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala index 1683050b86..3ed8339010 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} /** diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala index 26d4806edb..e8be1c4816 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala index a405e9acdd..d33c1af581 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.rdd -import org.apache.spark.{TaskContext, Partition, RDD} +import org.apache.spark.{TaskContext, Partition} private[spark] class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 114b504486..7b3a89f7e0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.{Dependency, Logging, Partition, SerializableWritable, SparkContext, TaskContext} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 4c3df0eaf4..697be8b997 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -17,12 +17,13 @@ package org.apache.spark.rdd -import org.apache.spark.{RangePartitioner, Logging, RDD} +import org.apache.spark.{RangePartitioner, Logging} /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through - * an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these - * functions. They will work with any key type that has a `scala.math.Ordered` implementation. + * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to + * use these functions. They will work with any key type that has a `scala.math.Ordered` + * implementation. */ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest, diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala new file mode 100644 index 0000000000..a47c512275 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -0,0 +1,702 @@ +/* + * 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.rdd + +import java.nio.ByteBuffer +import java.util.Date +import java.text.SimpleDateFormat +import java.util.{HashMap => JHashMap} + +import scala.collection.{mutable, Map} +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.hadoop.mapred._ +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.mapred.FileOutputFormat +import org.apache.hadoop.mapred.OutputFormat +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} +import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil +import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob} +import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter} + +import org.apache.spark._ +import org.apache.spark.SparkContext._ +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.Aggregator +import org.apache.spark.Partitioner +import org.apache.spark.Partitioner.defaultPartitioner + +/** + * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. + * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. + */ +class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) + extends Logging + with SparkHadoopMapReduceUtil + with Serializable { + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + * + * In addition, users can control the partitioning of the output RDD, and whether to perform + * map-side aggregation (if a mapper can produce multiple items with the same key). + */ + def combineByKey[C](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { + if (getKeyClass().isArray) { + if (mapSideCombine) { + throw new SparkException("Cannot use map-side combining with array keys.") + } + if (partitioner.isInstanceOf[HashPartitioner]) { + throw new SparkException("Default partitioner cannot partition array keys.") + } + } + val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) + if (self.partitioner == Some(partitioner)) { + self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) + } else if (mapSideCombine) { + val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) + val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) + .setSerializer(serializerClass) + partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = true) + } else { + // Don't apply map-side combiner. + // A sanity check to make sure mergeCombiners is not defined. + assert(mergeCombiners == null) + val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass) + values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) + } + } + + /** + * Simplified version of combineByKey that hash-partitions the output RDD. + */ + def combineByKey[C](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + numPartitions: Int): RDD[(K, C)] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { + // Serialize the zero value to a byte array so that we can get a new clone of it on each key + val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + // When deserializing, use a lazy val to create just one instance of the serializer per task + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) + + combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { + foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { + foldByKey(zeroValue, defaultPartitioner(self))(func) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. + */ + def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = { + combineByKey[V]((v: V) => v, func, func, partitioner) + } + + /** + * Merge the values for each key using an associative reduce function, but return the results + * immediately to the master as a Map. This will also perform the merging locally on each mapper + * before sending results to a reducer, similarly to a "combiner" in MapReduce. + */ + def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { + + if (getKeyClass().isArray) { + throw new SparkException("reduceByKeyLocally() does not support array keys") + } + + def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { + val map = new JHashMap[K, V] + iter.foreach { case (k, v) => + val old = map.get(k) + map.put(k, if (old == null) v else func(old, v)) + } + Iterator(map) + } + + def mergeMaps(m1: JHashMap[K, V], m2: JHashMap[K, V]): JHashMap[K, V] = { + m2.foreach { case (k, v) => + val old = m1.get(k) + m1.put(k, if (old == null) v else func(old, v)) + } + m1 + } + + self.mapPartitions(reducePartition).reduce(mergeMaps) + } + + /** Alias for reduceByKeyLocally */ + def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func) + + /** Count the number of elements for each key, and return the result to the master as a Map. */ + def countByKey(): Map[K, Long] = self.map(_._1).countByValue() + + /** + * (Experimental) Approximate version of countByKey that can return a partial result if it does + * not finish within a timeout. + */ + def countByKeyApprox(timeout: Long, confidence: Double = 0.95) + : PartialResult[Map[K, BoundedDouble]] = { + self.map(_._1).countByValueApprox(timeout, confidence) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + */ + def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { + reduceByKey(new HashPartitioner(numPartitions), func) + } + + /** + * Group the values for each key in the RDD into a single sequence. Allows controlling the + * partitioning of the resulting key-value pair RDD by passing a Partitioner. + */ + def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + // groupByKey shouldn't use map side combine because map side combine does not + // reduce the amount of data shuffled and requires all map side data be inserted + // into a hash table, leading to more objects in the old gen. + def createCombiner(v: V) = ArrayBuffer(v) + def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v + val bufs = combineByKey[ArrayBuffer[V]]( + createCombiner _, mergeValue _, null, partitioner, mapSideCombine=false) + bufs.asInstanceOf[RDD[(K, Seq[V])]] + } + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with into `numPartitions` partitions. + */ + def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + groupByKey(new HashPartitioner(numPartitions)) + } + + /** + * Return a copy of the RDD partitioned using the specified partitioner. + */ + def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { + if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { + throw new SparkException("Default partitioner cannot partition array keys.") + } + new ShuffledRDD[K, V, (K, V)](self, partitioner) + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + */ + def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + if (ws.isEmpty) { + vs.iterator.map(v => (v, None)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + } + } + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Option[V], W))] = { + this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + if (vs.isEmpty) { + ws.iterator.map(w => (None, w)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + } + } + } + + /** + * Simplified version of combineByKey that hash-partitions the resulting RDD using the + * existing partitioner/parallelism level. + */ + def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + : RDD[(K, C)] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. + */ + def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { + reduceByKey(defaultPartitioner(self), func) + } + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + def groupByKey(): RDD[(K, Seq[V])] = { + groupByKey(defaultPartitioner(self)) + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + join(other, defaultPartitioner(self, other)) + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + join(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. + */ + def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + leftOuterJoin(other, defaultPartitioner(self, other)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * into `numPartitions` partitions. + */ + def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + leftOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. + */ + def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + rightOuterJoin(other, defaultPartitioner(self, other)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD into the given number of partitions. + */ + def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + rightOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** + * Return the key-value pairs in this RDD to the master as a Map. + */ + def collectAsMap(): Map[K, V] = { + val data = self.toArray() + val map = new mutable.HashMap[K, V] + map.sizeHint(data.length) + data.foreach { case (k, v) => map.put(k, v) } + map + } + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValues[U](f: V => U): RDD[(K, U)] = { + val cleanF = self.context.clean(f) + new MappedValuesRDD(self, cleanF) + } + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { + val cleanF = self.context.clean(f) + new FlatMappedValuesRDD(self, cleanF) + } + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + throw new SparkException("Default partitioner cannot partition array keys.") + } + val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) + val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) + prfs.mapValues { case Seq(vs, ws) => + (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + } + } + + /** + * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this`, `other1` and `other2`. + */ + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) + : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + throw new SparkException("Default partitioner cannot partition array keys.") + } + val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) + val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) + prfs.mapValues { case Seq(vs, w1s, w2s) => + (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + } + } + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + cogroup(other, defaultPartitioner(self, other)) + } + + /** + * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this`, `other1` and `other2`. + */ + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + cogroup(other1, other2, defaultPartitioner(self, other1, other2)) + } + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + cogroup(other, new HashPartitioner(numPartitions)) + } + + /** + * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + * tuple with the list of values for that key in `this`, `other1` and `other2`. + */ + def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + cogroup(other1, other2, new HashPartitioner(numPartitions)) + } + + /** Alias for cogroup. */ + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + cogroup(other, defaultPartitioner(self, other)) + } + + /** Alias for cogroup. */ + def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + cogroup(other1, other2, defaultPartitioner(self, other1, other2)) + } + + /** + * Return an RDD with the pairs from `this` whose keys are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] = + subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size))) + + /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ + def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] = + subtractByKey(other, new HashPartitioner(numPartitions)) + + /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ + def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] = + new SubtractedRDD[K, V, W](self, other, p) + + /** + * Return the list of values in the RDD for key `key`. This operation is done efficiently if the + * RDD has a known partitioner by only searching the partition that the key maps to. + */ + def lookup(key: K): Seq[V] = { + self.partitioner match { + case Some(p) => + val index = p.getPartition(key) + def process(it: Iterator[(K, V)]): Seq[V] = { + val buf = new ArrayBuffer[V] + for ((k, v) <- it if k == key) { + buf += v + } + buf + } + val res = self.context.runJob(self, process _, Array(index), false) + res(0) + case None => + self.filter(_._1 == key).map(_._2).collect() + } + } + + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. + */ + def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + } + + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. Compress the result with the + * supplied codec. + */ + def saveAsHadoopFile[F <: OutputFormat[K, V]]( + path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) { + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) + } + + /** + * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` + * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. + */ + def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) { + saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) + } + + /** + * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` + * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. + */ + def saveAsNewAPIHadoopFile( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[_ <: NewOutputFormat[_, _]], + conf: Configuration = self.context.hadoopConfiguration) { + val job = new NewAPIHadoopJob(conf) + job.setOutputKeyClass(keyClass) + job.setOutputValueClass(valueClass) + val wrappedConf = new SerializableWritable(job.getConfiguration) + NewFileOutputFormat.setOutputPath(job, new Path(path)) + val formatter = new SimpleDateFormat("yyyyMMddHHmm") + val jobtrackerID = formatter.format(new Date()) + val stageId = self.id + def writeShard(context: TaskContext, iter: Iterator[(K,V)]): Int = { + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val attemptNumber = (context.attemptId % Int.MaxValue).toInt + /* "reduce task" */ + val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.splitId, attemptNumber) + val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) + val format = outputFormatClass.newInstance + val committer = format.getOutputCommitter(hadoopContext) + committer.setupTask(hadoopContext) + val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] + while (iter.hasNext) { + val (k, v) = iter.next() + writer.write(k, v) + } + writer.close(hadoopContext) + committer.commitTask(hadoopContext) + return 1 + } + val jobFormat = outputFormatClass.newInstance + /* apparently we need a TaskAttemptID to construct an OutputCommitter; + * however we're only going to use this local OutputCommitter for + * setupJob/commitJob, so we just use a dummy "map" task. + */ + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0) + val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) + val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) + jobCommitter.setupJob(jobTaskContext) + val count = self.context.runJob(self, writeShard _).sum + jobCommitter.commitJob(jobTaskContext) + jobCommitter.cleanupJob(jobTaskContext) + } + + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. Compress with the supplied codec. + */ + def saveAsHadoopFile( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[_ <: OutputFormat[_, _]], + codec: Class[_ <: CompressionCodec]) { + saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, + new JobConf(self.context.hadoopConfiguration), Some(codec)) + } + + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. + */ + def saveAsHadoopFile( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[_ <: OutputFormat[_, _]], + conf: JobConf = new JobConf(self.context.hadoopConfiguration), + codec: Option[Class[_ <: CompressionCodec]] = None) { + conf.setOutputKeyClass(keyClass) + conf.setOutputValueClass(valueClass) + // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug + conf.set("mapred.output.format.class", outputFormatClass.getName) + for (c <- codec) { + conf.setCompressMapOutput(true) + conf.set("mapred.output.compress", "true") + conf.setMapOutputCompressorClass(c) + conf.set("mapred.output.compression.codec", c.getCanonicalName) + conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + } + conf.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath(conf, SparkHadoopWriter.createPathFromString(path, conf)) + saveAsHadoopDataset(conf) + } + + /** + * Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for + * that storage system. The JobConf should set an OutputFormat and any output paths required + * (e.g. a table name to write to) in the same way as it would be configured for a Hadoop + * MapReduce job. + */ + def saveAsHadoopDataset(conf: JobConf) { + val outputFormatClass = conf.getOutputFormat + val keyClass = conf.getOutputKeyClass + val valueClass = conf.getOutputValueClass + if (outputFormatClass == null) { + throw new SparkException("Output format class not set") + } + if (keyClass == null) { + throw new SparkException("Output key class not set") + } + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + + logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")") + + val writer = new SparkHadoopWriter(conf) + writer.preSetup() + + def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val attemptNumber = (context.attemptId % Int.MaxValue).toInt + + writer.setup(context.stageId, context.splitId, attemptNumber) + writer.open() + + var count = 0 + while(iter.hasNext) { + val record = iter.next() + count += 1 + writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) + } + + writer.close() + writer.commit() + } + + self.context.runJob(self, writeToFile _) + writer.commitJob() + writer.cleanup() + } + + /** + * Return an RDD with the keys of each tuple. + */ + def keys: RDD[K] = self.map(_._1) + + /** + * Return an RDD with the values of each tuple. + */ + def values: RDD[V] = self.map(_._2) + + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure + + private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure +} + +private[spark] object Manifests { + val seqSeqManifest = classManifest[Seq[Seq[_]]] +} diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 8db3611054..6dbd4309aa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -23,6 +23,8 @@ import scala.collection.Map import org.apache.spark._ import java.io._ import scala.Serializable +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.Utils private[spark] class ParallelCollectionPartition[T: ClassManifest]( var rddId: Long, diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index 8e79a5c874..165cd412fc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} +import org.apache.spark.{NarrowDependency, SparkEnv, Partition, TaskContext} class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 98498d5ddf..d5304ab0ae 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source -import org.apache.spark.{RDD, SparkEnv, Partition, TaskContext} +import org.apache.spark.{SparkEnv, Partition, TaskContext} import org.apache.spark.broadcast.Broadcast diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala new file mode 100644 index 0000000000..e143ecd096 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -0,0 +1,942 @@ +/* + * 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.rdd + +import java.util.Random + +import scala.collection.Map +import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.io.BytesWritable +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.Text +import org.apache.hadoop.mapred.TextOutputFormat + +import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} + +import org.apache.spark.Partitioner._ +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.CountEvaluator +import org.apache.spark.partial.GroupedCountEvaluator +import org.apache.spark.partial.PartialResult +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.{Utils, BoundedPriorityQueue} + +import org.apache.spark.SparkContext._ +import org.apache.spark._ + +/** + * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, + * partitioned collection of elements that can be operated on in parallel. This class contains the + * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition, + * [[org.apache.spark.rdd.PairRDDFunctions]] contains operations available only on RDDs of key-value + * pairs, such as `groupByKey` and `join`; [[org.apache.spark.rdd.DoubleRDDFunctions]] contains + * operations available only on RDDs of Doubles; and [[org.apache.spark.rdd.SequenceFileRDDFunctions]] + * contains operations available on RDDs that can be saved as SequenceFiles. These operations are + * automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit + * conversions when you `import org.apache.spark.SparkContext._`. + * + * Internally, each RDD is characterized by five main properties: + * + * - A list of partitions + * - A function for computing each split + * - A list of dependencies on other RDDs + * - Optionally, a Partitioner for key-value RDDs (e.g. to say that the RDD is hash-partitioned) + * - Optionally, a list of preferred locations to compute each split on (e.g. block locations for + * an HDFS file) + * + * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD + * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for + * reading data from a new storage system) by overriding these functions. Please refer to the + * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details + * on RDD internals. + */ +abstract class RDD[T: ClassManifest]( + @transient private var sc: SparkContext, + @transient private var deps: Seq[Dependency[_]] + ) extends Serializable with Logging { + + /** Construct an RDD with just a one-to-one dependency on one parent */ + def this(@transient oneParent: RDD[_]) = + this(oneParent.context , List(new OneToOneDependency(oneParent))) + + // ======================================================================= + // Methods that should be implemented by subclasses of RDD + // ======================================================================= + + /** Implemented by subclasses to compute a given partition. */ + def compute(split: Partition, context: TaskContext): Iterator[T] + + /** + * Implemented by subclasses to return the set of partitions in this RDD. This method will only + * be called once, so it is safe to implement a time-consuming computation in it. + */ + protected def getPartitions: Array[Partition] + + /** + * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only + * be called once, so it is safe to implement a time-consuming computation in it. + */ + protected def getDependencies: Seq[Dependency[_]] = deps + + /** Optionally overridden by subclasses to specify placement preferences. */ + protected def getPreferredLocations(split: Partition): Seq[String] = Nil + + /** Optionally overridden by subclasses to specify how they are partitioned. */ + val partitioner: Option[Partitioner] = None + + // ======================================================================= + // Methods and fields available on all RDDs + // ======================================================================= + + /** The SparkContext that created this RDD. */ + def sparkContext: SparkContext = sc + + /** A unique ID for this RDD (within its SparkContext). */ + val id: Int = sc.newRddId() + + /** A friendly name for this RDD */ + var name: String = null + + /** Assign a name to this RDD */ + def setName(_name: String) = { + name = _name + this + } + + /** User-defined generator of this RDD*/ + var generator = Utils.getCallSiteInfo.firstUserClass + + /** Reset generator*/ + def setGenerator(_generator: String) = { + generator = _generator + } + + /** + * Set this RDD's storage level to persist its values across operations after the first time + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. + */ + def persist(newLevel: StorageLevel): RDD[T] = { + // TODO: Handle changes of StorageLevel + if (storageLevel != StorageLevel.NONE && newLevel != storageLevel) { + throw new UnsupportedOperationException( + "Cannot change storage level of an RDD after it was already assigned a level") + } + storageLevel = newLevel + // Register the RDD with the SparkContext + sc.persistentRdds(id) = this + this + } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + def cache(): RDD[T] = persist() + + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + * + * @param blocking Whether to block until all blocks are deleted. + * @return This RDD. + */ + def unpersist(blocking: Boolean = true): RDD[T] = { + logInfo("Removing RDD " + id + " from persistence list") + sc.env.blockManager.master.removeRdd(id, blocking) + sc.persistentRdds.remove(id) + storageLevel = StorageLevel.NONE + this + } + + /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ + def getStorageLevel = storageLevel + + // Our dependencies and partitions will be gotten by calling subclass's methods below, and will + // be overwritten when we're checkpointed + private var dependencies_ : Seq[Dependency[_]] = null + @transient private var partitions_ : Array[Partition] = null + + /** An Option holding our checkpoint RDD, if we are checkpointed */ + private def checkpointRDD: Option[RDD[T]] = checkpointData.flatMap(_.checkpointRDD) + + /** + * Get the list of dependencies of this RDD, taking into account whether the + * RDD is checkpointed or not. + */ + final def dependencies: Seq[Dependency[_]] = { + checkpointRDD.map(r => List(new OneToOneDependency(r))).getOrElse { + if (dependencies_ == null) { + dependencies_ = getDependencies + } + dependencies_ + } + } + + /** + * Get the array of partitions of this RDD, taking into account whether the + * RDD is checkpointed or not. + */ + final def partitions: Array[Partition] = { + checkpointRDD.map(_.partitions).getOrElse { + if (partitions_ == null) { + partitions_ = getPartitions + } + partitions_ + } + } + + /** + * Get the preferred locations of a partition (as hostnames), taking into account whether the + * RDD is checkpointed. + */ + final def preferredLocations(split: Partition): Seq[String] = { + checkpointRDD.map(_.getPreferredLocations(split)).getOrElse { + getPreferredLocations(split) + } + } + + /** + * Internal method to this RDD; will read from cache if applicable, or otherwise compute it. + * This should ''not'' be called by users directly, but is available for implementors of custom + * subclasses of RDD. + */ + final def iterator(split: Partition, context: TaskContext): Iterator[T] = { + if (storageLevel != StorageLevel.NONE) { + SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) + } else { + computeOrReadCheckpoint(split, context) + } + } + + /** + * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. + */ + private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = { + if (isCheckpointed) { + firstParent[T].iterator(split, context) + } else { + compute(split, context) + } + } + + // Transformations (return a new RDD) + + /** + * Return a new RDD by applying a function to all elements of this RDD. + */ + def map[U: ClassManifest](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) + + /** + * Return a new RDD by first applying a function to all elements of this + * RDD, and then flattening the results. + */ + def flatMap[U: ClassManifest](f: T => TraversableOnce[U]): RDD[U] = + new FlatMappedRDD(this, sc.clean(f)) + + /** + * Return a new RDD containing only the elements that satisfy a predicate. + */ + def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f)) + + /** + * Return a new RDD containing the distinct elements in this RDD. + */ + def distinct(numPartitions: Int): RDD[T] = + map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) + + def distinct(): RDD[T] = distinct(partitions.size) + + /** + * Return a new RDD that is reduced into `numPartitions` partitions. + */ + def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { + if (shuffle) { + // include a shuffle step so that our upstream tasks are still distributed + new CoalescedRDD( + new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), + new HashPartitioner(numPartitions)), + numPartitions).keys + } else { + new CoalescedRDD(this, numPartitions) + } + } + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = + new SampledRDD(this, withReplacement, fraction, seed) + + def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { + var fraction = 0.0 + var total = 0 + val multiplier = 3.0 + val initialCount = this.count() + var maxSelected = 0 + + if (num < 0) { + throw new IllegalArgumentException("Negative number of elements requested") + } + + if (initialCount > Integer.MAX_VALUE - 1) { + maxSelected = Integer.MAX_VALUE - 1 + } else { + maxSelected = initialCount.toInt + } + + if (num > initialCount && !withReplacement) { + total = maxSelected + fraction = multiplier * (maxSelected + 1) / initialCount + } else { + fraction = multiplier * (num + 1) / initialCount + total = num + } + + val rand = new Random(seed) + var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + + // If the first sample didn't turn out large enough, keep trying to take samples; + // this shouldn't happen often because we use a big multiplier for thei initial size + while (samples.length < total) { + samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + } + + Utils.randomizeInPlace(samples, rand).take(total) + } + + /** + * Return the union of this RDD and another one. Any identical elements will appear multiple + * times (use `.distinct()` to eliminate them). + */ + def union(other: RDD[T]): RDD[T] = new UnionRDD(sc, Array(this, other)) + + /** + * Return the union of this RDD and another one. Any identical elements will appear multiple + * times (use `.distinct()` to eliminate them). + */ + def ++(other: RDD[T]): RDD[T] = this.union(other) + + /** + * Return an RDD created by coalescing all elements within each partition into an array. + */ + def glom(): RDD[Array[T]] = new GlommedRDD(this) + + /** + * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of + * elements (a, b) where a is in `this` and b is in `other`. + */ + def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) + + /** + * Return an RDD of grouped items. + */ + def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = + groupBy[K](f, defaultPartitioner(this)) + + /** + * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements + * mapping to that key. + */ + def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + groupBy(f, new HashPartitioner(numPartitions)) + + /** + * Return an RDD of grouped items. + */ + def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + val cleanF = sc.clean(f) + this.map(t => (cleanF(t), t)).groupByKey(p) + } + + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: String): RDD[String] = new PipedRDD(this, command) + + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: String, env: Map[String, String]): RDD[String] = + new PipedRDD(this, command, env) + + + /** + * Return an RDD created by piping elements to a forked external process. + * The print behavior can be customized by providing two functions. + * + * @param command command to run in forked process. + * @param env environment variables to set. + * @param printPipeContext Before piping elements, this function is called as an oppotunity + * to pipe context data. Print line function (like out.println) will be + * passed as printPipeContext's parameter. + * @param printRDDElement Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the + * print line function (like out.println()) as the 2nd parameter. + * An example of pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the elements: + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * for (e <- record._2){f(e)} + * @return the result RDD + */ + def pipe( + command: Seq[String], + env: Map[String, String] = Map(), + printPipeContext: (String => Unit) => Unit = null, + printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = + new PipedRDD(this, command, env, + if (printPipeContext ne null) sc.clean(printPipeContext) else null, + if (printRDDElement ne null) sc.clean(printRDDElement) else null) + + /** + * Return a new RDD by applying a function to each partition of this RDD. + */ + def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = + new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) + + /** + * Return a new RDD by applying a function to each partition of this RDD, while tracking the index + * of the original partition. + */ + def mapPartitionsWithIndex[U: ClassManifest]( + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = + new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) + + /** + * Return a new RDD by applying a function to each partition of this RDD, while tracking the index + * of the original partition. + */ + @deprecated("use mapPartitionsWithIndex", "0.7.0") + def mapPartitionsWithSplit[U: ClassManifest]( + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = + new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning) + + /** + * Maps f over this RDD, where f takes an additional parameter of type A. This + * additional parameter is produced by constructA, which is called in each + * partition with the index of that partition. + */ + def mapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) + (f:(T, A) => U): RDD[U] = { + def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { + val a = constructA(index) + iter.map(t => f(t, a)) + } + new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning) + } + + /** + * FlatMaps f over this RDD, where f takes an additional parameter of type A. This + * additional parameter is produced by constructA, which is called in each + * partition with the index of that partition. + */ + def flatMapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false) + (f:(T, A) => Seq[U]): RDD[U] = { + def iterF(index: Int, iter: Iterator[T]): Iterator[U] = { + val a = constructA(index) + iter.flatMap(t => f(t, a)) + } + new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning) + } + + /** + * Applies f to each element of this RDD, where f takes an additional parameter of type A. + * This additional parameter is produced by constructA, which is called in each + * partition with the index of that partition. + */ + def foreachWith[A: ClassManifest](constructA: Int => A) + (f:(T, A) => Unit) { + def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { + val a = constructA(index) + iter.map(t => {f(t, a); t}) + } + (new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true)).foreach(_ => {}) + } + + /** + * Filters this RDD with p, where p takes an additional parameter of type A. This + * additional parameter is produced by constructA, which is called in each + * partition with the index of that partition. + */ + def filterWith[A: ClassManifest](constructA: Int => A) + (p:(T, A) => Boolean): RDD[T] = { + def iterF(index: Int, iter: Iterator[T]): Iterator[T] = { + val a = constructA(index) + iter.filter(t => p(t, a)) + } + new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true) + } + + /** + * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, + * second element in each RDD, etc. Assumes that the two RDDs have the *same number of + * partitions* and the *same number of elements in each partition* (e.g. one was made through + * a map on the other). + */ + def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) + + /** + * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by + * applying a function to the zipped partitions. Assumes that all the RDDs have the + * *same number of partitions*, but does *not* require them to have the same number + * of elements in each partition. + */ + def zipPartitions[B: ClassManifest, V: ClassManifest] + (rdd2: RDD[B]) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) + + def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], rdd3: RDD[C]) + (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) + + def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D]) + (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = + new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) + + + // Actions (launch a job to return a value to the user program) + + /** + * Applies a function f to all elements of this RDD. + */ + def foreach(f: T => Unit) { + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) + } + + /** + * Applies a function f to each partition of this RDD. + */ + def foreachPartition(f: Iterator[T] => Unit) { + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) + } + + /** + * Return an array that contains all of the elements in this RDD. + */ + def collect(): Array[T] = { + val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray) + Array.concat(results: _*) + } + + /** + * Return an array that contains all of the elements in this RDD. + */ + def toArray(): Array[T] = collect() + + /** + * Return an RDD that contains all matching values by applying `f`. + */ + def collect[U: ClassManifest](f: PartialFunction[T, U]): RDD[U] = { + filter(f.isDefinedAt).map(f) + } + + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: RDD[T]): RDD[T] = + subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size))) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: RDD[T], numPartitions: Int): RDD[T] = + subtract(other, new HashPartitioner(numPartitions)) + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: RDD[T], p: Partitioner): RDD[T] = { + if (partitioner == Some(p)) { + // Our partitioner knows how to handle T (which, since we have a partitioner, is + // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples + val p2 = new Partitioner() { + override def numPartitions = p.numPartitions + override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1) + } + // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies + // anyway, and when calling .keys, will not have a partitioner set, even though + // the SubtractedRDD will, thanks to p2's de-tupled partitioning, already be + // partitioned by the right/real keys (e.g. p). + this.map(x => (x, null)).subtractByKey(other.map((_, null)), p2).keys + } else { + this.map(x => (x, null)).subtractByKey(other.map((_, null)), p).keys + } + } + + /** + * Reduces the elements of this RDD using the specified commutative and associative binary operator. + */ + def reduce(f: (T, T) => T): T = { + val cleanF = sc.clean(f) + val reducePartition: Iterator[T] => Option[T] = iter => { + if (iter.hasNext) { + Some(iter.reduceLeft(cleanF)) + } else { + None + } + } + var jobResult: Option[T] = None + val mergeResult = (index: Int, taskResult: Option[T]) => { + if (taskResult != None) { + jobResult = jobResult match { + case Some(value) => Some(f(value, taskResult.get)) + case None => taskResult + } + } + } + sc.runJob(this, reducePartition, mergeResult) + // Get the final result out of our Option, or throw an exception if the RDD was empty + jobResult.getOrElse(throw new UnsupportedOperationException("empty collection")) + } + + /** + * Aggregate the elements of each partition, and then the results for all the partitions, using a + * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to + * modify t1 and return it as its result value to avoid object allocation; however, it should not + * modify t2. + */ + def fold(zeroValue: T)(op: (T, T) => T): T = { + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + val cleanOp = sc.clean(op) + val foldPartition = (iter: Iterator[T]) => iter.fold(zeroValue)(cleanOp) + val mergeResult = (index: Int, taskResult: T) => jobResult = op(jobResult, taskResult) + sc.runJob(this, foldPartition, mergeResult) + jobResult + } + + /** + * Aggregate the elements of each partition, and then the results for all the partitions, using + * given combine functions and a neutral "zero value". This function can return a different result + * type, U, than the type of this RDD, T. Thus, we need one operation for merging a T into an U + * and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are + * allowed to modify and return their first argument instead of creating a new U to avoid memory + * allocation. + */ + def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { + // Clone the zero value since we will also be serializing it as part of tasks + var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + val cleanSeqOp = sc.clean(seqOp) + val cleanCombOp = sc.clean(combOp) + val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) + val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult) + sc.runJob(this, aggregatePartition, mergeResult) + jobResult + } + + /** + * Return the number of elements in the RDD. + */ + def count(): Long = { + sc.runJob(this, (iter: Iterator[T]) => { + var result = 0L + while (iter.hasNext) { + result += 1L + iter.next() + } + result + }).sum + } + + /** + * (Experimental) Approximate version of count() that returns a potentially incomplete result + * within a timeout, even if not all tasks have finished. + */ + def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { + val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => + var result = 0L + while (iter.hasNext) { + result += 1L + iter.next() + } + result + } + val evaluator = new CountEvaluator(partitions.size, confidence) + sc.runApproximateJob(this, countElements, evaluator, timeout) + } + + /** + * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final + * combine step happens locally on the master, equivalent to running a single reduce task. + */ + def countByValue(): Map[T, Long] = { + if (elementClassManifest.erasure.isArray) { + throw new SparkException("countByValue() does not support arrays") + } + // TODO: This should perhaps be distributed by default. + def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = { + val map = new OLMap[T] + while (iter.hasNext) { + val v = iter.next() + map.put(v, map.getLong(v) + 1L) + } + Iterator(map) + } + def mergeMaps(m1: OLMap[T], m2: OLMap[T]): OLMap[T] = { + val iter = m2.object2LongEntrySet.fastIterator() + while (iter.hasNext) { + val entry = iter.next() + m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue) + } + return m1 + } + val myResult = mapPartitions(countPartition).reduce(mergeMaps) + myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map + } + + /** + * (Experimental) Approximate version of countByValue(). + */ + def countByValueApprox( + timeout: Long, + confidence: Double = 0.95 + ): PartialResult[Map[T, BoundedDouble]] = { + if (elementClassManifest.erasure.isArray) { + throw new SparkException("countByValueApprox() does not support arrays") + } + val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => + val map = new OLMap[T] + while (iter.hasNext) { + val v = iter.next() + map.put(v, map.getLong(v) + 1L) + } + map + } + val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence) + sc.runApproximateJob(this, countPartition, evaluator, timeout) + } + + /** + * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so + * it will be slow if a lot of partitions are required. In that case, use collect() to get the + * whole RDD instead. + */ + def take(num: Int): Array[T] = { + if (num == 0) { + return new Array[T](0) + } + val buf = new ArrayBuffer[T] + var p = 0 + while (buf.size < num && p < partitions.size) { + val left = num - buf.size + val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true) + buf ++= res(0) + if (buf.size == num) + return buf.toArray + p += 1 + } + return buf.toArray + } + + /** + * Return the first element in this RDD. + */ + def first(): T = take(1) match { + case Array(t) => t + case _ => throw new UnsupportedOperationException("empty collection") + } + + /** + * Returns the top K elements from this RDD as defined by + * the specified implicit Ordering[T]. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { + mapPartitions { items => + val queue = new BoundedPriorityQueue[T](num) + queue ++= items + Iterator.single(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord.reverse) + } + + /** + * Returns the first K elements from this RDD as defined by + * the specified implicit Ordering[T] and maintains the + * ordering. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) + + /** + * Save this RDD as a text file, using string representations of elements. + */ + def saveAsTextFile(path: String) { + this.map(x => (NullWritable.get(), new Text(x.toString))) + .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) + } + + /** + * Save this RDD as a compressed text file, using string representations of elements. + */ + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { + this.map(x => (NullWritable.get(), new Text(x.toString))) + .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) + } + + /** + * Save this RDD as a SequenceFile of serialized objects. + */ + def saveAsObjectFile(path: String) { + this.mapPartitions(iter => iter.grouped(10).map(_.toArray)) + .map(x => (NullWritable.get(), new BytesWritable(Utils.serialize(x)))) + .saveAsSequenceFile(path) + } + + /** + * Creates tuples of the elements in this RDD by applying `f`. + */ + def keyBy[K](f: T => K): RDD[(K, T)] = { + map(x => (f(x), x)) + } + + /** A private method for tests, to look at the contents of each partition */ + private[spark] def collectPartitions(): Array[Array[T]] = { + sc.runJob(this, (iter: Iterator[T]) => iter.toArray) + } + + /** + * Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint + * directory set with SparkContext.setCheckpointDir() and all references to its parent + * RDDs will be removed. This function must be called before any job has been + * executed on this RDD. It is strongly recommended that this RDD is persisted in + * memory, otherwise saving it on a file will require recomputation. + */ + def checkpoint() { + if (context.checkpointDir.isEmpty) { + throw new Exception("Checkpoint directory has not been set in the SparkContext") + } else if (checkpointData.isEmpty) { + checkpointData = Some(new RDDCheckpointData(this)) + checkpointData.get.markForCheckpoint() + } + } + + /** + * Return whether this RDD has been checkpointed or not + */ + def isCheckpointed: Boolean = { + checkpointData.map(_.isCheckpointed).getOrElse(false) + } + + /** + * Gets the name of the file to which this RDD was checkpointed + */ + def getCheckpointFile: Option[String] = { + checkpointData.flatMap(_.getCheckpointFile) + } + + // ======================================================================= + // Other internal methods and fields + // ======================================================================= + + private var storageLevel: StorageLevel = StorageLevel.NONE + + /** Record user function generating this RDD. */ + private[spark] val origin = Utils.formatSparkCallSite + + private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] + + private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None + + /** Returns the first parent RDD */ + protected[spark] def firstParent[U: ClassManifest] = { + dependencies.head.rdd.asInstanceOf[RDD[U]] + } + + /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ + def context = sc + + // Avoid handling doCheckpoint multiple times to prevent excessive recursion + private var doCheckpointCalled = false + + /** + * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler + * after a job using this RDD has completed (therefore the RDD has been materialized and + * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. + */ + private[spark] def doCheckpoint() { + if (!doCheckpointCalled) { + doCheckpointCalled = true + if (checkpointData.isDefined) { + checkpointData.get.doCheckpoint() + } else { + dependencies.foreach(_.rdd.doCheckpoint()) + } + } + } + + /** + * Changes the dependencies of this RDD from its original parents to a new RDD (`newRDD`) + * created from the checkpoint file, and forget its old dependencies and partitions. + */ + private[spark] def markCheckpointed(checkpointRDD: RDD[_]) { + clearDependencies() + partitions_ = null + deps = null // Forget the constructor argument for dependencies too + } + + /** + * Clears the dependencies of this RDD. This method must ensure that all references + * to the original parent RDDs is removed to enable the parent RDDs to be garbage + * collected. Subclasses of RDD may override this method for implementing their own cleaning + * logic. See [[org.apache.spark.rdd.UnionRDD]] for an example. + */ + protected def clearDependencies() { + dependencies_ = null + } + + /** A description of this RDD and its recursive dependencies for debugging. */ + def toDebugString: String = { + def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { + Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++ + rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) + } + debugString(this).mkString("\n") + } + + override def toString: String = "%s%s[%d] at %s".format( + Option(name).map(_ + " ").getOrElse(""), + getClass.getSimpleName, + id, + origin) + + def toJavaRDD() : JavaRDD[T] = { + new JavaRDD(this)(elementClassManifest) + } + +} diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala new file mode 100644 index 0000000000..6009a41570 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -0,0 +1,131 @@ +/* + * 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.rdd + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{Partition, SparkException, Logging} +import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} + +/** + * Enumeration to manage state transitions of an RDD through checkpointing + * [ Initialized --> marked for checkpointing --> checkpointing in progress --> checkpointed ] + */ +private[spark] object CheckpointState extends Enumeration { + type CheckpointState = Value + val Initialized, MarkedForCheckpoint, CheckpointingInProgress, Checkpointed = Value +} + +/** + * This class contains all the information related to RDD checkpointing. Each instance of this class + * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as, + * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations + * of the checkpointed RDD. + */ +private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T]) + extends Logging with Serializable { + + import CheckpointState._ + + // The checkpoint state of the associated RDD. + var cpState = Initialized + + // The file to which the associated RDD has been checkpointed to + @transient var cpFile: Option[String] = None + + // The CheckpointRDD created from the checkpoint file, that is, the new parent the associated RDD. + var cpRDD: Option[RDD[T]] = None + + // Mark the RDD for checkpointing + def markForCheckpoint() { + RDDCheckpointData.synchronized { + if (cpState == Initialized) cpState = MarkedForCheckpoint + } + } + + // Is the RDD already checkpointed + def isCheckpointed: Boolean = { + RDDCheckpointData.synchronized { cpState == Checkpointed } + } + + // Get the file to which this RDD was checkpointed to as an Option + def getCheckpointFile: Option[String] = { + RDDCheckpointData.synchronized { cpFile } + } + + // Do the checkpointing of the RDD. Called after the first job using that RDD is over. + def doCheckpoint() { + // If it is marked for checkpointing AND checkpointing is not already in progress, + // then set it to be in progress, else return + RDDCheckpointData.synchronized { + if (cpState == MarkedForCheckpoint) { + cpState = CheckpointingInProgress + } else { + return + } + } + + // Create the output path for the checkpoint + val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) + val fs = path.getFileSystem(new Configuration()) + if (!fs.mkdirs(path)) { + throw new SparkException("Failed to create checkpoint path " + path) + } + + // Save to file, and reload it as an RDD + rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString) _) + val newRDD = new CheckpointRDD[T](rdd.context, path.toString) + + // Change the dependencies and partitions of the RDD + RDDCheckpointData.synchronized { + cpFile = Some(path.toString) + cpRDD = Some(newRDD) + rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions + cpState = Checkpointed + RDDCheckpointData.clearTaskCaches() + logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id) + } + } + + // Get preferred location of a split after checkpointing + def getPreferredLocations(split: Partition): Seq[String] = { + RDDCheckpointData.synchronized { + cpRDD.get.preferredLocations(split) + } + } + + def getPartitions: Array[Partition] = { + RDDCheckpointData.synchronized { + cpRDD.get.partitions + } + } + + def checkpointRDD: Option[RDD[T]] = { + RDDCheckpointData.synchronized { + cpRDD + } + } +} + +private[spark] object RDDCheckpointData { + def clearTaskCaches() { + ShuffleMapTask.clearCache() + ResultTask.clearCache() + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 1e8d89e912..2c5253ae30 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -22,7 +22,7 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala new file mode 100644 index 0000000000..5fe4676029 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -0,0 +1,89 @@ +/* + * 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.rdd + +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.SequenceFileOutputFormat +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.io.Writable + +import org.apache.spark.SparkContext._ +import org.apache.spark.Logging + +/** + * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, + * through an implicit conversion. Note that this can't be part of PairRDDFunctions because + * we need more implicit parameters to convert our keys and values to Writable. + * + * Import `org.apache.spark.SparkContext._` at the top of their program to use these functions. + */ +class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest]( + self: RDD[(K, V)]) + extends Logging + with Serializable { + + private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = { + val c = { + if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { + classManifest[T].erasure + } 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]] + } + + /** + * Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key + * and value types. If the key or value are Writable, then we use their classes directly; + * otherwise we map primitive types such as Int and Double to IntWritable, DoubleWritable, etc, + * byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported + * file system. + */ + def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) { + def anyToWritable[U <% Writable](u: U): Writable = u + + val keyClass = getWritableClass[K] + val valueClass = getWritableClass[V] + val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) + val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) + + logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) + val format = classOf[SequenceFileOutputFormat[Writable, Writable]] + val jobConf = new JobConf(self.context.hadoopConfiguration) + if (!convertKey && !convertValue) { + self.saveAsHadoopFile(path, keyClass, valueClass, format, jobConf, codec) + } else if (!convertKey && convertValue) { + self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) + } else if (convertKey && !convertValue) { + self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) + } else if (convertKey && convertValue) { + self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index f0e9ab8b80..9537152335 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} +import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 7369dfaa74..8c1a29dfff 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -20,7 +20,6 @@ package org.apache.spark.rdd import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark.RDD import org.apache.spark.Partitioner import org.apache.spark.Dependency import org.apache.spark.TaskContext diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index fd02476b62..ae8a9f36a6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.rdd import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{Dependency, RangeDependency, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 5ae1db3e67..31e6fd519d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 3bd00d291b..567b67dfee 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5ac700bbf4..92add5b073 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 5b07933eed..0d99670648 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -23,6 +23,7 @@ import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 98ef4d1e63..c8b78bf00a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -28,6 +28,7 @@ import scala.collection.mutable.{Map, HashMap, ListBuffer} import scala.io.Source import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 2f157ccdd2..2b007cbe82 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,11 +17,14 @@ package org.apache.spark.scheduler -import org.apache.spark._ import java.io._ -import util.{MetadataCleaner, TimeStampedHashMap} import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.RDDCheckpointData +import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} + private[spark] object ResultTask { // A simple map between the stage id to the serialized byte array of a task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index ca716b44e8..764775fede 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -26,6 +26,8 @@ import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.storage._ import org.apache.spark.util.{TimeStampedHashMap, MetadataCleaner} +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.RDDCheckpointData private[spark] object ShuffleMapTask { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 3504424fa9..c3cf4b8907 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,8 +19,8 @@ package org.apache.spark.scheduler import java.util.Properties import org.apache.spark.scheduler.cluster.TaskInfo -import org.apache.spark.util.Distribution -import org.apache.spark.{Logging, SparkContext, TaskEndReason, Utils} +import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.{Logging, SparkContext, TaskEndReason} import org.apache.spark.executor.TaskMetrics sealed trait SparkListenerEvents diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 87b1fe4e0c..aa293dc6b3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,9 +17,8 @@ package org.apache.spark.scheduler -import java.net.URI - import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 776675d28c..5c7e5bb977 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -21,8 +21,9 @@ import java.io._ import scala.collection.mutable.Map import org.apache.spark.executor.TaskMetrics -import org.apache.spark.{Utils, SparkEnv} +import org.apache.spark.{SparkEnv} import java.nio.ByteBuffer +import org.apache.spark.util.Utils // Task result. Also contains updates to accumulator variables. // TODO: Use of distributed cache to return result is a hack to get around diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index a33307b83a..1b31c8c57e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} +import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState} import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala index bde2f73df4..d57eb3276f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{SparkContext, Utils} +import org.apache.spark.{SparkContext} /** * A backend interface for cluster scheduling systems that allows plugging in different ones under diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ac6dc7d879..d003bf1bba 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,10 +17,11 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Utils, Logging, SparkContext} +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.client.{Client, ClientListener} import org.apache.spark.deploy.{Command, ApplicationDescription} import scala.collection.mutable.HashMap +import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: ClusterScheduler, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala index 1cc5daf673..9c36d221f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -20,8 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState -import org.apache.spark.Utils -import org.apache.spark.util.SerializableBuffer +import org.apache.spark.util.{Utils, SerializableBuffer} private[spark] sealed trait StandaloneClusterMessage extends Serializable diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 3677a827e0..b4ea0be415 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -28,8 +28,9 @@ import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClient import akka.util.Duration import akka.util.duration._ -import org.apache.spark.{Utils, SparkException, Logging, TaskState} +import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ +import org.apache.spark.util.Utils /** * A standalone scheduler backend, which waits for standalone executors to connect to it through diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala index 7ce14be7fb..9685fb1a67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.Utils +import org.apache.spark.util.Utils /** * Information about a running task attempt inside a TaskSet. diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index f0ebe66d82..e8fa5e2f17 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -34,6 +34,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ +import org.apache.spark.util.Utils /** * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f6a2feab28..3dbe61d706 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -23,7 +23,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Utils, Logging, SparkContext} +import org.apache.spark.{SparkException, Logging, SparkContext} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ import java.io.File diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala index e002af1742..541f86e338 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -23,7 +23,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Utils, Logging, SparkContext} +import org.apache.spark.{SparkException, Logging, SparkContext} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ import java.io.File @@ -31,6 +31,7 @@ import org.apache.spark.scheduler.cluster._ import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections import org.apache.spark.TaskState +import org.apache.spark.util.Utils /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala new file mode 100644 index 0000000000..4de81617b1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -0,0 +1,82 @@ +/* + * 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.serializer + +import java.io._ +import java.nio.ByteBuffer + +import org.apache.spark.util.ByteBufferInputStream + +private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { + val objOut = new ObjectOutputStream(out) + def writeObject[T](t: T): SerializationStream = { objOut.writeObject(t); this } + def flush() { objOut.flush() } + def close() { objOut.close() } +} + +private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) +extends DeserializationStream { + val objIn = new ObjectInputStream(in) { + override def resolveClass(desc: ObjectStreamClass) = + Class.forName(desc.getName, false, loader) + } + + def readObject[T](): T = objIn.readObject().asInstanceOf[T] + def close() { objIn.close() } +} + +private[spark] class JavaSerializerInstance extends SerializerInstance { + def serialize[T](t: T): ByteBuffer = { + val bos = new ByteArrayOutputStream() + val out = serializeStream(bos) + out.writeObject(t) + out.close() + ByteBuffer.wrap(bos.toByteArray) + } + + def deserialize[T](bytes: ByteBuffer): T = { + val bis = new ByteBufferInputStream(bytes) + val in = deserializeStream(bis) + in.readObject().asInstanceOf[T] + } + + def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + val bis = new ByteBufferInputStream(bytes) + val in = deserializeStream(bis, loader) + in.readObject().asInstanceOf[T] + } + + def serializeStream(s: OutputStream): SerializationStream = { + new JavaSerializationStream(s) + } + + def deserializeStream(s: InputStream): DeserializationStream = { + new JavaDeserializationStream(s, Thread.currentThread.getContextClassLoader) + } + + def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { + new JavaDeserializationStream(s, loader) + } +} + +/** + * A Spark serializer that uses Java's built-in serialization. + */ +class JavaSerializer extends Serializer { + def newInstance(): SerializerInstance = new JavaSerializerInstance +} diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala new file mode 100644 index 0000000000..24ef204aa1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -0,0 +1,159 @@ +/* + * 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.serializer + +import java.nio.ByteBuffer +import java.io.{EOFException, InputStream, OutputStream} + +import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} +import com.esotericsoftware.kryo.{KryoException, Kryo} +import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} +import com.twitter.chill.ScalaKryoInstantiator + +import org.apache.spark.{SerializableWritable, Logging} +import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock, StorageLevel} + +import org.apache.spark.broadcast.HttpBroadcast + +/** + * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]]. + */ +class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { + private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + + def newKryoOutput() = new KryoOutput(bufferSize) + + def newKryoInput() = new KryoInput(bufferSize) + + def newKryo(): Kryo = { + val instantiator = new ScalaKryoInstantiator + val kryo = instantiator.newKryo() + val classLoader = Thread.currentThread.getContextClassLoader + + // Register some commonly used classes + val toRegister: Seq[AnyRef] = Seq( + ByteBuffer.allocate(1), + StorageLevel.MEMORY_ONLY, + PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY), + GotBlock("1", ByteBuffer.allocate(1)), + GetBlock("1") + ) + + for (obj <- toRegister) kryo.register(obj.getClass) + + // Allow sending SerializableWritable + kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) + kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) + + // Allow the user to register their own classes by setting spark.kryo.registrator + try { + Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + logDebug("Running user registrator: " + regCls) + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + reg.registerClasses(kryo) + } + } catch { + case _: Exception => println("Failed to register spark.kryo.registrator") + } + + kryo.setClassLoader(classLoader) + + // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops + kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + + kryo + } + + def newInstance(): SerializerInstance = { + new KryoSerializerInstance(this) + } +} + +private[spark] +class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { + val output = new KryoOutput(outStream) + + def writeObject[T](t: T): SerializationStream = { + kryo.writeClassAndObject(output, t) + this + } + + def flush() { output.flush() } + def close() { output.close() } +} + +private[spark] +class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { + val input = new KryoInput(inStream) + + def readObject[T](): T = { + try { + kryo.readClassAndObject(input).asInstanceOf[T] + } catch { + // DeserializationStream uses the EOF exception to indicate stopping condition. + case _: KryoException => throw new EOFException + } + } + + def close() { + // Kryo's Input automatically closes the input stream it is using. + input.close() + } +} + +private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { + val kryo = ks.newKryo() + val output = ks.newKryoOutput() + val input = ks.newKryoInput() + + def serialize[T](t: T): ByteBuffer = { + output.clear() + kryo.writeClassAndObject(output, t) + ByteBuffer.wrap(output.toBytes) + } + + def deserialize[T](bytes: ByteBuffer): T = { + input.setBuffer(bytes.array) + kryo.readClassAndObject(input).asInstanceOf[T] + } + + def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + val oldClassLoader = kryo.getClassLoader + kryo.setClassLoader(loader) + input.setBuffer(bytes.array) + val obj = kryo.readClassAndObject(input).asInstanceOf[T] + kryo.setClassLoader(oldClassLoader) + obj + } + + def serializeStream(s: OutputStream): SerializationStream = { + new KryoSerializationStream(kryo, s) + } + + def deserializeStream(s: InputStream): DeserializationStream = { + new KryoDeserializationStream(kryo, s) + } +} + +/** + * Interface implemented by clients to register their classes with Kryo when using Kryo + * serialization. + */ +trait KryoRegistrator { + def registerClasses(kryo: Kryo) +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index c91f0fc1ad..3aeda3879d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -27,12 +27,12 @@ import scala.collection.mutable.Queue import io.netty.buffer.ByteBuf import org.apache.spark.Logging -import org.apache.spark.Utils import org.apache.spark.SparkException import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 3299ac98d5..60fdc5f2ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,11 +29,11 @@ import akka.util.duration._ import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.{Logging, SparkEnv, SparkException, Utils} +import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer -import org.apache.spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.util._ import sun.nio.ch.DirectBuffer diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index a22a80decc..74207f59af 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap -import org.apache.spark.Utils +import org.apache.spark.util.Utils /** * This class represent an unique identifier for a BlockManager. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index baa4a1da50..c7b23ab094 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,8 +28,9 @@ import akka.pattern.ask import akka.util.Duration import akka.util.duration._ -import org.apache.spark.{Logging, Utils, SparkException} +import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index f4856020e5..678c38203c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -19,8 +19,9 @@ package org.apache.spark.storage import java.nio.ByteBuffer -import org.apache.spark.{Logging, Utils} +import org.apache.spark.{Logging} import org.apache.spark.network._ +import org.apache.spark.util.Utils /** * A network interface for BlockManager. Each slave should have one diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index fd945e065c..fc25ef0fae 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -28,12 +28,12 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.Utils import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.serializer.{Serializer, SerializationStream} import org.apache.spark.Logging import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.network.netty.PathResolver +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 828dc0f22d..3b3b2342fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -19,9 +19,9 @@ package org.apache.spark.storage import java.util.LinkedHashMap import java.util.concurrent.ArrayBlockingQueue -import org.apache.spark.{SizeEstimator, Utils} import java.nio.ByteBuffer import collection.mutable.ArrayBuffer +import org.apache.spark.util.{SizeEstimator, Utils} /** * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 0bba1dac54..2bb7715696 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,8 +17,9 @@ package org.apache.spark.storage -import org.apache.spark.{Utils, SparkContext} +import org.apache.spark.{SparkContext} import BlockManagerMasterActor.BlockStatus +import org.apache.spark.util.Utils private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 1d5afe9b08..f2ae8dd97d 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -19,9 +19,9 @@ package org.apache.spark.storage import akka.actor._ -import org.apache.spark.KryoSerializer import java.util.concurrent.ArrayBlockingQueue import util.Random +import org.apache.spark.serializer.KryoSerializer /** * This class tests the BlockManager and MemoryStore for thread safety and diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 4688effe0a..ad456ea565 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,12 +21,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging, SparkContext, SparkEnv, Utils} +import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index efe6b474e0..6e56c22d04 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -7,13 +7,14 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.{ExceptionFailure, Logging, Utils, SparkContext} +import org.apache.spark.{ExceptionFailure, Logging, SparkContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils private[spark] class ExecutorsUI(val sc: SparkContext) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ae02226300..86e0af0399 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -3,7 +3,7 @@ package org.apache.spark.ui.jobs import scala.Seq import scala.collection.mutable.{ListBuffer, HashMap, HashSet} -import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils} +import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 1bb7638bd9..6aecef5120 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,11 +29,12 @@ import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils} +import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.scheduler._ import collection.mutable import org.apache.spark.scheduler.cluster.SchedulingMode import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 2fe85bc0cf..a9969ab1c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -25,8 +25,8 @@ import scala.xml.Node import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ -import org.apache.spark.util.Distribution -import org.apache.spark.{ExceptionFailure, Utils} +import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.{ExceptionFailure} import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index beb0574548..71e58a977e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -5,9 +5,9 @@ import java.util.Date import scala.xml.Node import scala.collection.mutable.HashSet -import org.apache.spark.Utils import org.apache.spark.scheduler.cluster.{SchedulingMode, TaskInfo} import org.apache.spark.scheduler.Stage +import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 1eb4a7a85e..c3ec907370 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -22,9 +22,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{RDDInfo, StorageUtils} -import org.apache.spark.Utils import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 37baf17f7a..43c1257677 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,11 +21,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.Utils import org.apache.spark.storage.{StorageStatus, StorageUtils} import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala new file mode 100644 index 0000000000..7108595e3e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -0,0 +1,232 @@ +/* + * 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.util + +import java.lang.reflect.Field + +import scala.collection.mutable.Map +import scala.collection.mutable.Set + +import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} +import org.objectweb.asm.Opcodes._ +import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} +import org.apache.spark.Logging + +private[spark] object ClosureCleaner extends Logging { + // Get an ASM class reader for a given class from the JAR that loaded it + private def getClassReader(cls: Class[_]): ClassReader = { + // Copy data over, before delegating to ClassReader - else we can run out of open file handles. + val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" + val resourceStream = cls.getResourceAsStream(className) + // todo: Fixme - continuing with earlier behavior ... + if (resourceStream == null) return new ClassReader(resourceStream) + + val baos = new ByteArrayOutputStream(128) + Utils.copyStream(resourceStream, baos, true) + new ClassReader(new ByteArrayInputStream(baos.toByteArray)) + } + + // Check whether a class represents a Scala closure + private def isClosure(cls: Class[_]): Boolean = { + cls.getName.contains("$anonfun$") + } + + // Get a list of the classes of the outer objects of a given closure object, obj; + // the outer objects are defined as any closures that obj is nested within, plus + // possibly the class that the outermost closure is in, if any. We stop searching + // for outer objects beyond that because cloning the user's object is probably + // not a good idea (whereas we can clone closure objects just fine since we + // understand how all their fields are used). + private def getOuterClasses(obj: AnyRef): List[Class[_]] = { + for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { + f.setAccessible(true) + if (isClosure(f.getType)) { + return f.getType :: getOuterClasses(f.get(obj)) + } else { + return f.getType :: Nil // Stop at the first $outer that is not a closure + } + } + return Nil + } + + // Get a list of the outer objects for a given closure object. + private def getOuterObjects(obj: AnyRef): List[AnyRef] = { + for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { + f.setAccessible(true) + if (isClosure(f.getType)) { + return f.get(obj) :: getOuterObjects(f.get(obj)) + } else { + return f.get(obj) :: Nil // Stop at the first $outer that is not a closure + } + } + return Nil + } + + private def getInnerClasses(obj: AnyRef): List[Class[_]] = { + val seen = Set[Class[_]](obj.getClass) + var stack = List[Class[_]](obj.getClass) + while (!stack.isEmpty) { + val cr = getClassReader(stack.head) + stack = stack.tail + val set = Set[Class[_]]() + cr.accept(new InnerClosureFinder(set), 0) + for (cls <- set -- seen) { + seen += cls + stack = cls :: stack + } + } + return (seen - obj.getClass).toList + } + + private def createNullValue(cls: Class[_]): AnyRef = { + if (cls.isPrimitive) { + new java.lang.Byte(0: Byte) // Should be convertible to any primitive type + } else { + null + } + } + + def clean(func: AnyRef) { + // TODO: cache outerClasses / innerClasses / accessedFields + val outerClasses = getOuterClasses(func) + val innerClasses = getInnerClasses(func) + val outerObjects = getOuterObjects(func) + + val accessedFields = Map[Class[_], Set[String]]() + for (cls <- outerClasses) + accessedFields(cls) = Set[String]() + for (cls <- func.getClass :: innerClasses) + getClassReader(cls).accept(new FieldAccessFinder(accessedFields), 0) + //logInfo("accessedFields: " + accessedFields) + + val inInterpreter = { + try { + val interpClass = Class.forName("spark.repl.Main") + interpClass.getMethod("interp").invoke(null) != null + } catch { + case _: ClassNotFoundException => true + } + } + + var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse + var outer: AnyRef = null + if (outerPairs.size > 0 && !isClosure(outerPairs.head._1)) { + // The closure is ultimately nested inside a class; keep the object of that + // class without cloning it since we don't want to clone the user's objects. + outer = outerPairs.head._2 + outerPairs = outerPairs.tail + } + // Clone the closure objects themselves, nulling out any fields that are not + // used in the closure we're working on or any of its inner closures. + for ((cls, obj) <- outerPairs) { + outer = instantiateClass(cls, outer, inInterpreter) + for (fieldName <- accessedFields(cls)) { + val field = cls.getDeclaredField(fieldName) + field.setAccessible(true) + val value = field.get(obj) + //logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); + field.set(outer, value) + } + } + + if (outer != null) { + //logInfo("2: Setting $outer on " + func.getClass + " to " + outer); + val field = func.getClass.getDeclaredField("$outer") + field.setAccessible(true) + field.set(func, outer) + } + } + + private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { + //logInfo("Creating a " + cls + " with outer = " + outer) + if (!inInterpreter) { + // This is a bona fide closure class, whose constructor has no effects + // other than to set its fields, so use its constructor + val cons = cls.getConstructors()(0) + val params = cons.getParameterTypes.map(createNullValue).toArray + if (outer != null) + params(0) = outer // First param is always outer object + return cons.newInstance(params: _*).asInstanceOf[AnyRef] + } else { + // Use reflection to instantiate object without calling constructor + val rf = sun.reflect.ReflectionFactory.getReflectionFactory() + val parentCtor = classOf[java.lang.Object].getDeclaredConstructor() + val newCtor = rf.newConstructorForSerialization(cls, parentCtor) + val obj = newCtor.newInstance().asInstanceOf[AnyRef] + if (outer != null) { + //logInfo("3: Setting $outer on " + cls + " to " + outer); + val field = cls.getDeclaredField("$outer") + field.setAccessible(true) + field.set(obj, outer) + } + return obj + } + } +} + +private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + return new MethodVisitor(ASM4) { + override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { + if (op == GETFIELD) { + for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { + output(cl) += name + } + } + } + + override def visitMethodInsn(op: Int, owner: String, name: String, + desc: String) { + // Check for calls a getter method for a variable in an interpreter wrapper object. + // This means that the corresponding field will be accessed, so we should save it. + if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer")) { + for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { + output(cl) += name + } + } + } + } + } +} + +private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { + var myName: String = null + + override def visit(version: Int, access: Int, name: String, sig: String, + superName: String, interfaces: Array[String]) { + myName = name + } + + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + return new MethodVisitor(ASM4) { + override def visitMethodInsn(op: Int, owner: String, name: String, + desc: String) { + val argTypes = Type.getArgumentTypes(desc) + if (op == INVOKESPECIAL && name == "" && argTypes.length > 0 + && argTypes(0).toString.startsWith("L") // is it an object? + && argTypes(0).getInternalName == myName) + output += Class.forName( + owner.replace('/', '.'), + false, + Thread.currentThread.getContextClassLoader) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/MemoryParam.scala b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala index 0ee6707826..4869c9897a 100644 --- a/core/src/main/scala/org/apache/spark/util/MemoryParam.scala +++ b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala @@ -17,8 +17,6 @@ package org.apache.spark.util -import org.apache.spark.Utils - /** * An extractor object for parsing JVM memory strings, such as "10g", into an Int representing * the number of megabytes. Supports the same formats as Utils.memoryStringToMb. diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala new file mode 100644 index 0000000000..a25b37a2a9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -0,0 +1,284 @@ +/* + * 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.util + +import java.lang.reflect.Field +import java.lang.reflect.Modifier +import java.lang.reflect.{Array => JArray} +import java.util.IdentityHashMap +import java.util.concurrent.ConcurrentHashMap +import java.util.Random + +import javax.management.MBeanServer +import java.lang.management.ManagementFactory + +import scala.collection.mutable.ArrayBuffer + +import it.unimi.dsi.fastutil.ints.IntOpenHashSet +import org.apache.spark.Logging + +/** + * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in + * memory-aware caches. + * + * Based on the following JavaWorld article: + * http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html + */ +private[spark] object SizeEstimator extends Logging { + + // Sizes of primitive types + private val BYTE_SIZE = 1 + private val BOOLEAN_SIZE = 1 + private val CHAR_SIZE = 2 + private val SHORT_SIZE = 2 + private val INT_SIZE = 4 + private val LONG_SIZE = 8 + private val FLOAT_SIZE = 4 + private val DOUBLE_SIZE = 8 + + // Alignment boundary for objects + // TODO: Is this arch dependent ? + private val ALIGN_SIZE = 8 + + // A cache of ClassInfo objects for each class + private val classInfos = new ConcurrentHashMap[Class[_], ClassInfo] + + // Object and pointer sizes are arch dependent + private var is64bit = false + + // Size of an object reference + // Based on https://wikis.oracle.com/display/HotSpotInternals/CompressedOops + private var isCompressedOops = false + private var pointerSize = 4 + + // Minimum size of a java.lang.Object + private var objectSize = 8 + + initialize() + + // Sets object size, pointer size based on architecture and CompressedOops settings + // from the JVM. + private def initialize() { + is64bit = System.getProperty("os.arch").contains("64") + isCompressedOops = getIsCompressedOops + + objectSize = if (!is64bit) 8 else { + if(!isCompressedOops) { + 16 + } else { + 12 + } + } + pointerSize = if (is64bit && !isCompressedOops) 8 else 4 + classInfos.clear() + classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil)) + } + + private def getIsCompressedOops : Boolean = { + if (System.getProperty("spark.test.useCompressedOops") != null) { + return System.getProperty("spark.test.useCompressedOops").toBoolean + } + + try { + val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic" + val server = ManagementFactory.getPlatformMBeanServer() + + // NOTE: This should throw an exception in non-Sun JVMs + val hotSpotMBeanClass = Class.forName("com.sun.management.HotSpotDiagnosticMXBean") + val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption", + Class.forName("java.lang.String")) + + val bean = ManagementFactory.newPlatformMXBeanProxy(server, + hotSpotMBeanName, hotSpotMBeanClass) + // TODO: We could use reflection on the VMOption returned ? + return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") + } catch { + case e: Exception => { + // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB + val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024) + val guessInWords = if (guess) "yes" else "not" + logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords) + return guess + } + } + } + + /** + * The state of an ongoing size estimation. Contains a stack of objects to visit as well as an + * IdentityHashMap of visited objects, and provides utility methods for enqueueing new objects + * to visit. + */ + private class SearchState(val visited: IdentityHashMap[AnyRef, AnyRef]) { + val stack = new ArrayBuffer[AnyRef] + var size = 0L + + def enqueue(obj: AnyRef) { + if (obj != null && !visited.containsKey(obj)) { + visited.put(obj, null) + stack += obj + } + } + + def isFinished(): Boolean = stack.isEmpty + + def dequeue(): AnyRef = { + val elem = stack.last + stack.trimEnd(1) + return elem + } + } + + /** + * Cached information about each class. We remember two things: the "shell size" of the class + * (size of all non-static fields plus the java.lang.Object size), and any fields that are + * pointers to objects. + */ + private class ClassInfo( + val shellSize: Long, + val pointerFields: List[Field]) {} + + def estimate(obj: AnyRef): Long = estimate(obj, new IdentityHashMap[AnyRef, AnyRef]) + + private def estimate(obj: AnyRef, visited: IdentityHashMap[AnyRef, AnyRef]): Long = { + val state = new SearchState(visited) + state.enqueue(obj) + while (!state.isFinished) { + visitSingleObject(state.dequeue(), state) + } + return state.size + } + + private def visitSingleObject(obj: AnyRef, state: SearchState) { + val cls = obj.getClass + if (cls.isArray) { + visitArray(obj, cls, state) + } else if (obj.isInstanceOf[ClassLoader] || obj.isInstanceOf[Class[_]]) { + // Hadoop JobConfs created in the interpreter have a ClassLoader, which greatly confuses + // the size estimator since it references the whole REPL. Do nothing in this case. In + // general all ClassLoaders and Classes will be shared between objects anyway. + } else { + val classInfo = getClassInfo(cls) + state.size += classInfo.shellSize + for (field <- classInfo.pointerFields) { + state.enqueue(field.get(obj)) + } + } + } + + // Estimat the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. + private val ARRAY_SIZE_FOR_SAMPLING = 200 + private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING + + private def visitArray(array: AnyRef, cls: Class[_], state: SearchState) { + val length = JArray.getLength(array) + val elementClass = cls.getComponentType + + // Arrays have object header and length field which is an integer + var arrSize: Long = alignSize(objectSize + INT_SIZE) + + if (elementClass.isPrimitive) { + arrSize += alignSize(length * primitiveSize(elementClass)) + state.size += arrSize + } else { + arrSize += alignSize(length * pointerSize) + state.size += arrSize + + if (length <= ARRAY_SIZE_FOR_SAMPLING) { + for (i <- 0 until length) { + state.enqueue(JArray.get(array, i)) + } + } else { + // Estimate the size of a large array by sampling elements without replacement. + var size = 0.0 + val rand = new Random(42) + val drawn = new IntOpenHashSet(ARRAY_SAMPLE_SIZE) + for (i <- 0 until ARRAY_SAMPLE_SIZE) { + var index = 0 + do { + index = rand.nextInt(length) + } while (drawn.contains(index)) + drawn.add(index) + val elem = JArray.get(array, index) + size += SizeEstimator.estimate(elem, state.visited) + } + state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong + } + } + } + + private def primitiveSize(cls: Class[_]): Long = { + if (cls == classOf[Byte]) + BYTE_SIZE + else if (cls == classOf[Boolean]) + BOOLEAN_SIZE + else if (cls == classOf[Char]) + CHAR_SIZE + else if (cls == classOf[Short]) + SHORT_SIZE + else if (cls == classOf[Int]) + INT_SIZE + else if (cls == classOf[Long]) + LONG_SIZE + else if (cls == classOf[Float]) + FLOAT_SIZE + else if (cls == classOf[Double]) + DOUBLE_SIZE + else throw new IllegalArgumentException( + "Non-primitive class " + cls + " passed to primitiveSize()") + } + + /** + * Get or compute the ClassInfo for a given class. + */ + private def getClassInfo(cls: Class[_]): ClassInfo = { + // Check whether we've already cached a ClassInfo for this class + val info = classInfos.get(cls) + if (info != null) { + return info + } + + val parent = getClassInfo(cls.getSuperclass) + var shellSize = parent.shellSize + var pointerFields = parent.pointerFields + + for (field <- cls.getDeclaredFields) { + if (!Modifier.isStatic(field.getModifiers)) { + val fieldClass = field.getType + if (fieldClass.isPrimitive) { + shellSize += primitiveSize(fieldClass) + } else { + field.setAccessible(true) // Enable future get()'s on this field + shellSize += pointerSize + pointerFields = field :: pointerFields + } + } + } + + shellSize = alignSize(shellSize) + + // Create and cache a new ClassInfo + val newInfo = new ClassInfo(shellSize, pointerFields) + classInfos.put(cls, newInfo) + return newInfo + } + + private def alignSize(size: Long): Long = { + val rem = size % ALIGN_SIZE + return if (rem == 0) size else (size + ALIGN_SIZE - rem) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala new file mode 100644 index 0000000000..bb47fc0a2c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -0,0 +1,781 @@ +/* + * 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.util + +import java.io._ +import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} +import java.util.{Locale, Random, UUID} +import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} +import java.util.regex.Pattern + +import scala.collection.Map +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.JavaConversions._ +import scala.io.Source + +import com.google.common.io.Files +import com.google.common.util.concurrent.ThreadFactoryBuilder + +import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} + +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.deploy.SparkHadoopUtil +import java.nio.ByteBuffer +import org.apache.spark.{SparkEnv, SparkException, Logging} + + +/** + * Various utility methods used by Spark. + */ +private[spark] object Utils extends Logging { + + /** Serialize an object using Java serialization */ + def serialize[T](o: T): Array[Byte] = { + val bos = new ByteArrayOutputStream() + val oos = new ObjectOutputStream(bos) + oos.writeObject(o) + oos.close() + return bos.toByteArray + } + + /** Deserialize an object using Java serialization */ + def deserialize[T](bytes: Array[Byte]): T = { + val bis = new ByteArrayInputStream(bytes) + val ois = new ObjectInputStream(bis) + return ois.readObject.asInstanceOf[T] + } + + /** Deserialize an object using Java serialization and the given ClassLoader */ + def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = { + val bis = new ByteArrayInputStream(bytes) + val ois = new ObjectInputStream(bis) { + override def resolveClass(desc: ObjectStreamClass) = + Class.forName(desc.getName, false, loader) + } + return ois.readObject.asInstanceOf[T] + } + + /** Serialize via nested stream using specific serializer */ + def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)(f: SerializationStream => Unit) = { + val osWrapper = ser.serializeStream(new OutputStream { + def write(b: Int) = os.write(b) + + override def write(b: Array[Byte], off: Int, len: Int) = os.write(b, off, len) + }) + try { + f(osWrapper) + } finally { + osWrapper.close() + } + } + + /** Deserialize via nested stream using specific serializer */ + def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)(f: DeserializationStream => Unit) = { + val isWrapper = ser.deserializeStream(new InputStream { + def read(): Int = is.read() + + override def read(b: Array[Byte], off: Int, len: Int): Int = is.read(b, off, len) + }) + try { + f(isWrapper) + } finally { + isWrapper.close() + } + } + + /** + * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. + */ + def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = { + if (bb.hasArray) { + out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) + } else { + val bbval = new Array[Byte](bb.remaining()) + bb.get(bbval) + out.write(bbval) + } + } + + def isAlpha(c: Char): Boolean = { + (c >= 'A' && c <= 'Z') || (c >= 'a' && c <= 'z') + } + + /** Split a string into words at non-alphabetic characters */ + def splitWords(s: String): Seq[String] = { + val buf = new ArrayBuffer[String] + var i = 0 + while (i < s.length) { + var j = i + while (j < s.length && isAlpha(s.charAt(j))) { + j += 1 + } + if (j > i) { + buf += s.substring(i, j) + } + i = j + while (i < s.length && !isAlpha(s.charAt(i))) { + i += 1 + } + } + return buf + } + + private val shutdownDeletePaths = new collection.mutable.HashSet[String]() + + // Register the path to be deleted via shutdown hook + def registerShutdownDeleteDir(file: File) { + val absolutePath = file.getAbsolutePath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths += absolutePath + } + } + + // Is the path already registered to be deleted via a shutdown hook ? + def hasShutdownDeleteDir(file: File): Boolean = { + val absolutePath = file.getAbsolutePath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths.contains(absolutePath) + } + } + + // Note: if file is child of some registered path, while not equal to it, then return true; + // else false. This is to ensure that two shutdown hooks do not try to delete each others + // paths - resulting in IOException and incomplete cleanup. + def hasRootAsShutdownDeleteDir(file: File): Boolean = { + val absolutePath = file.getAbsolutePath() + val retval = shutdownDeletePaths.synchronized { + shutdownDeletePaths.find { path => + !absolutePath.equals(path) && absolutePath.startsWith(path) + }.isDefined + } + if (retval) { + logInfo("path = " + file + ", already present as root for deletion.") + } + retval + } + + /** Create a temporary directory inside the given parent directory */ + def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + var attempts = 0 + val maxAttempts = 10 + var dir: File = null + while (dir == null) { + attempts += 1 + if (attempts > maxAttempts) { + throw new IOException("Failed to create a temp directory (under " + root + ") after " + + maxAttempts + " attempts!") + } + try { + dir = new File(root, "spark-" + UUID.randomUUID.toString) + if (dir.exists() || !dir.mkdirs()) { + dir = null + } + } catch { case e: IOException => ; } + } + + registerShutdownDeleteDir(dir) + + // Add a shutdown hook to delete the temp dir when the JVM exits + Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dir " + dir) { + override def run() { + // Attempt to delete if some patch which is parent of this is not already registered. + if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir) + } + }) + dir + } + + /** Copy all data from an InputStream to an OutputStream */ + def copyStream(in: InputStream, + out: OutputStream, + closeStreams: Boolean = false) + { + val buf = new Array[Byte](8192) + var n = 0 + while (n != -1) { + n = in.read(buf) + if (n != -1) { + out.write(buf, 0, n) + } + } + if (closeStreams) { + in.close() + out.close() + } + } + + /** + * Download a file requested by the executor. Supports fetching the file in a variety of ways, + * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * + * Throws SparkException if the target file already exists and has different contents than + * the requested file. + */ + def fetchFile(url: String, targetDir: File) { + val filename = url.split("/").last + val tempDir = getLocalDir + val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) + val targetFile = new File(targetDir, filename) + val uri = new URI(url) + uri.getScheme match { + case "http" | "https" | "ftp" => + logInfo("Fetching " + url + " to " + tempFile) + val in = new URL(url).openStream() + val out = new FileOutputStream(tempFile) + Utils.copyStream(in, out, true) + if (targetFile.exists && !Files.equal(tempFile, targetFile)) { + tempFile.delete() + throw new SparkException( + "File " + targetFile + " exists and does not match contents of" + " " + url) + } else { + Files.move(tempFile, targetFile) + } + case "file" | null => + // In the case of a local file, copy the local file to the target directory. + // Note the difference between uri vs url. + val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) + if (targetFile.exists) { + // If the target file already exists, warn the user if + if (!Files.equal(sourceFile, targetFile)) { + throw new SparkException( + "File " + targetFile + " exists and does not match contents of" + " " + url) + } else { + // Do nothing if the file contents are the same, i.e. this file has been copied + // previously. + logInfo(sourceFile.getAbsolutePath + " has been previously copied to " + + targetFile.getAbsolutePath) + } + } else { + // The file does not exist in the target directory. Copy it there. + logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) + Files.copy(sourceFile, targetFile) + } + case _ => + // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others + val env = SparkEnv.get + val uri = new URI(url) + val conf = env.hadoop.newConfiguration() + val fs = FileSystem.get(uri, conf) + val in = fs.open(new Path(uri)) + val out = new FileOutputStream(tempFile) + Utils.copyStream(in, out, true) + if (targetFile.exists && !Files.equal(tempFile, targetFile)) { + tempFile.delete() + throw new SparkException("File " + targetFile + " exists and does not match contents of" + + " " + url) + } else { + Files.move(tempFile, targetFile) + } + } + // Decompress the file if it's a .tar or .tar.gz + if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) { + logInfo("Untarring " + filename) + Utils.execute(Seq("tar", "-xzf", filename), targetDir) + } else if (filename.endsWith(".tar")) { + logInfo("Untarring " + filename) + Utils.execute(Seq("tar", "-xf", filename), targetDir) + } + // Make the file executable - That's necessary for scripts + FileUtil.chmod(targetFile.getAbsolutePath, "a+x") + } + + /** + * Get a temporary directory using Spark's spark.local.dir property, if set. This will always + * return a single directory, even though the spark.local.dir property might be a list of + * multiple paths. + */ + def getLocalDir: String = { + System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + } + + /** + * Shuffle the elements of a collection into a random order, returning the + * result in a new collection. Unlike scala.util.Random.shuffle, this method + * uses a local random number generator, avoiding inter-thread contention. + */ + def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = { + randomizeInPlace(seq.toArray) + } + + /** + * Shuffle the elements of an array into a random order, modifying the + * original array. Returns the original array. + */ + def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = { + for (i <- (arr.length - 1) to 1 by -1) { + val j = rand.nextInt(i) + val tmp = arr(j) + arr(j) = arr(i) + arr(i) = tmp + } + arr + } + + /** + * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). + * Note, this is typically not used from within core spark. + */ + lazy val localIpAddress: String = findLocalIpAddress() + lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress) + + private def findLocalIpAddress(): String = { + val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") + if (defaultIpOverride != null) { + defaultIpOverride + } else { + val address = InetAddress.getLocalHost + if (address.isLoopbackAddress) { + // Address resolves to something like 127.0.1.1, which happens on Debian; try to find + // a better address using the local network interfaces + for (ni <- NetworkInterface.getNetworkInterfaces) { + for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && + !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) { + // We've found an address that looks reasonable! + logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + + " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress + + " instead (on interface " + ni.getName + ")") + logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") + return addr.getHostAddress + } + } + logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + + " a loopback address: " + address.getHostAddress + ", but we couldn't find any" + + " external IP address!") + logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") + } + address.getHostAddress + } + } + + private var customHostname: Option[String] = None + + /** + * Allow setting a custom host name because when we run on Mesos we need to use the same + * hostname it reports to the master. + */ + def setCustomHostname(hostname: String) { + // DEBUG code + Utils.checkHost(hostname) + customHostname = Some(hostname) + } + + /** + * Get the local machine's hostname. + */ + def localHostName(): String = { + customHostname.getOrElse(localIpAddressHostname) + } + + def getAddressHostName(address: String): String = { + InetAddress.getByName(address).getHostName + } + + def localHostPort(): String = { + val retval = System.getProperty("spark.hostPort", null) + if (retval == null) { + logErrorWithStack("spark.hostPort not set but invoking localHostPort") + return localHostName() + } + + retval + } + + def checkHost(host: String, message: String = "") { + assert(host.indexOf(':') == -1, message) + } + + def checkHostPort(hostPort: String, message: String = "") { + assert(hostPort.indexOf(':') != -1, message) + } + + // Used by DEBUG code : remove when all testing done + def logErrorWithStack(msg: String) { + try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } + } + + // Typically, this will be of order of number of nodes in cluster + // If not, we should change it to LRUCache or something. + private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() + + def parseHostPort(hostPort: String): (String, Int) = { + { + // Check cache first. + var cached = hostPortParseResults.get(hostPort) + if (cached != null) return cached + } + + val indx: Int = hostPort.lastIndexOf(':') + // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... + // but then hadoop does not support ipv6 right now. + // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 + if (-1 == indx) { + val retval = (hostPort, 0) + hostPortParseResults.put(hostPort, retval) + return retval + } + + val retval = (hostPort.substring(0, indx).trim(), hostPort.substring(indx + 1).trim().toInt) + hostPortParseResults.putIfAbsent(hostPort, retval) + hostPortParseResults.get(hostPort) + } + + private[spark] val daemonThreadFactory: ThreadFactory = + new ThreadFactoryBuilder().setDaemon(true).build() + + /** + * Wrapper over newCachedThreadPool. + */ + def newDaemonCachedThreadPool(): ThreadPoolExecutor = + Executors.newCachedThreadPool(daemonThreadFactory).asInstanceOf[ThreadPoolExecutor] + + /** + * Return the string to tell how long has passed in seconds. The passing parameter should be in + * millisecond. + */ + def getUsedTimeMs(startTimeMs: Long): String = { + return " " + (System.currentTimeMillis - startTimeMs) + " ms" + } + + /** + * Wrapper over newFixedThreadPool. + */ + def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = + Executors.newFixedThreadPool(nThreads, daemonThreadFactory).asInstanceOf[ThreadPoolExecutor] + + /** + * Delete a file or directory and its contents recursively. + */ + def deleteRecursively(file: File) { + if (file.isDirectory) { + for (child <- file.listFiles()) { + deleteRecursively(child) + } + } + if (!file.delete()) { + throw new IOException("Failed to delete: " + file) + } + } + + /** + * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. + * This is used to figure out how much memory to claim from Mesos based on the SPARK_MEM + * environment variable. + */ + def memoryStringToMb(str: String): Int = { + val lower = str.toLowerCase + if (lower.endsWith("k")) { + (lower.substring(0, lower.length-1).toLong / 1024).toInt + } else if (lower.endsWith("m")) { + lower.substring(0, lower.length-1).toInt + } else if (lower.endsWith("g")) { + lower.substring(0, lower.length-1).toInt * 1024 + } else if (lower.endsWith("t")) { + lower.substring(0, lower.length-1).toInt * 1024 * 1024 + } else {// no suffix, so it's just a number in bytes + (lower.toLong / 1024 / 1024).toInt + } + } + + /** + * Convert a quantity in bytes to a human-readable string such as "4.0 MB". + */ + def bytesToString(size: Long): String = { + val TB = 1L << 40 + val GB = 1L << 30 + val MB = 1L << 20 + val KB = 1L << 10 + + val (value, unit) = { + if (size >= 2*TB) { + (size.asInstanceOf[Double] / TB, "TB") + } else if (size >= 2*GB) { + (size.asInstanceOf[Double] / GB, "GB") + } else if (size >= 2*MB) { + (size.asInstanceOf[Double] / MB, "MB") + } else if (size >= 2*KB) { + (size.asInstanceOf[Double] / KB, "KB") + } else { + (size.asInstanceOf[Double], "B") + } + } + "%.1f %s".formatLocal(Locale.US, value, unit) + } + + /** + * Returns a human-readable string representing a duration such as "35ms" + */ + def msDurationToString(ms: Long): String = { + val second = 1000 + val minute = 60 * second + val hour = 60 * minute + + ms match { + case t if t < second => + "%d ms".format(t) + case t if t < minute => + "%.1f s".format(t.toFloat / second) + case t if t < hour => + "%.1f m".format(t.toFloat / minute) + case t => + "%.2f h".format(t.toFloat / hour) + } + } + + /** + * Convert a quantity in megabytes to a human-readable string such as "4.0 MB". + */ + def megabytesToString(megabytes: Long): String = { + bytesToString(megabytes * 1024L * 1024L) + } + + /** + * Execute a command in the given working directory, throwing an exception if it completes + * with an exit code other than 0. + */ + def execute(command: Seq[String], workingDir: File) { + val process = new ProcessBuilder(command: _*) + .directory(workingDir) + .redirectErrorStream(true) + .start() + new Thread("read stdout for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getInputStream).getLines) { + System.err.println(line) + } + } + }.start() + val exitCode = process.waitFor() + if (exitCode != 0) { + throw new SparkException("Process " + command + " exited with code " + exitCode) + } + } + + /** + * Execute a command in the current working directory, throwing an exception if it completes + * with an exit code other than 0. + */ + def execute(command: Seq[String]) { + execute(command, new File(".")) + } + + /** + * Execute a command and get its output, throwing an exception if it yields a code other than 0. + */ + def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), + extraEnvironment: Map[String, String] = Map.empty): String = { + val builder = new ProcessBuilder(command: _*) + .directory(workingDir) + val environment = builder.environment() + for ((key, value) <- extraEnvironment) { + environment.put(key, value) + } + val process = builder.start() + new Thread("read stderr for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getErrorStream).getLines) { + System.err.println(line) + } + } + }.start() + val output = new StringBuffer + val stdoutThread = new Thread("read stdout for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getInputStream).getLines) { + output.append(line) + } + } + } + stdoutThread.start() + val exitCode = process.waitFor() + stdoutThread.join() // Wait for it to finish reading output + if (exitCode != 0) { + throw new SparkException("Process " + command + " exited with code " + exitCode) + } + output.toString + } + + /** + * A regular expression to match classes of the "core" Spark API that we want to skip when + * finding the call site of a method. + */ + private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r + + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + val firstUserLine: Int, val firstUserClass: String) + + /** + * When called inside a class in the spark package, returns the name of the user code class + * (outside the spark package) that called into Spark, as well as which Spark method they called. + * This is used, for example, to tell users where in their code each RDD got created. + */ + def getCallSiteInfo: CallSiteInfo = { + val trace = Thread.currentThread.getStackTrace().filter( el => + (!el.getMethodName.contains("getStackTrace"))) + + // Keep crawling up the stack trace until we find the first function not inside of the spark + // package. We track the last (shallowest) contiguous Spark method. This might be an RDD + // transformation, a SparkContext function (such as parallelize), or anything else that leads + // to instantiation of an RDD. We also track the first (deepest) user method, file, and line. + var lastSparkMethod = "" + var firstUserFile = "" + var firstUserLine = 0 + var finished = false + var firstUserClass = "" + + for (el <- trace) { + if (!finished) { + if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) { + lastSparkMethod = if (el.getMethodName == "") { + // Spark method is a constructor; get its class name + el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) + } else { + el.getMethodName + } + } + else { + firstUserLine = el.getLineNumber + firstUserFile = el.getFileName + firstUserClass = el.getClassName + finished = true + } + } + } + new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) + } + + def formatSparkCallSite = { + val callSiteInfo = getCallSiteInfo + "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, + callSiteInfo.firstUserLine) + } + + /** Return a string containing part of a file from byte 'start' to 'end'. */ + def offsetBytes(path: String, start: Long, end: Long): String = { + val file = new File(path) + val length = file.length() + val effectiveEnd = math.min(length, end) + val effectiveStart = math.max(0, start) + val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) + val stream = new FileInputStream(file) + + stream.skip(effectiveStart) + stream.read(buff) + stream.close() + Source.fromBytes(buff).mkString + } + + /** + * Clone an object using a Spark serializer. + */ + def clone[T](value: T, serializer: SerializerInstance): T = { + serializer.deserialize[T](serializer.serialize(value)) + } + + /** + * Detect whether this thread might be executing a shutdown hook. Will always return true if + * the current thread is a running a shutdown hook but may spuriously return true otherwise (e.g. + * if System.exit was just called by a concurrent thread). + * + * Currently, this detects whether the JVM is shutting down by Runtime#addShutdownHook throwing + * an IllegalStateException. + */ + def inShutdown(): Boolean = { + try { + val hook = new Thread { + override def run() {} + } + Runtime.getRuntime.addShutdownHook(hook) + Runtime.getRuntime.removeShutdownHook(hook) + } catch { + case ise: IllegalStateException => return true + } + return false + } + + def isSpace(c: Char): Boolean = { + " \t\r\n".indexOf(c) != -1 + } + + /** + * Split a string of potentially quoted arguments from the command line the way that a shell + * would do it to determine arguments to a command. For example, if the string is 'a "b c" d', + * then it would be parsed as three arguments: 'a', 'b c' and 'd'. + */ + def splitCommandString(s: String): Seq[String] = { + val buf = new ArrayBuffer[String] + var inWord = false + var inSingleQuote = false + var inDoubleQuote = false + var curWord = new StringBuilder + def endWord() { + buf += curWord.toString + curWord.clear() + } + var i = 0 + while (i < s.length) { + var nextChar = s.charAt(i) + if (inDoubleQuote) { + if (nextChar == '"') { + inDoubleQuote = false + } else if (nextChar == '\\') { + if (i < s.length - 1) { + // Append the next character directly, because only " and \ may be escaped in + // double quotes after the shell's own expansion + curWord.append(s.charAt(i + 1)) + i += 1 + } + } else { + curWord.append(nextChar) + } + } else if (inSingleQuote) { + if (nextChar == '\'') { + inSingleQuote = false + } else { + curWord.append(nextChar) + } + // Backslashes are not treated specially in single quotes + } else if (nextChar == '"') { + inWord = true + inDoubleQuote = true + } else if (nextChar == '\'') { + inWord = true + inSingleQuote = true + } else if (!isSpace(nextChar)) { + curWord.append(nextChar) + inWord = true + } else if (inWord && isSpace(nextChar)) { + endWord() + inWord = false + } + i += 1 + } + if (inWord || inDoubleQuote || inSingleQuote) { + endWord() + } + return buf + } + + /* Calculates 'x' modulo 'mod', takes to consideration sign of x, + * i.e. if 'x' is negative, than 'x' % 'mod' is negative too + * so function return (x % mod) + mod in that case. + */ + def nonNegativeMod(x: Int, mod: Int): Int = { + val rawMod = x % mod + rawMod + (if (rawMod < 0) mod else 0) + } +} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 23b14f4245..d9103aebb7 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.spark.rdd._ import org.apache.spark.SparkContext._ import storage.StorageLevel +import org.apache.spark.util.Utils class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { initLogging() diff --git a/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala deleted file mode 100644 index 8494899b98..0000000000 --- a/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala +++ /dev/null @@ -1,146 +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 - -import java.io.NotSerializableException - -import org.scalatest.FunSuite -import org.apache.spark.LocalSparkContext._ -import SparkContext._ - -class ClosureCleanerSuite extends FunSuite { - test("closures inside an object") { - assert(TestObject.run() === 30) // 6 + 7 + 8 + 9 - } - - test("closures inside a class") { - val obj = new TestClass - assert(obj.run() === 30) // 6 + 7 + 8 + 9 - } - - test("closures inside a class with no default constructor") { - val obj = new TestClassWithoutDefaultConstructor(5) - assert(obj.run() === 30) // 6 + 7 + 8 + 9 - } - - test("closures that don't use fields of the outer class") { - val obj = new TestClassWithoutFieldAccess - assert(obj.run() === 30) // 6 + 7 + 8 + 9 - } - - test("nested closures inside an object") { - assert(TestObjectWithNesting.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 - } - - test("nested closures inside a class") { - val obj = new TestClassWithNesting(1) - assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 - } -} - -// A non-serializable class we create in closures to make sure that we aren't -// keeping references to unneeded variables from our outer closures. -class NonSerializable {} - -object TestObject { - def run(): Int = { - var nonSer = new NonSerializable - var x = 5 - return withSpark(new SparkContext("local", "test")) { sc => - val nums = sc.parallelize(Array(1, 2, 3, 4)) - nums.map(_ + x).reduce(_ + _) - } - } -} - -class TestClass extends Serializable { - var x = 5 - - def getX = x - - def run(): Int = { - var nonSer = new NonSerializable - return withSpark(new SparkContext("local", "test")) { sc => - val nums = sc.parallelize(Array(1, 2, 3, 4)) - nums.map(_ + getX).reduce(_ + _) - } - } -} - -class TestClassWithoutDefaultConstructor(x: Int) extends Serializable { - def getX = x - - def run(): Int = { - var nonSer = new NonSerializable - return withSpark(new SparkContext("local", "test")) { sc => - val nums = sc.parallelize(Array(1, 2, 3, 4)) - nums.map(_ + getX).reduce(_ + _) - } - } -} - -// This class is not serializable, but we aren't using any of its fields in our -// closures, so they won't have a $outer pointing to it and should still work. -class TestClassWithoutFieldAccess { - var nonSer = new NonSerializable - - def run(): Int = { - var nonSer2 = new NonSerializable - var x = 5 - return withSpark(new SparkContext("local", "test")) { sc => - val nums = sc.parallelize(Array(1, 2, 3, 4)) - nums.map(_ + x).reduce(_ + _) - } - } -} - - -object TestObjectWithNesting { - def run(): Int = { - var nonSer = new NonSerializable - var answer = 0 - return withSpark(new SparkContext("local", "test")) { sc => - val nums = sc.parallelize(Array(1, 2, 3, 4)) - var y = 1 - for (i <- 1 to 4) { - var nonSer2 = new NonSerializable - var x = i - answer += nums.map(_ + x + y).reduce(_ + _) - } - answer - } - } -} - -class TestClassWithNesting(val y: Int) extends Serializable { - def getY = y - - def run(): Int = { - var nonSer = new NonSerializable - var answer = 0 - return withSpark(new SparkContext("local", "test")) { sc => - val nums = sc.parallelize(Array(1, 2, 3, 4)) - for (i <- 1 to 4) { - var nonSer2 = new NonSerializable - var x = i - answer += nums.map(_ + x + getY).reduce(_ + _) - } - answer - } - } -} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index b08aad1a6f..01a72d8401 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ import org.scalatest.time.SpanSugar._ +import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index ee89a7a387..af448fcb37 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import org.scalatest.FunSuite import SparkContext._ +import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object // for this because any local variables used in the task closures will rightfully diff --git a/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala deleted file mode 100644 index d7b23c93fe..0000000000 --- a/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala +++ /dev/null @@ -1,208 +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 - -import scala.collection.mutable - -import org.scalatest.FunSuite -import com.esotericsoftware.kryo._ - -import KryoTest._ - -class KryoSerializerSuite extends FunSuite with SharedSparkContext { - test("basic types") { - val ser = (new KryoSerializer).newInstance() - def check[T](t: T) { - assert(ser.deserialize[T](ser.serialize(t)) === t) - } - check(1) - check(1L) - check(1.0f) - check(1.0) - check(1.toByte) - check(1.toShort) - check("") - check("hello") - check(Integer.MAX_VALUE) - check(Integer.MIN_VALUE) - check(java.lang.Long.MAX_VALUE) - check(java.lang.Long.MIN_VALUE) - check[String](null) - check(Array(1, 2, 3)) - check(Array(1L, 2L, 3L)) - check(Array(1.0, 2.0, 3.0)) - check(Array(1.0f, 2.9f, 3.9f)) - check(Array("aaa", "bbb", "ccc")) - check(Array("aaa", "bbb", null)) - check(Array(true, false, true)) - check(Array('a', 'b', 'c')) - check(Array[Int]()) - check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) - } - - test("pairs") { - val ser = (new KryoSerializer).newInstance() - def check[T](t: T) { - assert(ser.deserialize[T](ser.serialize(t)) === t) - } - check((1, 1)) - check((1, 1L)) - check((1L, 1)) - check((1L, 1L)) - check((1.0, 1)) - check((1, 1.0)) - check((1.0, 1.0)) - check((1.0, 1L)) - check((1L, 1.0)) - check((1.0, 1L)) - check(("x", 1)) - check(("x", 1.0)) - check(("x", 1L)) - check((1, "x")) - check((1.0, "x")) - check((1L, "x")) - check(("x", "x")) - } - - test("Scala data structures") { - val ser = (new KryoSerializer).newInstance() - def check[T](t: T) { - assert(ser.deserialize[T](ser.serialize(t)) === t) - } - check(List[Int]()) - check(List[Int](1, 2, 3)) - check(List[String]()) - check(List[String]("x", "y", "z")) - check(None) - check(Some(1)) - check(Some("hi")) - check(mutable.ArrayBuffer(1, 2, 3)) - check(mutable.ArrayBuffer("1", "2", "3")) - check(mutable.Map()) - check(mutable.Map(1 -> "one", 2 -> "two")) - check(mutable.Map("one" -> 1, "two" -> 2)) - check(mutable.HashMap(1 -> "one", 2 -> "two")) - check(mutable.HashMap("one" -> 1, "two" -> 2)) - check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) - } - - test("custom registrator") { - import KryoTest._ - System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) - - val ser = (new KryoSerializer).newInstance() - def check[T](t: T) { - assert(ser.deserialize[T](ser.serialize(t)) === t) - } - - check(CaseClass(17, "hello")) - - val c1 = new ClassWithNoArgConstructor - c1.x = 32 - check(c1) - - val c2 = new ClassWithoutNoArgConstructor(47) - check(c2) - - val hashMap = new java.util.HashMap[String, String] - hashMap.put("foo", "bar") - check(hashMap) - - System.clearProperty("spark.kryo.registrator") - } - - test("kryo with collect") { - val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) - assert(control === result.toSeq) - } - - test("kryo with parallelize") { - val control = 1 :: 2 :: Nil - val result = sc.parallelize(control.map(new ClassWithoutNoArgConstructor(_))).map(_.x).collect() - assert (control === result.toSeq) - } - - test("kryo with parallelize for specialized tuples") { - assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).count === 3) - } - - test("kryo with parallelize for primitive arrays") { - assert (sc.parallelize( Array(1, 2, 3) ).count === 3) - } - - test("kryo with collect for specialized tuples") { - assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11)) - } - - test("kryo with reduce") { - val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) - .reduce((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x - assert(control.sum === result) - } - - // TODO: this still doesn't work - ignore("kryo with fold") { - val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) - .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x - assert(10 + control.sum === result) - } - - override def beforeAll() { - System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) - super.beforeAll() - } - - override def afterAll() { - super.afterAll() - System.clearProperty("spark.kryo.registrator") - System.clearProperty("spark.serializer") - } -} - -object KryoTest { - case class CaseClass(i: Int, s: String) {} - - class ClassWithNoArgConstructor { - var x: Int = 0 - override def equals(other: Any) = other match { - case c: ClassWithNoArgConstructor => x == c.x - case _ => false - } - } - - class ClassWithoutNoArgConstructor(val x: Int) { - override def equals(other: Any) = other match { - case c: ClassWithoutNoArgConstructor => x == c.x - case _ => false - } - } - - class MyRegistrator extends KryoRegistrator { - override def registerClasses(k: Kryo) { - k.register(classOf[CaseClass]) - k.register(classOf[ClassWithNoArgConstructor]) - k.register(classOf[ClassWithoutNoArgConstructor]) - k.register(classOf[java.util.HashMap[_, _]]) - } - } -} diff --git a/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala deleted file mode 100644 index f79752b34e..0000000000 --- a/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala +++ /dev/null @@ -1,299 +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 - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet - -import org.scalatest.FunSuite - -import com.google.common.io.Files -import org.apache.spark.SparkContext._ - - -class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { - test("groupByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with duplicates") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with negative key hash codes") { - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesForMinus1 = groups.find(_._1 == -1).get._2 - assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with many output partitions") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) - val groups = pairs.groupByKey(10).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("reduceByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with collectAsMap") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collectAsMap() - assert(sums.size === 2) - assert(sums(1) === 7) - assert(sums(2) === 1) - } - - test("reduceByKey with many output partitons") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_, 10).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with partitioner") { - val p = new Partitioner() { - def numPartitions = 2 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) - val sums = pairs.reduceByKey(_+_) - assert(sums.collect().toSet === Set((1, 4), (0, 1))) - assert(sums.partitioner === Some(p)) - // count the dependencies to make sure there is only 1 ShuffledRDD - val deps = new HashSet[RDD[_]]() - def visit(r: RDD[_]) { - for (dep <- r.dependencies) { - deps += dep.rdd - visit(dep.rdd) - } - } - visit(sums) - assert(deps.size === 2) // ShuffledRDD, ParallelCollection - } - - test("join") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (1, 'y')), - (1, (2, 'x')), - (1, (2, 'y')), - (1, (3, 'x')), - (1, (3, 'y')) - )) - } - - test("leftOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("rightOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - test("join with no matches") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - test("join with many output partitions") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2, 10).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("groupWith") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.groupWith(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) - )) - } - - test("zero-partition RDD") { - val emptyDir = Files.createTempDir() - val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.partitions.size == 0) - assert(file.collect().toList === Nil) - // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } - - test("keys and values") { - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) - assert(rdd.keys.collect().toList === List(1, 2)) - assert(rdd.values.collect().toList === List("a", "b")) - } - - test("default partitioner uses partition size") { - // specify 2000 partitions - val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) - // do a map, which loses the partitioner - val b = a.map(a => (a, (a * 2).toString)) - // then a group by, and see we didn't revert to 2 partitions - val c = b.groupByKey() - assert(c.partitions.size === 2000) - } - - test("default partitioner uses largest partitioner") { - val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) - val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) - val c = a.join(b) - assert(c.partitions.size === 2000) - } - - test("subtract") { - val a = sc.parallelize(Array(1, 2, 3), 2) - val b = sc.parallelize(Array(2, 3, 4), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set(1)) - assert(c.partitions.size === a.partitions.size) - } - - test("subtract with narrow dependency") { - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set((1, "a"), (3, "c"))) - // Ideally we could keep the original partitioner... - assert(c.partitioner === None) - } - - test("subtractByKey") { - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) - val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitions.size === a.partitions.size) - } - - test("subtractByKey with narrow dependency") { - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitioner.get === p) - } - - test("foldByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.foldByKey(0)(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("foldByKey with mutable result type") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() - // Fold the values using in-place mutation - val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() - assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) - // Check that the mutable objects in the original RDD were not changed - assert(bufs.collect().toSet === Set( - (1, ArrayBuffer(1)), - (1, ArrayBuffer(2)), - (1, ArrayBuffer(3)), - (1, ArrayBuffer(1)), - (2, ArrayBuffer(1)))) - } -} diff --git a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala index adbe805916..5a18dd13ff 100644 --- a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala @@ -2,7 +2,7 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.PartitionPruningRDD +import org.apache.spark.rdd.{RDD, PartitionPruningRDD} class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7669cf6fb1..7d938917f2 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark -import org.scalatest.FunSuite +import scala.math.abs import scala.collection.mutable.ArrayBuffer -import SparkContext._ + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ import org.apache.spark.util.StatCounter -import scala.math.abs +import org.apache.spark.rdd.RDD class PartitioningSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/RDDSuite.scala b/core/src/test/scala/org/apache/spark/RDDSuite.scala deleted file mode 100644 index 342ba8adb2..0000000000 --- a/core/src/test/scala/org/apache/spark/RDDSuite.scala +++ /dev/null @@ -1,389 +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 - -import scala.collection.mutable.HashMap -import org.scalatest.FunSuite -import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Span, Millis} -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd._ -import scala.collection.parallel.mutable - -class RDDSuite extends FunSuite with SharedSparkContext { - - test("basic operations") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - assert(nums.collect().toList === List(1, 2, 3, 4)) - val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) - assert(dups.distinct().count() === 4) - assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? - assert(dups.distinct.collect === dups.distinct().collect) - assert(dups.distinct(2).collect === dups.distinct().collect) - assert(nums.reduce(_ + _) === 10) - assert(nums.fold(0)(_ + _) === 10) - assert(nums.map(_.toString).collect().toList === List("1", "2", "3", "4")) - assert(nums.filter(_ > 2).collect().toList === List(3, 4)) - assert(nums.flatMap(x => 1 to x).collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4)) - assert(nums.union(nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) - assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) - assert(nums.collect({ case i if i >= 3 => i.toString }).collect().toList === List("3", "4")) - assert(nums.keyBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) - val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) - assert(partitionSums.collect().toList === List(3, 7)) - - val partitionSumsWithSplit = nums.mapPartitionsWithSplit { - case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) - } - assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7))) - - val partitionSumsWithIndex = nums.mapPartitionsWithIndex { - case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) - } - assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7))) - - intercept[UnsupportedOperationException] { - nums.filter(_ > 5).reduce(_ + _) - } - } - - test("SparkContext.union") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - assert(sc.union(nums).collect().toList === List(1, 2, 3, 4)) - assert(sc.union(nums, nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) - assert(sc.union(Seq(nums)).collect().toList === List(1, 2, 3, 4)) - assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) - } - - test("aggregate") { - val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) - type StringMap = HashMap[String, Int] - val emptyMap = new StringMap { - override def default(key: String): Int = 0 - } - val mergeElement: (StringMap, (String, Int)) => StringMap = (map, pair) => { - map(pair._1) += pair._2 - map - } - val mergeMaps: (StringMap, StringMap) => StringMap = (map1, map2) => { - for ((key, value) <- map2) { - map1(key) += value - } - map1 - } - val result = pairs.aggregate(emptyMap)(mergeElement, mergeMaps) - assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) - } - - test("basic caching") { - val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - assert(rdd.collect().toList === List(1, 2, 3, 4)) - assert(rdd.collect().toList === List(1, 2, 3, 4)) - assert(rdd.collect().toList === List(1, 2, 3, 4)) - } - - test("caching with failures") { - val onlySplit = new Partition { override def index: Int = 0 } - var shouldFail = true - val rdd = new RDD[Int](sc, Nil) { - override def getPartitions: Array[Partition] = Array(onlySplit) - override val getDependencies = List[Dependency[_]]() - override def compute(split: Partition, context: TaskContext): Iterator[Int] = { - if (shouldFail) { - throw new Exception("injected failure") - } else { - return Array(1, 2, 3, 4).iterator - } - } - }.cache() - val thrown = intercept[Exception]{ - rdd.collect() - } - assert(thrown.getMessage.contains("injected failure")) - shouldFail = false - assert(rdd.collect().toList === List(1, 2, 3, 4)) - } - - test("empty RDD") { - val empty = new EmptyRDD[Int](sc) - assert(empty.count === 0) - assert(empty.collect().size === 0) - - val thrown = intercept[UnsupportedOperationException]{ - empty.reduce(_+_) - } - assert(thrown.getMessage.contains("empty")) - - val emptyKv = new EmptyRDD[(Int, Int)](sc) - val rdd = sc.parallelize(1 to 2, 2).map(x => (x, x)) - assert(rdd.join(emptyKv).collect().size === 0) - assert(rdd.rightOuterJoin(emptyKv).collect().size === 0) - assert(rdd.leftOuterJoin(emptyKv).collect().size === 2) - assert(rdd.cogroup(emptyKv).collect().size === 2) - assert(rdd.union(emptyKv).collect().size === 2) - } - - test("cogrouped RDDs") { - val data = sc.parallelize(1 to 10, 10) - - val coalesced1 = data.coalesce(2) - assert(coalesced1.collect().toList === (1 to 10).toList) - assert(coalesced1.glom().collect().map(_.toList).toList === - List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10))) - - // Check that the narrow dependency is also specified correctly - assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList === - List(0, 1, 2, 3, 4)) - assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === - List(5, 6, 7, 8, 9)) - - val coalesced2 = data.coalesce(3) - assert(coalesced2.collect().toList === (1 to 10).toList) - assert(coalesced2.glom().collect().map(_.toList).toList === - List(List(1, 2, 3), List(4, 5, 6), List(7, 8, 9, 10))) - - val coalesced3 = data.coalesce(10) - assert(coalesced3.collect().toList === (1 to 10).toList) - assert(coalesced3.glom().collect().map(_.toList).toList === - (1 to 10).map(x => List(x)).toList) - - // If we try to coalesce into more partitions than the original RDD, it should just - // keep the original number of partitions. - val coalesced4 = data.coalesce(20) - assert(coalesced4.collect().toList === (1 to 10).toList) - assert(coalesced4.glom().collect().map(_.toList).toList === - (1 to 10).map(x => List(x)).toList) - - // we can optionally shuffle to keep the upstream parallel - val coalesced5 = data.coalesce(1, shuffle = true) - assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] != - null) - } - test("cogrouped RDDs with locality") { - val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) - val coal3 = data3.coalesce(3) - val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation) - assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") - - // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 - val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) - val coalesced1 = data.coalesce(3) - assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") - - val splits = coalesced1.glom().collect().map(_.toList).toList - assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) - - assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") - - // If we try to coalesce into more partitions than the original RDD, it should just - // keep the original number of partitions. - val coalesced4 = data.coalesce(20) - val listOfLists = coalesced4.glom().collect().map(_.toList).toList - val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } - assert( sortedList === (1 to 9). - map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") - } - - test("cogrouped RDDs with locality, large scale (10K partitions)") { - // large scale experiment - import collection.mutable - val rnd = scala.util.Random - val partitions = 10000 - val numMachines = 50 - val machines = mutable.ListBuffer[String]() - (1 to numMachines).foreach(machines += "m"+_) - - val blocks = (1 to partitions).map(i => - { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) - - val data2 = sc.makeRDD(blocks) - val coalesced2 = data2.coalesce(numMachines*2) - - // test that you get over 90% locality in each group - val minLocality = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") - - // test that the groups are load balanced with 100 +/- 20 elements in each - val maxImbalance = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) - .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) - - val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs - val coalesced3 = data3.coalesce(numMachines*2) - val minLocality2 = coalesced3.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.)((perc, loc) => math.min(perc,loc)) - assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.).toInt + "%") - } - - test("zipped RDDs") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val zipped = nums.zip(nums.map(_ + 1.0)) - assert(zipped.glom().map(_.toList).collect().toList === - List(List((1, 2.0), (2, 3.0)), List((3, 4.0), (4, 5.0)))) - - intercept[IllegalArgumentException] { - nums.zip(sc.parallelize(1 to 4, 1)).collect() - } - } - - test("partition pruning") { - val data = sc.parallelize(1 to 10, 10) - // Note that split number starts from 0, so > 8 means only 10th partition left. - val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) - assert(prunedRdd.partitions.size === 1) - val prunedData = prunedRdd.collect() - assert(prunedData.size === 1) - assert(prunedData(0) === 10) - } - - test("mapWith") { - import java.util.Random - val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) - val randoms = ones.mapWith( - (index: Int) => new Random(index + 42)) - {(t: Int, prng: Random) => prng.nextDouble * t}.collect() - val prn42_3 = { - val prng42 = new Random(42) - prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble() - } - val prn43_3 = { - val prng43 = new Random(43) - prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble() - } - assert(randoms(2) === prn42_3) - assert(randoms(5) === prn43_3) - } - - test("flatMapWith") { - import java.util.Random - val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) - val randoms = ones.flatMapWith( - (index: Int) => new Random(index + 42)) - {(t: Int, prng: Random) => - val random = prng.nextDouble() - Seq(random * t, random * t * 10)}. - collect() - val prn42_3 = { - val prng42 = new Random(42) - prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble() - } - val prn43_3 = { - val prng43 = new Random(43) - prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble() - } - assert(randoms(5) === prn42_3 * 10) - assert(randoms(11) === prn43_3 * 10) - } - - test("filterWith") { - import java.util.Random - val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2) - val sample = ints.filterWith( - (index: Int) => new Random(index + 42)) - {(t: Int, prng: Random) => prng.nextInt(3) == 0}. - collect() - val checkSample = { - val prng42 = new Random(42) - val prng43 = new Random(43) - Array(1, 2, 3, 4, 5, 6).filter{i => - if (i < 4) 0 == prng42.nextInt(3) - else 0 == prng43.nextInt(3)} - } - assert(sample.size === checkSample.size) - for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) - } - - test("top with predefined ordering") { - val nums = Array.range(1, 100000) - val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) - val topK = ints.top(5) - assert(topK.size === 5) - assert(topK === nums.reverse.take(5)) - } - - test("top with custom ordering") { - val words = Vector("a", "b", "c", "d") - implicit val ord = implicitly[Ordering[String]].reverse - val rdd = sc.makeRDD(words, 2) - val topK = rdd.top(2) - assert(topK.size === 2) - assert(topK.sorted === Array("b", "a")) - } - - test("takeOrdered with predefined ordering") { - val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) - val rdd = sc.makeRDD(nums, 2) - val sortedLowerK = rdd.takeOrdered(5) - assert(sortedLowerK.size === 5) - assert(sortedLowerK === Array(1, 2, 3, 4, 5)) - } - - test("takeOrdered with custom ordering") { - val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) - implicit val ord = implicitly[Ordering[Int]].reverse - val rdd = sc.makeRDD(nums, 2) - val sortedTopK = rdd.takeOrdered(5) - assert(sortedTopK.size === 5) - assert(sortedTopK === Array(10, 9, 8, 7, 6)) - assert(sortedTopK === nums.sorted(ord).take(5)) - } - - test("takeSample") { - val data = sc.parallelize(1 to 100, 2) - for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=false, 20, seed) - assert(sample.size === 20) // Got exactly 20 elements - assert(sample.toSet.size === 20) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") - } - for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=false, 200, seed) - assert(sample.size === 100) // Got only 100 elements - assert(sample.toSet.size === 100) // Elements are distinct - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") - } - for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 20, seed) - assert(sample.size === 20) // Got exactly 20 elements - assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") - } - for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 100, seed) - assert(sample.size === 100) // Got exactly 100 elements - // Chance of getting all distinct elements is astronomically low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") - } - for (seed <- 1 to 5) { - val sample = data.takeSample(withReplacement=true, 200, seed) - assert(sample.size === 200) // Got exactly 200 elements - // Chance of getting all distinct elements is still quite low, so test we got < 100 - assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") - } - } - - test("runJob on an invalid partition") { - intercept[IllegalArgumentException] { - sc.runJob(sc.parallelize(1 to 10, 2), {iter: Iterator[Int] => iter.size}, Seq(0, 1, 2), false) - } - } -} diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 357175e89e..db717865db 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -22,8 +22,9 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass -import org.apache.spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} +import org.apache.spark.rdd.{RDD, SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} import org.apache.spark.util.MutablePair +import org.apache.spark.serializer.KryoSerializer class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala deleted file mode 100644 index 214ac74898..0000000000 --- a/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala +++ /dev/null @@ -1,164 +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 - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfterAll -import org.scalatest.PrivateMethodTester - -class DummyClass1 {} - -class DummyClass2 { - val x: Int = 0 -} - -class DummyClass3 { - val x: Int = 0 - val y: Double = 0.0 -} - -class DummyClass4(val d: DummyClass3) { - val x: Int = 0 -} - -object DummyString { - def apply(str: String) : DummyString = new DummyString(str.toArray) -} -class DummyString(val arr: Array[Char]) { - override val hashCode: Int = 0 - // JDK-7 has an extra hash32 field http://hg.openjdk.java.net/jdk7u/jdk7u6/jdk/rev/11987e85555f - @transient val hash32: Int = 0 -} - -class SizeEstimatorSuite - extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { - - var oldArch: String = _ - var oldOops: String = _ - - override def beforeAll() { - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - } - - override def afterAll() { - resetOrClear("os.arch", oldArch) - resetOrClear("spark.test.useCompressedOops", oldOops) - } - - test("simple classes") { - assert(SizeEstimator.estimate(new DummyClass1) === 16) - assert(SizeEstimator.estimate(new DummyClass2) === 16) - assert(SizeEstimator.estimate(new DummyClass3) === 24) - assert(SizeEstimator.estimate(new DummyClass4(null)) === 24) - assert(SizeEstimator.estimate(new DummyClass4(new DummyClass3)) === 48) - } - - // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors - // (Sun vs IBM). Use a DummyString class to make tests deterministic. - test("strings") { - assert(SizeEstimator.estimate(DummyString("")) === 40) - assert(SizeEstimator.estimate(DummyString("a")) === 48) - assert(SizeEstimator.estimate(DummyString("ab")) === 48) - assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) - } - - test("primitive arrays") { - assert(SizeEstimator.estimate(new Array[Byte](10)) === 32) - assert(SizeEstimator.estimate(new Array[Char](10)) === 40) - assert(SizeEstimator.estimate(new Array[Short](10)) === 40) - assert(SizeEstimator.estimate(new Array[Int](10)) === 56) - assert(SizeEstimator.estimate(new Array[Long](10)) === 96) - assert(SizeEstimator.estimate(new Array[Float](10)) === 56) - assert(SizeEstimator.estimate(new Array[Double](10)) === 96) - assert(SizeEstimator.estimate(new Array[Int](1000)) === 4016) - assert(SizeEstimator.estimate(new Array[Long](1000)) === 8016) - } - - test("object arrays") { - // Arrays containing nulls should just have one pointer per element - assert(SizeEstimator.estimate(new Array[String](10)) === 56) - assert(SizeEstimator.estimate(new Array[AnyRef](10)) === 56) - - // For object arrays with non-null elements, each object should take one pointer plus - // however many bytes that class takes. (Note that Array.fill calls the code in its - // second parameter separately for each object, so we get distinct objects.) - assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)) === 216) - assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)) === 216) - assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)) === 296) - assert(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)) === 56) - - // Past size 100, our samples 100 elements, but we should still get the right size. - assert(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)) === 28016) - - // If an array contains the *same* element many times, we should only count it once. - val d1 = new DummyClass1 - assert(SizeEstimator.estimate(Array.fill(10)(d1)) === 72) // 10 pointers plus 8-byte object - assert(SizeEstimator.estimate(Array.fill(100)(d1)) === 432) // 100 pointers plus 8-byte object - - // Same thing with huge array containing the same element many times. Note that this won't - // return exactly 4032 because it can't tell that *all* the elements will equal the first - // one it samples, but it should be close to that. - - // TODO: If we sample 100 elements, this should always be 4176 ? - val estimatedSize = SizeEstimator.estimate(Array.fill(1000)(d1)) - assert(estimatedSize >= 4000, "Estimated size " + estimatedSize + " should be more than 4000") - assert(estimatedSize <= 4200, "Estimated size " + estimatedSize + " should be less than 4100") - } - - test("32-bit arch") { - val arch = System.setProperty("os.arch", "x86") - - val initialize = PrivateMethod[Unit]('initialize) - SizeEstimator invokePrivate initialize() - - assert(SizeEstimator.estimate(DummyString("")) === 40) - assert(SizeEstimator.estimate(DummyString("a")) === 48) - assert(SizeEstimator.estimate(DummyString("ab")) === 48) - assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) - - resetOrClear("os.arch", arch) - } - - // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors - // (Sun vs IBM). Use a DummyString class to make tests deterministic. - test("64-bit arch with no compressed oops") { - val arch = System.setProperty("os.arch", "amd64") - val oops = System.setProperty("spark.test.useCompressedOops", "false") - - val initialize = PrivateMethod[Unit]('initialize) - SizeEstimator invokePrivate initialize() - - assert(SizeEstimator.estimate(DummyString("")) === 56) - assert(SizeEstimator.estimate(DummyString("a")) === 64) - assert(SizeEstimator.estimate(DummyString("ab")) === 64) - assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 72) - - resetOrClear("os.arch", arch) - resetOrClear("spark.test.useCompressedOops", oops) - } - - def resetOrClear(prop: String, oldValue: String) { - if (oldValue != null) { - System.setProperty(prop, oldValue) - } else { - System.clearProperty(prop) - } - } -} diff --git a/core/src/test/scala/org/apache/spark/SortingSuite.scala b/core/src/test/scala/org/apache/spark/SortingSuite.scala deleted file mode 100644 index f4fa9511dd..0000000000 --- a/core/src/test/scala/org/apache/spark/SortingSuite.scala +++ /dev/null @@ -1,123 +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 - -import org.scalatest.FunSuite -import org.scalatest.BeforeAndAfter -import org.scalatest.matchers.ShouldMatchers -import SparkContext._ - -class SortingSuite extends FunSuite with SharedSparkContext with ShouldMatchers with Logging { - - test("sortByKey") { - val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) - assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) - } - - test("large array") { - val rand = new scala.util.Random() - val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 2) - val sorted = pairs.sortByKey() - assert(sorted.partitions.size === 2) - assert(sorted.collect() === pairArr.sortBy(_._1)) - } - - test("large array with one split") { - val rand = new scala.util.Random() - val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 2) - val sorted = pairs.sortByKey(true, 1) - assert(sorted.partitions.size === 1) - assert(sorted.collect() === pairArr.sortBy(_._1)) - } - - test("large array with many partitions") { - val rand = new scala.util.Random() - val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 2) - val sorted = pairs.sortByKey(true, 20) - assert(sorted.partitions.size === 20) - assert(sorted.collect() === pairArr.sortBy(_._1)) - } - - test("sort descending") { - val rand = new scala.util.Random() - val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 2) - assert(pairs.sortByKey(false).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) - } - - test("sort descending with one split") { - val rand = new scala.util.Random() - val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 1) - assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) - } - - test("sort descending with many partitions") { - val rand = new scala.util.Random() - val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 2) - assert(pairs.sortByKey(false, 20).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) - } - - test("more partitions than elements") { - val rand = new scala.util.Random() - val pairArr = Array.fill(10) { (rand.nextInt(), rand.nextInt()) } - val pairs = sc.parallelize(pairArr, 30) - assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) - } - - test("empty RDD") { - val pairArr = new Array[(Int, Int)](0) - val pairs = sc.parallelize(pairArr, 2) - assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) - } - - test("partition balancing") { - val pairArr = (1 to 1000).map(x => (x, x)).toArray - val sorted = sc.parallelize(pairArr, 4).sortByKey() - assert(sorted.collect() === pairArr.sortBy(_._1)) - val partitions = sorted.collectPartitions() - logInfo("Partition lengths: " + partitions.map(_.length).mkString(", ")) - partitions(0).length should be > 180 - partitions(1).length should be > 180 - partitions(2).length should be > 180 - partitions(3).length should be > 180 - partitions(0).last should be < partitions(1).head - partitions(1).last should be < partitions(2).head - partitions(2).last should be < partitions(3).head - } - - test("partition balancing for descending sort") { - val pairArr = (1 to 1000).map(x => (x, x)).toArray - val sorted = sc.parallelize(pairArr, 4).sortByKey(false) - assert(sorted.collect() === pairArr.sortBy(_._1).reverse) - val partitions = sorted.collectPartitions() - logInfo("partition lengths: " + partitions.map(_.length).mkString(", ")) - partitions(0).length should be > 180 - partitions(1).length should be > 180 - partitions(2).length should be > 180 - partitions(3).length should be > 180 - partitions(0).last should be > partitions(1).head - partitions(1).last should be > partitions(2).head - partitions(2).last should be > partitions(3).head - } -} - diff --git a/core/src/test/scala/org/apache/spark/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/UtilsSuite.scala deleted file mode 100644 index 3a908720a8..0000000000 --- a/core/src/test/scala/org/apache/spark/UtilsSuite.scala +++ /dev/null @@ -1,139 +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 - -import com.google.common.base.Charsets -import com.google.common.io.Files -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File} -import org.scalatest.FunSuite -import org.apache.commons.io.FileUtils -import scala.util.Random - -class UtilsSuite extends FunSuite { - - test("bytesToString") { - assert(Utils.bytesToString(10) === "10.0 B") - assert(Utils.bytesToString(1500) === "1500.0 B") - assert(Utils.bytesToString(2000000) === "1953.1 KB") - assert(Utils.bytesToString(2097152) === "2.0 MB") - assert(Utils.bytesToString(2306867) === "2.2 MB") - assert(Utils.bytesToString(5368709120L) === "5.0 GB") - assert(Utils.bytesToString(5L * 1024L * 1024L * 1024L * 1024L) === "5.0 TB") - } - - test("copyStream") { - //input array initialization - val bytes = Array.ofDim[Byte](9000) - Random.nextBytes(bytes) - - val os = new ByteArrayOutputStream() - Utils.copyStream(new ByteArrayInputStream(bytes), os) - - assert(os.toByteArray.toList.equals(bytes.toList)) - } - - test("memoryStringToMb") { - assert(Utils.memoryStringToMb("1") === 0) - assert(Utils.memoryStringToMb("1048575") === 0) - assert(Utils.memoryStringToMb("3145728") === 3) - - assert(Utils.memoryStringToMb("1024k") === 1) - assert(Utils.memoryStringToMb("5000k") === 4) - assert(Utils.memoryStringToMb("4024k") === Utils.memoryStringToMb("4024K")) - - assert(Utils.memoryStringToMb("1024m") === 1024) - assert(Utils.memoryStringToMb("5000m") === 5000) - assert(Utils.memoryStringToMb("4024m") === Utils.memoryStringToMb("4024M")) - - assert(Utils.memoryStringToMb("2g") === 2048) - assert(Utils.memoryStringToMb("3g") === Utils.memoryStringToMb("3G")) - - assert(Utils.memoryStringToMb("2t") === 2097152) - assert(Utils.memoryStringToMb("3t") === Utils.memoryStringToMb("3T")) - } - - test("splitCommandString") { - assert(Utils.splitCommandString("") === Seq()) - assert(Utils.splitCommandString("a") === Seq("a")) - assert(Utils.splitCommandString("aaa") === Seq("aaa")) - assert(Utils.splitCommandString("a b c") === Seq("a", "b", "c")) - assert(Utils.splitCommandString(" a b\t c ") === Seq("a", "b", "c")) - assert(Utils.splitCommandString("a 'b c'") === Seq("a", "b c")) - assert(Utils.splitCommandString("a 'b c' d") === Seq("a", "b c", "d")) - assert(Utils.splitCommandString("'b c'") === Seq("b c")) - assert(Utils.splitCommandString("a \"b c\"") === Seq("a", "b c")) - assert(Utils.splitCommandString("a \"b c\" d") === Seq("a", "b c", "d")) - assert(Utils.splitCommandString("\"b c\"") === Seq("b c")) - assert(Utils.splitCommandString("a 'b\" c' \"d' e\"") === Seq("a", "b\" c", "d' e")) - assert(Utils.splitCommandString("a\t'b\nc'\nd") === Seq("a", "b\nc", "d")) - assert(Utils.splitCommandString("a \"b\\\\c\"") === Seq("a", "b\\c")) - assert(Utils.splitCommandString("a \"b\\\"c\"") === Seq("a", "b\"c")) - assert(Utils.splitCommandString("a 'b\\\"c'") === Seq("a", "b\\\"c")) - assert(Utils.splitCommandString("'a'b") === Seq("ab")) - assert(Utils.splitCommandString("'a''b'") === Seq("ab")) - assert(Utils.splitCommandString("\"a\"b") === Seq("ab")) - assert(Utils.splitCommandString("\"a\"\"b\"") === Seq("ab")) - assert(Utils.splitCommandString("''") === Seq("")) - assert(Utils.splitCommandString("\"\"") === Seq("")) - } - - test("string formatting of time durations") { - val second = 1000 - val minute = second * 60 - val hour = minute * 60 - def str = Utils.msDurationToString(_) - - assert(str(123) === "123 ms") - assert(str(second) === "1.0 s") - assert(str(second + 462) === "1.5 s") - assert(str(hour) === "1.00 h") - assert(str(minute) === "1.0 m") - assert(str(minute + 4 * second + 34) === "1.1 m") - assert(str(10 * hour + minute + 4 * second) === "10.02 h") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") - } - - test("reading offset bytes of a file") { - val tmpDir2 = Files.createTempDir() - val f1Path = tmpDir2 + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) - f1.close() - - // Read first few bytes - assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") - - // Read some middle bytes - assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") - - // Read last few bytes - assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - - // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - - // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - - // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") - - FileUtils.deleteDirectory(tmpDir2) - } -} - diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala new file mode 100644 index 0000000000..31f97fc139 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -0,0 +1,300 @@ +/* + * 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.rdd + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import org.scalatest.FunSuite + +import com.google.common.io.Files +import org.apache.spark.SparkContext._ +import org.apache.spark.{Partitioner, SharedSparkContext} + + +class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) + val groups = pairs.groupByKey(10).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_, 10).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = new HashSet[RDD[_]]() + def visit(r: RDD[_]) { + for (dep <- r.dependencies) { + deps += dep.rdd + visit(dep.rdd) + } + } + visit(sums) + assert(deps.size === 2) // ShuffledRDD, ParallelCollection + } + + test("join") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("join all-to-all") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("join with no matches") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2, 10).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.groupWith(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), + (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), + (3, (ArrayBuffer(1), ArrayBuffer())), + (4, (ArrayBuffer(), ArrayBuffer('w'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + test("default partitioner uses largest partitioner") { + val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) + val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) + val c = a.join(b) + assert(c.partitions.size === 2000) + } + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + test("subtractByKey with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitioner.get === p) + } + + test("foldByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala new file mode 100644 index 0000000000..adc971050e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -0,0 +1,391 @@ +/* + * 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.rdd + +import scala.collection.mutable.HashMap +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.time.{Span, Millis} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ +import scala.collection.parallel.mutable +import org.apache.spark._ +import org.apache.spark.rdd.CoalescedRDDPartition + +class RDDSuite extends FunSuite with SharedSparkContext { + + test("basic operations") { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + assert(nums.collect().toList === List(1, 2, 3, 4)) + val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) + assert(dups.distinct().count() === 4) + assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? + assert(dups.distinct.collect === dups.distinct().collect) + assert(dups.distinct(2).collect === dups.distinct().collect) + assert(nums.reduce(_ + _) === 10) + assert(nums.fold(0)(_ + _) === 10) + assert(nums.map(_.toString).collect().toList === List("1", "2", "3", "4")) + assert(nums.filter(_ > 2).collect().toList === List(3, 4)) + assert(nums.flatMap(x => 1 to x).collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4)) + assert(nums.union(nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) + assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) + assert(nums.collect({ case i if i >= 3 => i.toString }).collect().toList === List("3", "4")) + assert(nums.keyBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) + val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) + assert(partitionSums.collect().toList === List(3, 7)) + + val partitionSumsWithSplit = nums.mapPartitionsWithSplit { + case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) + } + assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7))) + + val partitionSumsWithIndex = nums.mapPartitionsWithIndex { + case(split, iter) => Iterator((split, iter.reduceLeft(_ + _))) + } + assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7))) + + intercept[UnsupportedOperationException] { + nums.filter(_ > 5).reduce(_ + _) + } + } + + test("SparkContext.union") { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + assert(sc.union(nums).collect().toList === List(1, 2, 3, 4)) + assert(sc.union(nums, nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) + assert(sc.union(Seq(nums)).collect().toList === List(1, 2, 3, 4)) + assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) + } + + test("aggregate") { + val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) + type StringMap = HashMap[String, Int] + val emptyMap = new StringMap { + override def default(key: String): Int = 0 + } + val mergeElement: (StringMap, (String, Int)) => StringMap = (map, pair) => { + map(pair._1) += pair._2 + map + } + val mergeMaps: (StringMap, StringMap) => StringMap = (map1, map2) => { + for ((key, value) <- map2) { + map1(key) += value + } + map1 + } + val result = pairs.aggregate(emptyMap)(mergeElement, mergeMaps) + assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) + } + + test("basic caching") { + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(rdd.collect().toList === List(1, 2, 3, 4)) + } + + test("caching with failures") { + val onlySplit = new Partition { override def index: Int = 0 } + var shouldFail = true + val rdd = new RDD[Int](sc, Nil) { + override def getPartitions: Array[Partition] = Array(onlySplit) + override val getDependencies = List[Dependency[_]]() + override def compute(split: Partition, context: TaskContext): Iterator[Int] = { + if (shouldFail) { + throw new Exception("injected failure") + } else { + return Array(1, 2, 3, 4).iterator + } + } + }.cache() + val thrown = intercept[Exception]{ + rdd.collect() + } + assert(thrown.getMessage.contains("injected failure")) + shouldFail = false + assert(rdd.collect().toList === List(1, 2, 3, 4)) + } + + test("empty RDD") { + val empty = new EmptyRDD[Int](sc) + assert(empty.count === 0) + assert(empty.collect().size === 0) + + val thrown = intercept[UnsupportedOperationException]{ + empty.reduce(_+_) + } + assert(thrown.getMessage.contains("empty")) + + val emptyKv = new EmptyRDD[(Int, Int)](sc) + val rdd = sc.parallelize(1 to 2, 2).map(x => (x, x)) + assert(rdd.join(emptyKv).collect().size === 0) + assert(rdd.rightOuterJoin(emptyKv).collect().size === 0) + assert(rdd.leftOuterJoin(emptyKv).collect().size === 2) + assert(rdd.cogroup(emptyKv).collect().size === 2) + assert(rdd.union(emptyKv).collect().size === 2) + } + + test("cogrouped RDDs") { + val data = sc.parallelize(1 to 10, 10) + + val coalesced1 = data.coalesce(2) + assert(coalesced1.collect().toList === (1 to 10).toList) + assert(coalesced1.glom().collect().map(_.toList).toList === + List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10))) + + // Check that the narrow dependency is also specified correctly + assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList === + List(0, 1, 2, 3, 4)) + assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === + List(5, 6, 7, 8, 9)) + + val coalesced2 = data.coalesce(3) + assert(coalesced2.collect().toList === (1 to 10).toList) + assert(coalesced2.glom().collect().map(_.toList).toList === + List(List(1, 2, 3), List(4, 5, 6), List(7, 8, 9, 10))) + + val coalesced3 = data.coalesce(10) + assert(coalesced3.collect().toList === (1 to 10).toList) + assert(coalesced3.glom().collect().map(_.toList).toList === + (1 to 10).map(x => List(x)).toList) + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + assert(coalesced4.collect().toList === (1 to 10).toList) + assert(coalesced4.glom().collect().map(_.toList).toList === + (1 to 10).map(x => List(x)).toList) + + // we can optionally shuffle to keep the upstream parallel + val coalesced5 = data.coalesce(1, shuffle = true) + assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] != + null) + } + test("cogrouped RDDs with locality") { + val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) + val coal3 = data3.coalesce(3) + val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation) + assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") + + // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 + val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) + val coalesced1 = data.coalesce(3) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") + + val splits = coalesced1.glom().collect().map(_.toList).toList + assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) + + assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + val listOfLists = coalesced4.glom().collect().map(_.toList).toList + val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } + assert( sortedList === (1 to 9). + map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") + } + + test("cogrouped RDDs with locality, large scale (10K partitions)") { + // large scale experiment + import collection.mutable + val rnd = scala.util.Random + val partitions = 10000 + val numMachines = 50 + val machines = mutable.ListBuffer[String]() + (1 to numMachines).foreach(machines += "m"+_) + + val blocks = (1 to partitions).map(i => + { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) + + val data2 = sc.makeRDD(blocks) + val coalesced2 = data2.coalesce(numMachines*2) + + // test that you get over 90% locality in each group + val minLocality = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.)((perc, loc) => math.min(perc,loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) + assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + + val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs + val coalesced3 = data3.coalesce(numMachines*2) + val minLocality2 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.)((perc, loc) => math.min(perc,loc)) + assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + + (minLocality2*100.).toInt + "%") + } + + test("zipped RDDs") { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val zipped = nums.zip(nums.map(_ + 1.0)) + assert(zipped.glom().map(_.toList).collect().toList === + List(List((1, 2.0), (2, 3.0)), List((3, 4.0), (4, 5.0)))) + + intercept[IllegalArgumentException] { + nums.zip(sc.parallelize(1 to 4, 1)).collect() + } + } + + test("partition pruning") { + val data = sc.parallelize(1 to 10, 10) + // Note that split number starts from 0, so > 8 means only 10th partition left. + val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) + assert(prunedRdd.partitions.size === 1) + val prunedData = prunedRdd.collect() + assert(prunedData.size === 1) + assert(prunedData(0) === 10) + } + + test("mapWith") { + import java.util.Random + val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) + val randoms = ones.mapWith( + (index: Int) => new Random(index + 42)) + {(t: Int, prng: Random) => prng.nextDouble * t}.collect() + val prn42_3 = { + val prng42 = new Random(42) + prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble() + } + val prn43_3 = { + val prng43 = new Random(43) + prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble() + } + assert(randoms(2) === prn42_3) + assert(randoms(5) === prn43_3) + } + + test("flatMapWith") { + import java.util.Random + val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) + val randoms = ones.flatMapWith( + (index: Int) => new Random(index + 42)) + {(t: Int, prng: Random) => + val random = prng.nextDouble() + Seq(random * t, random * t * 10)}. + collect() + val prn42_3 = { + val prng42 = new Random(42) + prng42.nextDouble(); prng42.nextDouble(); prng42.nextDouble() + } + val prn43_3 = { + val prng43 = new Random(43) + prng43.nextDouble(); prng43.nextDouble(); prng43.nextDouble() + } + assert(randoms(5) === prn42_3 * 10) + assert(randoms(11) === prn43_3 * 10) + } + + test("filterWith") { + import java.util.Random + val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2) + val sample = ints.filterWith( + (index: Int) => new Random(index + 42)) + {(t: Int, prng: Random) => prng.nextInt(3) == 0}. + collect() + val checkSample = { + val prng42 = new Random(42) + val prng43 = new Random(43) + Array(1, 2, 3, 4, 5, 6).filter{i => + if (i < 4) 0 == prng42.nextInt(3) + else 0 == prng43.nextInt(3)} + } + assert(sample.size === checkSample.size) + for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) + } + + test("top with predefined ordering") { + val nums = Array.range(1, 100000) + val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) + val topK = ints.top(5) + assert(topK.size === 5) + assert(topK === nums.reverse.take(5)) + } + + test("top with custom ordering") { + val words = Vector("a", "b", "c", "d") + implicit val ord = implicitly[Ordering[String]].reverse + val rdd = sc.makeRDD(words, 2) + val topK = rdd.top(2) + assert(topK.size === 2) + assert(topK.sorted === Array("b", "a")) + } + + test("takeOrdered with predefined ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val rdd = sc.makeRDD(nums, 2) + val sortedLowerK = rdd.takeOrdered(5) + assert(sortedLowerK.size === 5) + assert(sortedLowerK === Array(1, 2, 3, 4, 5)) + } + + test("takeOrdered with custom ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + implicit val ord = implicitly[Ordering[Int]].reverse + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(10, 9, 8, 7, 6)) + assert(sortedTopK === nums.sorted(ord).take(5)) + } + + test("takeSample") { + val data = sc.parallelize(1 to 100, 2) + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=false, 20, seed) + assert(sample.size === 20) // Got exactly 20 elements + assert(sample.toSet.size === 20) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=false, 200, seed) + assert(sample.size === 100) // Got only 100 elements + assert(sample.toSet.size === 100) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 20, seed) + assert(sample.size === 20) // Got exactly 20 elements + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 100, seed) + assert(sample.size === 100) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 200, seed) + assert(sample.size === 200) // Got exactly 200 elements + // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + } + } + + test("runJob on an invalid partition") { + intercept[IllegalArgumentException] { + sc.runJob(sc.parallelize(1 to 10, 2), {iter: Iterator[Int] => iter.size}, Seq(0, 1, 2), false) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala new file mode 100644 index 0000000000..2f7bd370fc --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -0,0 +1,125 @@ +/* + * 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.rdd + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter +import org.scalatest.matchers.ShouldMatchers + +import org.apache.spark.{Logging, SharedSparkContext} +import org.apache.spark.SparkContext._ + +class SortingSuite extends FunSuite with SharedSparkContext with ShouldMatchers with Logging { + + test("sortByKey") { + val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) + assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) + } + + test("large array") { + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + val sorted = pairs.sortByKey() + assert(sorted.partitions.size === 2) + assert(sorted.collect() === pairArr.sortBy(_._1)) + } + + test("large array with one split") { + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + val sorted = pairs.sortByKey(true, 1) + assert(sorted.partitions.size === 1) + assert(sorted.collect() === pairArr.sortBy(_._1)) + } + + test("large array with many partitions") { + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + val sorted = pairs.sortByKey(true, 20) + assert(sorted.partitions.size === 20) + assert(sorted.collect() === pairArr.sortBy(_._1)) + } + + test("sort descending") { + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + assert(pairs.sortByKey(false).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) + } + + test("sort descending with one split") { + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 1) + assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) + } + + test("sort descending with many partitions") { + val rand = new scala.util.Random() + val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 2) + assert(pairs.sortByKey(false, 20).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) + } + + test("more partitions than elements") { + val rand = new scala.util.Random() + val pairArr = Array.fill(10) { (rand.nextInt(), rand.nextInt()) } + val pairs = sc.parallelize(pairArr, 30) + assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) + } + + test("empty RDD") { + val pairArr = new Array[(Int, Int)](0) + val pairs = sc.parallelize(pairArr, 2) + assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) + } + + test("partition balancing") { + val pairArr = (1 to 1000).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 4).sortByKey() + assert(sorted.collect() === pairArr.sortBy(_._1)) + val partitions = sorted.collectPartitions() + logInfo("Partition lengths: " + partitions.map(_.length).mkString(", ")) + partitions(0).length should be > 180 + partitions(1).length should be > 180 + partitions(2).length should be > 180 + partitions(3).length should be > 180 + partitions(0).last should be < partitions(1).head + partitions(1).last should be < partitions(2).head + partitions(2).last should be < partitions(3).head + } + + test("partition balancing for descending sort") { + val pairArr = (1 to 1000).map(x => (x, x)).toArray + val sorted = sc.parallelize(pairArr, 4).sortByKey(false) + assert(sorted.collect() === pairArr.sortBy(_._1).reverse) + val partitions = sorted.collectPartitions() + logInfo("partition lengths: " + partitions.map(_.length).mkString(", ")) + partitions(0).length should be > 180 + partitions(1).length should be > 180 + partitions(2).length should be > 180 + partitions(3).length should be > 180 + partitions(0).last should be > partitions(1).head + partitions(1).last should be > partitions(2).head + partitions(2).last should be > partitions(3).head + } +} + diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 94df282b28..94f66c94c6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.LocalSparkContext import org.apache.spark.MapOutputTracker -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.Partition import org.apache.spark.TaskContext diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index f5b3e97222..cece60dda7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -19,11 +19,15 @@ package org.apache.spark.scheduler import java.util.Properties import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import scala.collection.mutable + import org.apache.spark._ import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 0347cc02d7..e31a116a75 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.apache.spark.TaskContext -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.Partition import org.apache.spark.LocalSparkContext diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index a4f63baf3d..ff70a2cdf0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ import org.apache.spark.scheduler._ import org.apache.spark.executor.TaskMetrics import java.nio.ByteBuffer -import org.apache.spark.util.FakeClock +import org.apache.spark.util.{Utils, FakeClock} /** * A mock ClusterScheduler implementation that just remembers information about tasks started and diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala new file mode 100644 index 0000000000..0164dda0ba --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -0,0 +1,208 @@ +/* + * 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.serializer + +import scala.collection.mutable + +import com.esotericsoftware.kryo.Kryo + +import org.scalatest.FunSuite +import org.apache.spark.SharedSparkContext +import org.apache.spark.serializer.KryoTest._ + +class KryoSerializerSuite extends FunSuite with SharedSparkContext { + test("basic types") { + val ser = (new KryoSerializer).newInstance() + def check[T](t: T) { + assert(ser.deserialize[T](ser.serialize(t)) === t) + } + check(1) + check(1L) + check(1.0f) + check(1.0) + check(1.toByte) + check(1.toShort) + check("") + check("hello") + check(Integer.MAX_VALUE) + check(Integer.MIN_VALUE) + check(java.lang.Long.MAX_VALUE) + check(java.lang.Long.MIN_VALUE) + check[String](null) + check(Array(1, 2, 3)) + check(Array(1L, 2L, 3L)) + check(Array(1.0, 2.0, 3.0)) + check(Array(1.0f, 2.9f, 3.9f)) + check(Array("aaa", "bbb", "ccc")) + check(Array("aaa", "bbb", null)) + check(Array(true, false, true)) + check(Array('a', 'b', 'c')) + check(Array[Int]()) + check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) + } + + test("pairs") { + val ser = (new KryoSerializer).newInstance() + def check[T](t: T) { + assert(ser.deserialize[T](ser.serialize(t)) === t) + } + check((1, 1)) + check((1, 1L)) + check((1L, 1)) + check((1L, 1L)) + check((1.0, 1)) + check((1, 1.0)) + check((1.0, 1.0)) + check((1.0, 1L)) + check((1L, 1.0)) + check((1.0, 1L)) + check(("x", 1)) + check(("x", 1.0)) + check(("x", 1L)) + check((1, "x")) + check((1.0, "x")) + check((1L, "x")) + check(("x", "x")) + } + + test("Scala data structures") { + val ser = (new KryoSerializer).newInstance() + def check[T](t: T) { + assert(ser.deserialize[T](ser.serialize(t)) === t) + } + check(List[Int]()) + check(List[Int](1, 2, 3)) + check(List[String]()) + check(List[String]("x", "y", "z")) + check(None) + check(Some(1)) + check(Some("hi")) + check(mutable.ArrayBuffer(1, 2, 3)) + check(mutable.ArrayBuffer("1", "2", "3")) + check(mutable.Map()) + check(mutable.Map(1 -> "one", 2 -> "two")) + check(mutable.Map("one" -> 1, "two" -> 2)) + check(mutable.HashMap(1 -> "one", 2 -> "two")) + check(mutable.HashMap("one" -> 1, "two" -> 2)) + check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) + } + + test("custom registrator") { + System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) + + val ser = (new KryoSerializer).newInstance() + def check[T](t: T) { + assert(ser.deserialize[T](ser.serialize(t)) === t) + } + + check(CaseClass(17, "hello")) + + val c1 = new ClassWithNoArgConstructor + c1.x = 32 + check(c1) + + val c2 = new ClassWithoutNoArgConstructor(47) + check(c2) + + val hashMap = new java.util.HashMap[String, String] + hashMap.put("foo", "bar") + check(hashMap) + + System.clearProperty("spark.kryo.registrator") + } + + test("kryo with collect") { + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + assert(control === result.toSeq) + } + + test("kryo with parallelize") { + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control.map(new ClassWithoutNoArgConstructor(_))).map(_.x).collect() + assert (control === result.toSeq) + } + + test("kryo with parallelize for specialized tuples") { + assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).count === 3) + } + + test("kryo with parallelize for primitive arrays") { + assert (sc.parallelize( Array(1, 2, 3) ).count === 3) + } + + test("kryo with collect for specialized tuples") { + assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11)) + } + + test("kryo with reduce") { + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) + .reduce((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x + assert(control.sum === result) + } + + // TODO: this still doesn't work + ignore("kryo with fold") { + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) + .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x + assert(10 + control.sum === result) + } + + override def beforeAll() { + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) + super.beforeAll() + } + + override def afterAll() { + super.afterAll() + System.clearProperty("spark.kryo.registrator") + System.clearProperty("spark.serializer") + } +} + +object KryoTest { + case class CaseClass(i: Int, s: String) {} + + class ClassWithNoArgConstructor { + var x: Int = 0 + override def equals(other: Any) = other match { + case c: ClassWithNoArgConstructor => x == c.x + case _ => false + } + } + + class ClassWithoutNoArgConstructor(val x: Int) { + override def equals(other: Any) = other match { + case c: ClassWithoutNoArgConstructor => x == c.x + case _ => false + } + } + + class MyRegistrator extends KryoRegistrator { + override def registerClasses(k: Kryo) { + k.register(classOf[CaseClass]) + k.register(classOf[ClassWithNoArgConstructor]) + k.register(classOf[ClassWithoutNoArgConstructor]) + k.register(classOf[java.util.HashMap[_, _]]) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 88ba10f2f2..038a9acb85 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -29,12 +29,8 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.JavaSerializer -import org.apache.spark.KryoSerializer -import org.apache.spark.SizeEstimator -import org.apache.spark.Utils -import org.apache.spark.util.AkkaUtils -import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream} +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala new file mode 100644 index 0000000000..0ed366fb70 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -0,0 +1,147 @@ +/* + * 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.util + +import java.io.NotSerializableException + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.LocalSparkContext._ + +class ClosureCleanerSuite extends FunSuite { + test("closures inside an object") { + assert(TestObject.run() === 30) // 6 + 7 + 8 + 9 + } + + test("closures inside a class") { + val obj = new TestClass + assert(obj.run() === 30) // 6 + 7 + 8 + 9 + } + + test("closures inside a class with no default constructor") { + val obj = new TestClassWithoutDefaultConstructor(5) + assert(obj.run() === 30) // 6 + 7 + 8 + 9 + } + + test("closures that don't use fields of the outer class") { + val obj = new TestClassWithoutFieldAccess + assert(obj.run() === 30) // 6 + 7 + 8 + 9 + } + + test("nested closures inside an object") { + assert(TestObjectWithNesting.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 + } + + test("nested closures inside a class") { + val obj = new TestClassWithNesting(1) + assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 + } +} + +// A non-serializable class we create in closures to make sure that we aren't +// keeping references to unneeded variables from our outer closures. +class NonSerializable {} + +object TestObject { + def run(): Int = { + var nonSer = new NonSerializable + var x = 5 + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + x).reduce(_ + _) + } + } +} + +class TestClass extends Serializable { + var x = 5 + + def getX = x + + def run(): Int = { + var nonSer = new NonSerializable + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + getX).reduce(_ + _) + } + } +} + +class TestClassWithoutDefaultConstructor(x: Int) extends Serializable { + def getX = x + + def run(): Int = { + var nonSer = new NonSerializable + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + getX).reduce(_ + _) + } + } +} + +// This class is not serializable, but we aren't using any of its fields in our +// closures, so they won't have a $outer pointing to it and should still work. +class TestClassWithoutFieldAccess { + var nonSer = new NonSerializable + + def run(): Int = { + var nonSer2 = new NonSerializable + var x = 5 + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map(_ + x).reduce(_ + _) + } + } +} + + +object TestObjectWithNesting { + def run(): Int = { + var nonSer = new NonSerializable + var answer = 0 + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + var y = 1 + for (i <- 1 to 4) { + var nonSer2 = new NonSerializable + var x = i + answer += nums.map(_ + x + y).reduce(_ + _) + } + answer + } + } +} + +class TestClassWithNesting(val y: Int) extends Serializable { + def getY = y + + def run(): Int = { + var nonSer = new NonSerializable + var answer = 0 + return withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + for (i <- 1 to 4) { + var nonSer2 = new NonSerializable + var x = i + answer += nums.map(_ + x + getY).reduce(_ + _) + } + answer + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala new file mode 100644 index 0000000000..4e40dcbdee --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -0,0 +1,164 @@ +/* + * 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.util + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfterAll +import org.scalatest.PrivateMethodTester + +class DummyClass1 {} + +class DummyClass2 { + val x: Int = 0 +} + +class DummyClass3 { + val x: Int = 0 + val y: Double = 0.0 +} + +class DummyClass4(val d: DummyClass3) { + val x: Int = 0 +} + +object DummyString { + def apply(str: String) : DummyString = new DummyString(str.toArray) +} +class DummyString(val arr: Array[Char]) { + override val hashCode: Int = 0 + // JDK-7 has an extra hash32 field http://hg.openjdk.java.net/jdk7u/jdk7u6/jdk/rev/11987e85555f + @transient val hash32: Int = 0 +} + +class SizeEstimatorSuite + extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { + + var oldArch: String = _ + var oldOops: String = _ + + override def beforeAll() { + // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + oldArch = System.setProperty("os.arch", "amd64") + oldOops = System.setProperty("spark.test.useCompressedOops", "true") + } + + override def afterAll() { + resetOrClear("os.arch", oldArch) + resetOrClear("spark.test.useCompressedOops", oldOops) + } + + test("simple classes") { + assert(SizeEstimator.estimate(new DummyClass1) === 16) + assert(SizeEstimator.estimate(new DummyClass2) === 16) + assert(SizeEstimator.estimate(new DummyClass3) === 24) + assert(SizeEstimator.estimate(new DummyClass4(null)) === 24) + assert(SizeEstimator.estimate(new DummyClass4(new DummyClass3)) === 48) + } + + // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors + // (Sun vs IBM). Use a DummyString class to make tests deterministic. + test("strings") { + assert(SizeEstimator.estimate(DummyString("")) === 40) + assert(SizeEstimator.estimate(DummyString("a")) === 48) + assert(SizeEstimator.estimate(DummyString("ab")) === 48) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) + } + + test("primitive arrays") { + assert(SizeEstimator.estimate(new Array[Byte](10)) === 32) + assert(SizeEstimator.estimate(new Array[Char](10)) === 40) + assert(SizeEstimator.estimate(new Array[Short](10)) === 40) + assert(SizeEstimator.estimate(new Array[Int](10)) === 56) + assert(SizeEstimator.estimate(new Array[Long](10)) === 96) + assert(SizeEstimator.estimate(new Array[Float](10)) === 56) + assert(SizeEstimator.estimate(new Array[Double](10)) === 96) + assert(SizeEstimator.estimate(new Array[Int](1000)) === 4016) + assert(SizeEstimator.estimate(new Array[Long](1000)) === 8016) + } + + test("object arrays") { + // Arrays containing nulls should just have one pointer per element + assert(SizeEstimator.estimate(new Array[String](10)) === 56) + assert(SizeEstimator.estimate(new Array[AnyRef](10)) === 56) + + // For object arrays with non-null elements, each object should take one pointer plus + // however many bytes that class takes. (Note that Array.fill calls the code in its + // second parameter separately for each object, so we get distinct objects.) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)) === 216) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)) === 216) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)) === 296) + assert(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)) === 56) + + // Past size 100, our samples 100 elements, but we should still get the right size. + assert(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)) === 28016) + + // If an array contains the *same* element many times, we should only count it once. + val d1 = new DummyClass1 + assert(SizeEstimator.estimate(Array.fill(10)(d1)) === 72) // 10 pointers plus 8-byte object + assert(SizeEstimator.estimate(Array.fill(100)(d1)) === 432) // 100 pointers plus 8-byte object + + // Same thing with huge array containing the same element many times. Note that this won't + // return exactly 4032 because it can't tell that *all* the elements will equal the first + // one it samples, but it should be close to that. + + // TODO: If we sample 100 elements, this should always be 4176 ? + val estimatedSize = SizeEstimator.estimate(Array.fill(1000)(d1)) + assert(estimatedSize >= 4000, "Estimated size " + estimatedSize + " should be more than 4000") + assert(estimatedSize <= 4200, "Estimated size " + estimatedSize + " should be less than 4100") + } + + test("32-bit arch") { + val arch = System.setProperty("os.arch", "x86") + + val initialize = PrivateMethod[Unit]('initialize) + SizeEstimator invokePrivate initialize() + + assert(SizeEstimator.estimate(DummyString("")) === 40) + assert(SizeEstimator.estimate(DummyString("a")) === 48) + assert(SizeEstimator.estimate(DummyString("ab")) === 48) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) + + resetOrClear("os.arch", arch) + } + + // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors + // (Sun vs IBM). Use a DummyString class to make tests deterministic. + test("64-bit arch with no compressed oops") { + val arch = System.setProperty("os.arch", "amd64") + val oops = System.setProperty("spark.test.useCompressedOops", "false") + + val initialize = PrivateMethod[Unit]('initialize) + SizeEstimator invokePrivate initialize() + + assert(SizeEstimator.estimate(DummyString("")) === 56) + assert(SizeEstimator.estimate(DummyString("a")) === 64) + assert(SizeEstimator.estimate(DummyString("ab")) === 64) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 72) + + resetOrClear("os.arch", arch) + resetOrClear("spark.test.useCompressedOops", oops) + } + + def resetOrClear(prop: String, oldValue: String) { + if (oldValue != null) { + System.setProperty(prop, oldValue) + } else { + System.clearProperty(prop) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala new file mode 100644 index 0000000000..e2859caf58 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -0,0 +1,139 @@ +/* + * 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.util + +import com.google.common.base.Charsets +import com.google.common.io.Files +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File} +import org.scalatest.FunSuite +import org.apache.commons.io.FileUtils +import scala.util.Random + +class UtilsSuite extends FunSuite { + + test("bytesToString") { + assert(Utils.bytesToString(10) === "10.0 B") + assert(Utils.bytesToString(1500) === "1500.0 B") + assert(Utils.bytesToString(2000000) === "1953.1 KB") + assert(Utils.bytesToString(2097152) === "2.0 MB") + assert(Utils.bytesToString(2306867) === "2.2 MB") + assert(Utils.bytesToString(5368709120L) === "5.0 GB") + assert(Utils.bytesToString(5L * 1024L * 1024L * 1024L * 1024L) === "5.0 TB") + } + + test("copyStream") { + //input array initialization + val bytes = Array.ofDim[Byte](9000) + Random.nextBytes(bytes) + + val os = new ByteArrayOutputStream() + Utils.copyStream(new ByteArrayInputStream(bytes), os) + + assert(os.toByteArray.toList.equals(bytes.toList)) + } + + test("memoryStringToMb") { + assert(Utils.memoryStringToMb("1") === 0) + assert(Utils.memoryStringToMb("1048575") === 0) + assert(Utils.memoryStringToMb("3145728") === 3) + + assert(Utils.memoryStringToMb("1024k") === 1) + assert(Utils.memoryStringToMb("5000k") === 4) + assert(Utils.memoryStringToMb("4024k") === Utils.memoryStringToMb("4024K")) + + assert(Utils.memoryStringToMb("1024m") === 1024) + assert(Utils.memoryStringToMb("5000m") === 5000) + assert(Utils.memoryStringToMb("4024m") === Utils.memoryStringToMb("4024M")) + + assert(Utils.memoryStringToMb("2g") === 2048) + assert(Utils.memoryStringToMb("3g") === Utils.memoryStringToMb("3G")) + + assert(Utils.memoryStringToMb("2t") === 2097152) + assert(Utils.memoryStringToMb("3t") === Utils.memoryStringToMb("3T")) + } + + test("splitCommandString") { + assert(Utils.splitCommandString("") === Seq()) + assert(Utils.splitCommandString("a") === Seq("a")) + assert(Utils.splitCommandString("aaa") === Seq("aaa")) + assert(Utils.splitCommandString("a b c") === Seq("a", "b", "c")) + assert(Utils.splitCommandString(" a b\t c ") === Seq("a", "b", "c")) + assert(Utils.splitCommandString("a 'b c'") === Seq("a", "b c")) + assert(Utils.splitCommandString("a 'b c' d") === Seq("a", "b c", "d")) + assert(Utils.splitCommandString("'b c'") === Seq("b c")) + assert(Utils.splitCommandString("a \"b c\"") === Seq("a", "b c")) + assert(Utils.splitCommandString("a \"b c\" d") === Seq("a", "b c", "d")) + assert(Utils.splitCommandString("\"b c\"") === Seq("b c")) + assert(Utils.splitCommandString("a 'b\" c' \"d' e\"") === Seq("a", "b\" c", "d' e")) + assert(Utils.splitCommandString("a\t'b\nc'\nd") === Seq("a", "b\nc", "d")) + assert(Utils.splitCommandString("a \"b\\\\c\"") === Seq("a", "b\\c")) + assert(Utils.splitCommandString("a \"b\\\"c\"") === Seq("a", "b\"c")) + assert(Utils.splitCommandString("a 'b\\\"c'") === Seq("a", "b\\\"c")) + assert(Utils.splitCommandString("'a'b") === Seq("ab")) + assert(Utils.splitCommandString("'a''b'") === Seq("ab")) + assert(Utils.splitCommandString("\"a\"b") === Seq("ab")) + assert(Utils.splitCommandString("\"a\"\"b\"") === Seq("ab")) + assert(Utils.splitCommandString("''") === Seq("")) + assert(Utils.splitCommandString("\"\"") === Seq("")) + } + + test("string formatting of time durations") { + val second = 1000 + val minute = second * 60 + val hour = minute * 60 + def str = Utils.msDurationToString(_) + + assert(str(123) === "123 ms") + assert(str(second) === "1.0 s") + assert(str(second + 462) === "1.5 s") + assert(str(hour) === "1.00 h") + assert(str(minute) === "1.0 m") + assert(str(minute + 4 * second + 34) === "1.1 m") + assert(str(10 * hour + minute + 4 * second) === "10.02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") + } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } +} + diff --git a/docs/configuration.md b/docs/configuration.md index 55df18b6fb..58e9434bdc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -36,13 +36,13 @@ there are at least five properties that you will commonly want to control: spark.serializer - org.apache.spark.JavaSerializer + org.apache.spark.serializer.
JavaSerializer Class to use for serializing objects that will be sent over the network or need to be cached in serialized form. The default of Java serialization works with any Serializable Java object but is - quite slow, so we recommend using org.apache.spark.KryoSerializer + quite slow, so we recommend using org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization when speed is necessary. Can be any subclass of - org.apache.spark.Serializer. + org.apache.spark.Serializer. @@ -51,7 +51,7 @@ there are at least five properties that you will commonly want to control: If you use Kryo serialization, set this class to register your custom classes with Kryo. It should be set to a class that extends - KryoRegistrator. + KryoRegistrator. See the tuning guide for more details. @@ -171,7 +171,7 @@ Apart from these, the following properties are also available, and may be useful spark.closure.serializer - org.apache.spark.JavaSerializer + org.apache.spark.serializer.
JavaSerializer Serializer class to use for closures. Generally Java is fine unless your distributed functions (e.g. map functions) reference large objects in the driver program. diff --git a/docs/quick-start.md b/docs/quick-start.md index 8cbc55bed2..70c3df8095 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -109,7 +109,7 @@ We'll create a very simple Spark job in Scala. So simple, in fact, that it's nam {% highlight scala %} /*** SimpleJob.scala ***/ import org.apache.spark.SparkContext -import SparkContext._ +import org.apache.spark.SparkContext._ object SimpleJob { def main(args: Array[String]) { diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2cf319a263..f7768e55fc 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -37,7 +37,7 @@ Finally, you need to import some Spark classes and implicit conversions into you {% highlight scala %} import org.apache.spark.SparkContext -import SparkContext._ +import org.apache.spark.SparkContext._ {% endhighlight %} # Initializing Spark diff --git a/docs/tuning.md b/docs/tuning.md index 3563d110c9..28d88a2659 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -38,17 +38,17 @@ in your operations) and performance. It provides two serialization libraries: `Serializable` types and requires you to *register* the classes you'll use in the program in advance for best performance. -You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer")` +You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")` *before* creating your SparkContext. The only reason it is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. Finally, to register your classes with Kryo, create a public class that extends -[`org.apache.spark.KryoRegistrator`](api/core/index.html#org.apache.spark.KryoRegistrator) and set the +[`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` system property to point to it, as follows: {% highlight scala %} import com.esotericsoftware.kryo.Kryo -import org.apache.spark.KryoRegistrator +import org.apache.spark.serializer.KryoRegistrator class MyRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo) { @@ -58,7 +58,7 @@ class MyRegistrator extends KryoRegistrator { } // Make sure to set these properties *before* creating a SparkContext! -System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") +System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator") val sc = new SparkContext(...) {% endhighlight %} @@ -217,7 +217,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest parent RDD's number of partitions. You can pass the level of parallelism as a second argument -(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.PairRDDFunctions) documentation), +(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index b190e83c4d..cfafbaf23e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.examples.bagel import org.apache.spark._ import org.apache.spark.SparkContext._ +import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.bagel._ import org.apache.spark.bagel.Bagel._ diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index b1f606e48e..72b5c7b88e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -37,7 +37,7 @@ object WikipediaPageRank { System.exit(-1) } - System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 3bfa48eaf3..ddf6855325 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -17,19 +17,16 @@ package org.apache.spark.examples.bagel -import org.apache.spark._ -import serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import org.apache.spark.SparkContext._ - -import org.apache.spark.bagel._ -import org.apache.spark.bagel.Bagel._ - -import scala.xml.{XML,NodeSeq} +import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import scala.xml.{XML, NodeSeq} -import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} -import java.nio.ByteBuffer +import org.apache.spark._ +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD object WikipediaPageRankStandalone { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 822da8c9b5..fad512eeba 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.examples -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index 2e3d9ccf00..0b45c30d20 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.util.RawTextHelper * lines have the word 'the' in them. This is useful for benchmarking purposes. This * will only work with spark.streaming.util.RawTextSender running on all worker nodes * and with Spark using Kryo serialization (set Java property "spark.serializer" to - * "org.apache.spark.KryoSerializer"). + * "org.apache.spark.serializer.KryoSerializer"). * Usage: RawNetworkGrep * is the Spark master URL * is the number rawNetworkStreams, which should be same as number diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 4f4a7f5296..60cb44ce89 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -1,6 +1,6 @@ package org.apache.spark.mllib.classification -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD trait ClassificationModel extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 91bb50c829..50aede9c07 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.classification import scala.math.round -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index c92c7cc3f3..3511e24bce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.classification import scala.math.signum -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 2c3db099fa..edbf77dbcc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -20,8 +20,9 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import scala.util.Random -import org.apache.spark.{SparkContext, RDD} +import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD import org.apache.spark.Logging import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index d1fe5d138d..cfc81c985a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index b62c9b3340..b77364e08d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix @@ -29,8 +30,9 @@ import scala.collection.mutable.ArrayBuffer * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -class GradientDescent(var gradient: Gradient, var updater: Updater) extends Optimizer { - +class GradientDescent(var gradient: Gradient, var updater: Updater) + extends Optimizer with Logging +{ private var stepSize: Double = 1.0 private var numIterations: Int = 100 private var regParam: Double = 0.0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 50059d385d..94d30b56f2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD trait Optimizer { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 218217acfe..be002d02bc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -21,9 +21,10 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import scala.util.Sorting -import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import org.apache.spark.{HashPartitioner, Partitioner, SparkContext} import org.apache.spark.storage.StorageLevel -import org.apache.spark.KryoRegistrator +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ import com.esotericsoftware.kryo.Kryo @@ -432,7 +433,7 @@ object ALS { val (master, ratingsFile, rank, iters, outputDir) = (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) val blocks = if (args.length == 6) args(5).toInt else -1 - System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) System.setProperty("spark.kryo.referenceTracking", "false") System.setProperty("spark.kryoserializer.buffer.mb", "8") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index ae9fe48aec..af43d89c70 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.recommendation -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 06015110ac..f98b0b536d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkException} +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index df3beb1959..d959695325 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 71f968471c..ae95ea24fc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 8dd325efc0..423afc32d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.regression -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD trait RegressionModel extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 228ab9e4e8..b29508d2b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 7fd4623071..8b55bce7c4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.util -import org.apache.spark.{RDD, Logging} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 6500d47183..9109189dff 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD /** * Generate test data for KMeans. This class first chooses k cluster centers diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 4c49d484b4..bc5045fb05 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -22,7 +22,8 @@ import scala.util.Random import org.jblas.DoubleMatrix -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index f553298fc5..52c4a71d62 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 7eb69ae81c..5aec867257 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -21,7 +21,8 @@ import scala.util.Random import org.jblas.DoubleMatrix -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.util.MLUtils /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 0aeafbe23c..d91b74c3ac 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.util -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index d3f191b05b..6e9f667635 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -2,9 +2,10 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.{RDD, SparkContext} - import org.jblas.DoubleMatrix + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 906e9221a1..8fbf296509 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -114,9 +114,9 @@ class SparkContext(object): self.addPyFile(path) # Create a temporary directory inside spark.local.dir: - local_dir = self._jvm.org.apache.spark.Utils.getLocalDir() + local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir() self._temp_dir = \ - self._jvm.org.apache.spark.Utils.createTempDir(local_dir).getAbsolutePath() + self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() @property def defaultParallelism(self): diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 7e244e48a2..e6e35c9b5d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -28,7 +28,7 @@ import scala.reflect.NameTransformer import SparkIMain._ import org.apache.spark.HttpServer -import org.apache.spark.Utils +import org.apache.spark.util.Utils import org.apache.spark.SparkEnv /** An interpreter for Scala code. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index 362247cc38..80da6bd30b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -23,7 +23,8 @@ import org.apache.spark.util.MetadataCleaner //import Time._ -import org.apache.spark.{RDD, Logging} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index 290ad37812..6bf275f5af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming -import org.apache.spark.Utils +import org.apache.spark.util.Utils case class Duration (private val millis: Long) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala index d8a7381e87..757bc98981 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -22,8 +22,9 @@ import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} import org.apache.spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} -import org.apache.spark.{Manifests, RDD, Partitioner, HashPartitioner} +import org.apache.spark.{Partitioner, HashPartitioner} import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{Manifests, RDD, PairRDDFunctions} import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer @@ -101,8 +102,8 @@ extends Serializable { /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more - * information. + * combineByKey for RDDs. Please refer to combineByKey in + * [[org.apache.spark.rdd.PairRDDFunctions]] for more information. */ def combineByKey[C: ClassManifest]( createCombiner: V => C, @@ -379,7 +380,7 @@ extends Serializable { /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. - * [[org.apache.spark.Paxrtitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. Note, that * this function may generate a different a tuple with a different key diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 3852ac2dab..878725c705 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -24,6 +24,7 @@ import akka.zeromq.Subscribe import org.apache.spark.streaming.dstream._ import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.receivers.ActorReceiver import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy import org.apache.spark.streaming.receivers.ZeroMQReceiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index f8c8d8ece1..d1932b6b05 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -21,7 +21,7 @@ import org.apache.spark.streaming.{Duration, Time, DStream} import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.api.java.JavaRDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 2e6fe9a9c4..459695b7ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -26,7 +26,7 @@ import org.apache.spark.streaming._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import java.util -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import JavaDStream._ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index c203dccd17..978fca33ad 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -25,14 +25,15 @@ import scala.collection.JavaConversions._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.Partitioner import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration import org.apache.spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD} import org.apache.spark.storage.StorageLevel import com.google.common.base.Optional -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.PairRDDFunctions class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifiest: ClassManifest[K], @@ -147,7 +148,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Combine elements of each key in DStream's RDDs using custom function. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more + * combineByKey for RDDs. Please refer to combineByKey in [[PairRDDFunctions]] for more * information. */ def combineByKey[C](createCombiner: JFunction[V, C], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index f10beb1db3..54ba3e6025 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,23 +17,26 @@ package org.apache.spark.streaming.api.java -import org.apache.spark.streaming._ -import receivers.{ActorReceiver, ReceiverSupervisorStrategy} -import org.apache.spark.streaming.dstream._ -import org.apache.spark.storage.StorageLevel -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import java.lang.{Long => JLong, Integer => JInt} +import java.io.InputStream +import java.util.{Map => JMap} + +import scala.collection.JavaConversions._ + import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import twitter4j.Status import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe -import scala.collection.JavaConversions._ -import java.lang.{Long => JLong, Integer => JInt} -import java.io.InputStream -import java.util.{Map => JMap} import twitter4j.auth.Authorization -import org.apache.spark.RDD + +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala index 4a9d82211f..4eddc755b9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.CoGroupedRDD import org.apache.spark.streaming.{Time, DStream, Duration} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 35cc4cb396..a9a05c9981 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, StreamingContext} /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 1c265ed972..fea0573b77 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index 3166c68760..91ee2c1a36 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class FilteredDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 21950ad6ac..ca7d7ca49e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index 8377cfe60c..b37966f9a7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala index 3fb443143c..18de772946 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext +import java.net.InetSocketAddress +import java.io.{ObjectInput, ObjectOutput, Externalizable} +import java.nio.ByteBuffer -import org.apache.spark.Utils -import org.apache.spark.storage.StorageLevel +import scala.collection.JavaConversions._ import org.apache.flume.source.avro.AvroSourceProtocol import org.apache.flume.source.avro.AvroFlumeEvent @@ -28,11 +29,9 @@ import org.apache.flume.source.avro.Status import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer -import scala.collection.JavaConversions._ - -import java.net.InetSocketAddress -import java.io.{ObjectInput, ObjectOutput, Externalizable} -import java.nio.ByteBuffer +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.util.Utils +import org.apache.spark.storage.StorageLevel private[streaming] class FlumeInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index c1f95650c8..e21bac4602 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Job, Time} private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index 1e4c7e7fde..4294b07d91 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class GlommedDStream[T: ClassManifest](parent: DStream[T]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index 1d79d707bb..5329601a6f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 312e0c0567..8290df90a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index af688dde5f..b1682afea3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class MappedDStream[T: ClassManifest, U: ClassManifest] ( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 3d68da36a2..31f9891560 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,22 +17,21 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} - -import org.apache.spark.{Logging, SparkEnv, RDD} -import org.apache.spark.rdd.BlockRDD -import org.apache.spark.storage.StorageLevel +import java.util.concurrent.ArrayBlockingQueue +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import java.nio.ByteBuffer - import akka.actor.{Props, Actor} import akka.pattern.ask import akka.dispatch.Await import akka.util.duration._ + import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import java.util.concurrent.ArrayBlockingQueue +import org.apache.spark.streaming._ +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.storage.StorageLevel /** * Abstract class for defining any InputDStream that has to start a receiver on worker diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index b43ecaeebe..7d9f3521b1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import scala.collection.mutable.Queue diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index b6c672f899..b88a4db959 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} import org.apache.spark.Partitioner import org.apache.spark.SparkContext._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index 3a0bd2acd7..a95e66d761 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index c1c9f808f0..362a6bf4cc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.Partitioner import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index edba2032b4..60485adef9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Time} private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index 97eab97b2f..c696bb70a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import collection.mutable.ArrayBuffer import org.apache.spark.rdd.UnionRDD diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index dbbea39e81..3c57294269 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Duration, Interval, Time, DStream} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index 50d72298e4..6977957126 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.util -import org.apache.spark.{Logging, RDD} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream.ForEachDStream import StreamingContext._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 249f6a22ae..fc8655a083 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -20,10 +20,11 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import org.apache.spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket -import org.apache.spark.{Logging, KryoSerializer} +import org.apache.spark.{Logging} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import scala.io.Source import java.io.IOException +import org.apache.spark.serializer.KryoSerializer /** * A helper program that sends blocks of Kryo-serialized text strings out on a socket at a diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 31c2fa0208..37dd9c4cc6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -20,8 +20,6 @@ package org.apache.spark.streaming import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream} import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.{RDD, Logging} - import collection.mutable.ArrayBuffer import collection.mutable.SynchronizedBuffer @@ -29,6 +27,9 @@ import java.io.{ObjectInputStream, IOException} import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD + /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, * replayable, reliable message queue like Kafka. It requires a sequence as input, and diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 50335e5736..f824c472ae 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.api.java._ -import org.apache.spark.rdd.OrderedRDDFunctions +import org.apache.spark.rdd.{RDD, DoubleRDDFunctions, PairRDDFunctions, OrderedRDDFunctions} import org.apache.spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} @@ -139,7 +139,7 @@ object JavaAPICompletenessChecker { scalaType match { case ParameterizedType(name, parameters, typebounds) => name match { - case "org.apache.spark.RDD" => + case "org.apache.spark.rdd.RDD" => if (parameters(0).name == classOf[Tuple2[_, _]].getName) { val tupleParams = parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) @@ -211,23 +211,23 @@ object JavaAPICompletenessChecker { // This list also includes a few methods that are only used by the web UI or other // internal Spark components. val excludedNames = Seq( - "org.apache.spark.RDD.origin", - "org.apache.spark.RDD.elementClassManifest", - "org.apache.spark.RDD.checkpointData", - "org.apache.spark.RDD.partitioner", - "org.apache.spark.RDD.partitions", - "org.apache.spark.RDD.firstParent", - "org.apache.spark.RDD.doCheckpoint", - "org.apache.spark.RDD.markCheckpointed", - "org.apache.spark.RDD.clearDependencies", - "org.apache.spark.RDD.getDependencies", - "org.apache.spark.RDD.getPartitions", - "org.apache.spark.RDD.dependencies", - "org.apache.spark.RDD.getPreferredLocations", - "org.apache.spark.RDD.collectPartitions", - "org.apache.spark.RDD.computeOrReadCheckpoint", - "org.apache.spark.PairRDDFunctions.getKeyClass", - "org.apache.spark.PairRDDFunctions.getValueClass", + "org.apache.spark.rdd.RDD.origin", + "org.apache.spark.rdd.RDD.elementClassManifest", + "org.apache.spark.rdd.RDD.checkpointData", + "org.apache.spark.rdd.RDD.partitioner", + "org.apache.spark.rdd.RDD.partitions", + "org.apache.spark.rdd.RDD.firstParent", + "org.apache.spark.rdd.RDD.doCheckpoint", + "org.apache.spark.rdd.RDD.markCheckpointed", + "org.apache.spark.rdd.RDD.clearDependencies", + "org.apache.spark.rdd.RDD.getDependencies", + "org.apache.spark.rdd.RDD.getPartitions", + "org.apache.spark.rdd.RDD.dependencies", + "org.apache.spark.rdd.RDD.getPreferredLocations", + "org.apache.spark.rdd.RDD.collectPartitions", + "org.apache.spark.rdd.RDD.computeOrReadCheckpoint", + "org.apache.spark.rdd.PairRDDFunctions.getKeyClass", + "org.apache.spark.rdd.PairRDDFunctions.getValueClass", "org.apache.spark.SparkContext.stringToText", "org.apache.spark.SparkContext.makeRDD", "org.apache.spark.SparkContext.runJob", -- cgit v1.2.3