Skip to content

Commit

Permalink
solve conflicts and remove unnecessary implicts
Browse files Browse the repository at this point in the history
  • Loading branch information
zhzhan committed Oct 1, 2014
2 parents dc7bdb3 + d10bf00 commit 3dd50e8
Show file tree
Hide file tree
Showing 123 changed files with 3,674 additions and 1,648 deletions.
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ conf/*.cmd
conf/*.properties
conf/*.conf
conf/*.xml
conf/slaves
docs/_site
docs/api
target/
Expand Down
1 change: 1 addition & 0 deletions .rat-excludes
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ log4j.properties
log4j.properties.template
metrics.properties.template
slaves
slaves.template
spark-env.sh
spark-env.cmd
spark-env.sh.template
Expand Down
File renamed without changes.
269 changes: 269 additions & 0 deletions core/src/main/java/org/apache/spark/TaskContext.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,269 @@
/*
* 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.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;

import scala.Function0;
import scala.Function1;
import scala.Unit;
import scala.collection.JavaConversions;

import org.apache.spark.annotation.DeveloperApi;
import org.apache.spark.executor.TaskMetrics;
import org.apache.spark.util.TaskCompletionListener;
import org.apache.spark.util.TaskCompletionListenerException;

/**
* :: DeveloperApi ::
* Contextual information about a task which can be read or mutated during execution.
*/
@DeveloperApi
public class TaskContext implements Serializable {

private int stageId;
private int partitionId;
private long attemptId;
private boolean runningLocally;
private TaskMetrics taskMetrics;

/**
* :: DeveloperApi ::
* Contextual information about a task which can be read or mutated during execution.
*
* @param stageId stage id
* @param partitionId index of the partition
* @param attemptId the number of attempts to execute this task
* @param runningLocally whether the task is running locally in the driver JVM
* @param taskMetrics performance metrics of the task
*/
@DeveloperApi
public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally,
TaskMetrics taskMetrics) {
this.attemptId = attemptId;
this.partitionId = partitionId;
this.runningLocally = runningLocally;
this.stageId = stageId;
this.taskMetrics = taskMetrics;
}

/**
* :: DeveloperApi ::
* Contextual information about a task which can be read or mutated during execution.
*
* @param stageId stage id
* @param partitionId index of the partition
* @param attemptId the number of attempts to execute this task
* @param runningLocally whether the task is running locally in the driver JVM
*/
@DeveloperApi
public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally) {
this.attemptId = attemptId;
this.partitionId = partitionId;
this.runningLocally = runningLocally;
this.stageId = stageId;
this.taskMetrics = TaskMetrics.empty();
}

/**
* :: DeveloperApi ::
* Contextual information about a task which can be read or mutated during execution.
*
* @param stageId stage id
* @param partitionId index of the partition
* @param attemptId the number of attempts to execute this task
*/
@DeveloperApi
public TaskContext(int stageId, int partitionId, long attemptId) {
this.attemptId = attemptId;
this.partitionId = partitionId;
this.runningLocally = false;
this.stageId = stageId;
this.taskMetrics = TaskMetrics.empty();
}

private static ThreadLocal<TaskContext> taskContext =
new ThreadLocal<TaskContext>();

/**
* :: Internal API ::
* This is spark internal API, not intended to be called from user programs.
*/
public static void setTaskContext(TaskContext tc) {
taskContext.set(tc);
}

public static TaskContext get() {
return taskContext.get();
}

/** :: Internal API :: */
public static void unset() {
taskContext.remove();
}

// List of callback functions to execute when the task completes.
private transient List<TaskCompletionListener> onCompleteCallbacks =
new ArrayList<TaskCompletionListener>();

// Whether the corresponding task has been killed.
private volatile boolean interrupted = false;

// Whether the task has completed.
private volatile boolean completed = false;

/**
* Checks whether the task has completed.
*/
public boolean isCompleted() {
return completed;
}

/**
* Checks whether the task has been killed.
*/
public boolean isInterrupted() {
return interrupted;
}

/**
* Add a (Java friendly) listener to be executed on task completion.
* This will be called in all situation - success, failure, or cancellation.
* <p/>
* An example use is for HadoopRDD to register a callback to close the input stream.
*/
public TaskContext addTaskCompletionListener(TaskCompletionListener listener) {
onCompleteCallbacks.add(listener);
return this;
}

/**
* Add a listener in the form of a Scala closure to be executed on task completion.
* This will be called in all situations - success, failure, or cancellation.
* <p/>
* An example use is for HadoopRDD to register a callback to close the input stream.
*/
public TaskContext addTaskCompletionListener(final Function1<TaskContext, Unit> f) {
onCompleteCallbacks.add(new TaskCompletionListener() {
@Override
public void onTaskCompletion(TaskContext context) {
f.apply(context);
}
});
return this;
}

