From dd00b7c83fd0a4fa1cbd9115f2e0a8e69bc519b9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 3 Feb 2015 13:47:49 -0800 Subject: [PATCH] Move CommitDeniedException to executors package; remove `@DeveloperAPI` annotation. --- .../main/scala/org/apache/spark/SparkEnv.scala | 3 +-- .../org/apache/spark/SparkHadoopWriter.scala | 2 +- .../{ => executor}/CommitDeniedException.scala | 16 ++++++++++------ 3 files changed, 12 insertions(+), 9 deletions(-) rename core/src/main/scala/org/apache/spark/{ => executor}/CommitDeniedException.scala (77%) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5fde6051bc089..f0edf5d9cb280 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -20,8 +20,6 @@ package org.apache.spark import java.io.File import java.net.Socket -import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorActor - import scala.collection.JavaConversions._ import scala.collection.mutable import scala.util.Properties @@ -37,6 +35,7 @@ import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.{OutputCommitCoordinator, LiveListenerBus} +import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorActor import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index c0e91a84ac3d8..838db5b7729f1 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,9 +26,9 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path +import org.apache.spark.executor.CommitDeniedException import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD -import org.apache.spark.util.AkkaUtils /** * Internal helper class that saves an RDD using a Hadoop OutputFormat. diff --git a/core/src/main/scala/org/apache/spark/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala similarity index 77% rename from core/src/main/scala/org/apache/spark/CommitDeniedException.scala rename to core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala index 3a910d4339265..2ef67ba83e2be 100644 --- a/core/src/main/scala/org/apache/spark/CommitDeniedException.scala +++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala @@ -15,18 +15,22 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.executor -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.{TaskCommitDenied, TaskEndReason} /** - * :: DeveloperApi :: * Exception thrown when a task attempts to commit output to Hadoop, but * is denied by the driver. */ -@DeveloperApi -class CommitDeniedException(msg: String, jobID: Int, splitID: Int, attemptID: Int) +class CommitDeniedException( + msg: String, + jobID: Int, + splitID: Int, + attemptID: Int) extends Exception(msg) { - def toTaskEndReason(): TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID) + + def toTaskEndReason: TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID) + }