aboutsummaryrefslogtreecommitdiff
path: root/tools
diff options
context:
space:
mode:
Diffstat (limited to 'tools')
-rw-r--r--tools/pom.xml135
-rw-r--r--tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala42
2 files changed, 42 insertions, 135 deletions
diff --git a/tools/pom.xml b/tools/pom.xml
index 1125aba4f1..95b5e80e5b 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -30,6 +30,24 @@
<name>Spark Project Tools</name>
<url>http://spark-project.org/</url>
+ <dependencies>
+ <dependency>
+ <groupId>org.spark-project</groupId>
+ <artifactId>spark-core</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.spark-project</groupId>
+ <artifactId>spark-streaming</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
<build>
<outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
@@ -48,121 +66,4 @@
</plugin>
</plugins>
</build>
-
- <profiles>
- <profile>
- <id>hadoop1</id>
- <dependencies>
- <dependency>
- <groupId>org.spark-project</groupId>
- <artifactId>spark-core</artifactId>
- <version>${project.version}</version>
- <classifier>hadoop1</classifier>
- </dependency>
- <dependency>
- <groupId>org.spark-project</groupId>
- <artifactId>spark-streaming</artifactId>
- <version>${project.version}</version>
- <classifier>hadoop1</classifier>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <scope>provided</scope>
- </dependency>
- </dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <configuration>
- <classifier>hadoop1</classifier>
- </configuration>
- </plugin>
- </plugins>
- </build>
- </profile>
- <profile>
- <id>hadoop2</id>
- <dependencies>
- <dependency>
- <groupId>org.spark-project</groupId>
- <artifactId>spark-core</artifactId>
- <version>${project.version}</version>
- <classifier>hadoop2</classifier>
- </dependency>
- <dependency>
- <groupId>org.spark-project</groupId>
- <artifactId>spark-streaming</artifactId>
- <version>${project.version}</version>
- <classifier>hadoop2</classifier>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-core</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-client</artifactId>
- <scope>provided</scope>
- </dependency>
- </dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <configuration>
- <classifier>hadoop2</classifier>
- </configuration>
- </plugin>
- </plugins>
- </build>
- </profile>
- <profile>
- <id>hadoop2-yarn</id>
- <dependencies>
- <dependency>
- <groupId>org.spark-project</groupId>
- <artifactId>spark-core</artifactId>
- <version>${project.version}</version>
- <classifier>hadoop2-yarn</classifier>
- </dependency>
- <dependency>
- <groupId>org.spark-project</groupId>
- <artifactId>spark-streaming</artifactId>
- <version>${project.version}</version>
- <classifier>hadoop2-yarn</classifier>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-client</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-api</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-common</artifactId>
- <scope>provided</scope>
- </dependency>
- </dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <configuration>
- <classifier>hadoop2-yarn</classifier>
- </configuration>
- </plugin>
- </plugins>
- </build>
- </profile>
- </profiles>
</project>
diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala
index 3a55f50812..f45d0b281c 100644
--- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala
+++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala
@@ -17,13 +17,15 @@
package spark.tools
-import spark._
import java.lang.reflect.Method
+
import scala.collection.mutable.ArrayBuffer
+
+import spark._
import spark.api.java._
+import spark.rdd.OrderedRDDFunctions
import spark.streaming.{PairDStreamFunctions, DStream, StreamingContext}
import spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext}
-import scala.Tuple2
private[spark] abstract class SparkType(val name: String)
@@ -121,7 +123,7 @@ object JavaAPICompletenessChecker {
SparkMethod(name, returnType, parameters)
}
- private def toJavaType(scalaType: SparkType): SparkType = {
+ private def toJavaType(scalaType: SparkType, isReturnType: Boolean): SparkType = {
val renameSubstitutions = Map(
"scala.collection.Map" -> "java.util.Map",
// TODO: the JavaStreamingContext API accepts Array arguments
@@ -140,40 +142,43 @@ object JavaAPICompletenessChecker {
case "spark.RDD" =>
if (parameters(0).name == classOf[Tuple2[_, _]].getName) {
val tupleParams =
- parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType)
+ parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs)
ParameterizedType(classOf[JavaPairRDD[_, _]].getName, tupleParams)
} else {
- ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(toJavaType))
+ ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(applySubs))
}
case "spark.streaming.DStream" =>
if (parameters(0).name == classOf[Tuple2[_, _]].getName) {
val tupleParams =
- parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType)
+ parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs)
ParameterizedType("spark.streaming.api.java.JavaPairDStream", tupleParams)
} else {
ParameterizedType("spark.streaming.api.java.JavaDStream",
- parameters.map(toJavaType))
+ parameters.map(applySubs))
+ }
+ case "scala.Option" => {
+ if (isReturnType) {
+ ParameterizedType("com.google.common.base.Optional", parameters.map(applySubs))
+ } else {
+ applySubs(parameters(0))
}
- // TODO: Spark Streaming uses Guava's Optional in place of Option, leading to some
- // false-positives here:
- case "scala.Option" =>
- toJavaType(parameters(0))
+ }
case "scala.Function1" =>
val firstParamName = parameters.last.name
if (firstParamName.startsWith("scala.collection.Traversable") ||
firstParamName.startsWith("scala.collection.Iterator")) {
ParameterizedType("spark.api.java.function.FlatMapFunction",
Seq(parameters(0),
- parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(toJavaType))
+ parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(applySubs))
} else if (firstParamName == "scala.runtime.BoxedUnit") {
ParameterizedType("spark.api.java.function.VoidFunction",
- parameters.dropRight(1).map(toJavaType))
+ parameters.dropRight(1).map(applySubs))
} else {
- ParameterizedType("spark.api.java.function.Function", parameters.map(toJavaType))
+ ParameterizedType("spark.api.java.function.Function", parameters.map(applySubs))
}
case _ =>
ParameterizedType(renameSubstitutions.getOrElse(name, name),
- parameters.map(toJavaType))
+ parameters.map(applySubs))
}
case BaseType(name) =>
if (renameSubstitutions.contains(name)) {
@@ -194,8 +199,9 @@ object JavaAPICompletenessChecker {
private def toJavaMethod(method: SparkMethod): SparkMethod = {
val params = method.parameters
- .filterNot(_.name == "scala.reflect.ClassManifest").map(toJavaType)
- SparkMethod(method.name, toJavaType(method.returnType), params)
+ .filterNot(_.name == "scala.reflect.ClassManifest")
+ .map(toJavaType(_, isReturnType = false))
+ SparkMethod(method.name, toJavaType(method.returnType, isReturnType = true), params)
}
private def isExcludedByName(method: Method): Boolean = {
@@ -332,7 +338,7 @@ object JavaAPICompletenessChecker {
println()
println("Missing OrderedRDD methods")
- printMissingMethods(classOf[OrderedRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]])
+ printMissingMethods(classOf[OrderedRDDFunctions[_, _, _]], classOf[JavaPairRDD[_, _]])
println()
println("Missing SparkContext methods")