/**
* Add a callback function to be executed on task completion. An example use
* is for HadoopRDD to register a callback to close the input stream.
* Will be called in any situation - success, failure, or cancellation.
*
* Deprecated: use addTaskCompletionListener
*
* @param f Callback function.
*/
@Deprecated
public void addOnCompleteCallback(final Function0<Unit> f) {
onCompleteCallbacks.add(new TaskCompletionListener() {
@Override
public void onTaskCompletion(TaskContext context) {
f.apply();
}
});
}

/**
* ::Internal API::
* Marks the task as completed and triggers the listeners.
*/
public void markTaskCompleted() throws TaskCompletionListenerException {
completed = true;
List<String> errorMsgs = new ArrayList<String>(2);
// Process complete callbacks in the reverse order of registration
List<TaskCompletionListener> revlist =
new ArrayList<TaskCompletionListener>(onCompleteCallbacks);
Collections.reverse(revlist);
for (TaskCompletionListener tcl: revlist) {
try {
tcl.onTaskCompletion(this);
} catch (Throwable e) {
errorMsgs.add(e.getMessage());
}
}

if (!errorMsgs.isEmpty()) {
throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs));
}
}

/**
* ::Internal API::
* Marks the task for interruption, i.e. cancellation.
*/
public void markInterrupted() {
interrupted = true;
}

@Deprecated
/** Deprecated: use getStageId() */
public int stageId() {
return stageId;
}

@Deprecated
/** Deprecated: use getPartitionId() */
public int partitionId() {
return partitionId;
}

@Deprecated
/** Deprecated: use getAttemptId() */
public long attemptId() {
return attemptId;
}

@Deprecated
/** Deprecated: use isRunningLocally() */
public boolean runningLocally() {
return runningLocally;
}

public boolean isRunningLocally() {
return runningLocally;
}

public int getStageId() {
return stageId;
}

public int getPartitionId() {
return partitionId;
}

public long getAttemptId() {
return attemptId;
}

/** ::Internal API:: */
public TaskMetrics taskMetrics() {
return taskMetrics;
}
}
37 changes: 8 additions & 29 deletions core/src/main/scala/org/apache/spark/MapOutputTracker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,12 @@
package org.apache.spark

import java.io._
import java.util.concurrent.ConcurrentHashMap
import java.util.zip.{GZIPInputStream, GZIPOutputStream}

import scala.collection.mutable.{HashSet, HashMap, Map}
import scala.concurrent.Await
import scala.collection.JavaConversions._

import akka.actor._
import akka.pattern.ask
Expand Down Expand Up @@ -84,6 +86,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
* On the master, it serves as the source of map outputs recorded from ShuffleMapTasks.
* On the workers, it simply serves as a cache, in which a miss triggers a fetch from the
* master's corresponding HashMap.
*
* Note: because mapStatuses is accessed concurrently, subclasses should make sure it's a
* thread-safe map.
*/
protected val mapStatuses: Map[Int, Array[MapStatus]]

Expand Down Expand Up @@ -339,11 +344,11 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf)
* MapOutputTrackerMaster.
*/
private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) {
protected val mapStatuses = new HashMap[Int, Array[MapStatus]]
protected val mapStatuses: Map[Int, Array[MapStatus]] =
new ConcurrentHashMap[Int, Array[MapStatus]]
}

private[spark] object MapOutputTracker {
private val LOG_BASE = 1.1

// Serialize an array of map output locations into an efficient byte format so that we can send
// it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will
Expand Down Expand Up @@ -379,34 +384,8 @@ private[spark] object MapOutputTracker {
throw new MetadataFetchFailedException(
shuffleId, reduceId, "Missing an output location for shuffle " + shuffleId)
} else {
(status.location, decompressSize(status.compressedSizes(reduceId)))
(status.location, status.getSizeForBlock(reduceId))
}
}
}

/**
* Compress a size in bytes to 8 bits for efficient reporting of map output sizes.
* We do this by encoding the log base 1.1 of the size as an integer, which can support
* sizes up to 35 GB with at most 10% error.
*/
def compressSize(size: Long): Byte = {
if (size == 0) {
0
} else if (size <= 1L) {
1
} else {
math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte
}
}

/**
* Decompress an 8-bit encoded block size, using the reverse operation of compressSize.
*/
def decompressSize(compressedSize: Byte): Long = {
if (compressedSize == 0) {
0
} else {
math.pow(LOG_BASE, compressedSize & 0xFF).toLong
}
}
}
Loading

0 comments on commit 3dd50e8

Please sign in to comment.