diff options
author | Matei Zaharia <matei@eecs.berkeley.edu> | 2013-09-01 00:32:28 -0700 |
---|---|---|
committer | Matei Zaharia <matei@eecs.berkeley.edu> | 2013-09-01 14:13:16 -0700 |
commit | 0a8cc309211c62f8824d76618705c817edcf2424 (patch) | |
tree | 86cb7e3b2611b9287335699dcd535f145ccf56d6 /tools/src/main/scala | |
parent | 5b4dea21439e86b61447bdb1613b2ddff9ffba9f (diff) | |
download | spark-0a8cc309211c62f8824d76618705c817edcf2424.tar.gz spark-0a8cc309211c62f8824d76618705c817edcf2424.tar.bz2 spark-0a8cc309211c62f8824d76618705c817edcf2424.zip |
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
Diffstat (limited to 'tools/src/main/scala')
-rw-r--r-- | tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala | 38 |
1 files changed, 19 insertions, 19 deletions
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", |