зеркало из https://github.com/microsoft/spark.git
Initial work to rename package to org.apache.spark
This commit is contained in:
Родитель
a30fac16ca
Коммит
46eecd110a
|
@ -27,7 +27,7 @@ Or, for the Python API, the Python shell (`./pyspark`).
|
|||
Spark also comes with several sample programs in the `examples` directory.
|
||||
To run one of them, use `./run-example <class> <params>`. For example:
|
||||
|
||||
./run-example spark.examples.SparkLR local[2]
|
||||
./run-example org.apache.spark.examples.SparkLR local[2]
|
||||
|
||||
will run the Logistic Regression example locally on 2 CPUs.
|
||||
|
||||
|
|
|
@ -19,13 +19,13 @@
|
|||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-parent</artifactId>
|
||||
<version>0.8.0-SNAPSHOT</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-assembly</artifactId>
|
||||
<name>Spark Project Assembly</name>
|
||||
<url>http://spark-project.org/</url>
|
||||
|
@ -40,27 +40,27 @@
|
|||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-bagel</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-mllib</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-repl</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-streaming</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
@ -121,7 +121,7 @@
|
|||
<id>hadoop2-yarn</id>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-yarn</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
|
|
@ -30,9 +30,9 @@
|
|||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>
|
||||
${project.parent.basedir}/core/src/main/resources/spark/ui/static/
|
||||
${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/
|
||||
</directory>
|
||||
<outputDirectory>/ui-resources/spark/ui/static</outputDirectory>
|
||||
<outputDirectory>/ui-resources/org/apache/spark/ui/static</outputDirectory>
|
||||
<includes>
|
||||
<include>**/*</include>
|
||||
</includes>
|
||||
|
@ -63,10 +63,10 @@
|
|||
<dependencySets>
|
||||
<dependencySet>
|
||||
<includes>
|
||||
<include>org.spark-project:*:jar</include>
|
||||
<include>org.apache.spark:*:jar</include>
|
||||
</includes>
|
||||
<excludes>
|
||||
<exclude>org.spark-project:spark-assembly:jar</exclude>
|
||||
<exclude>org.apache.spark:spark-assembly:jar</exclude>
|
||||
</excludes>
|
||||
</dependencySet>
|
||||
<dependencySet>
|
||||
|
@ -77,7 +77,7 @@
|
|||
<useProjectArtifact>false</useProjectArtifact>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop:*:jar</exclude>
|
||||
<exclude>org.spark-project:*:jar</exclude>
|
||||
<exclude>org.apache.spark:*:jar</exclude>
|
||||
</excludes>
|
||||
</dependencySet>
|
||||
</dependencySets>
|
||||
|
|
|
@ -19,13 +19,13 @@
|
|||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-parent</artifactId>
|
||||
<version>0.8.0-SNAPSHOT</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-bagel</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<name>Spark Project Bagel</name>
|
||||
|
@ -33,7 +33,7 @@
|
|||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core</artifactId>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
|
|
|
@ -15,32 +15,31 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.bagel
|
||||
package org.apache.spark.bagel
|
||||
|
||||
import spark._
|
||||
import spark.SparkContext._
|
||||
import org.apache.spark._
|
||||
import org.apache.spark.SparkContext._
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import storage.StorageLevel
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
object Bagel extends Logging {
|
||||
val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK
|
||||
|
||||
/**
|
||||
* Runs a Bagel program.
|
||||
* @param sc [[spark.SparkContext]] to use for the program.
|
||||
* @param sc [[org.apache.spark.SparkContext]] to use for the program.
|
||||
* @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be
|
||||
* the vertex id.
|
||||
* @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an
|
||||
* empty array, i.e. sc.parallelize(Array[K, Message]()).
|
||||
* @param combiner [[spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one
|
||||
* @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one
|
||||
* message before sending (which often involves network I/O).
|
||||
* @param aggregator [[spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep,
|
||||
* @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep,
|
||||
* and provides the result to each vertex in the next superstep.
|
||||
* @param partitioner [[spark.Partitioner]] partitions values by key
|
||||
* @param partitioner [[org.apache.spark.Partitioner]] partitions values by key
|
||||
* @param numPartitions number of partitions across which to split the graph.
|
||||
* Default is the default parallelism of the SparkContext
|
||||
* @param storageLevel [[spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep.
|
||||
* @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep.
|
||||
* Defaults to caching in memory.
|
||||
* @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex,
|
||||
* optional Aggregator and the current superstep,
|
||||
|
@ -98,7 +97,7 @@ object Bagel extends Logging {
|
|||
verts
|
||||
}
|
||||
|
||||
/** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default storage level */
|
||||
/** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default storage level */
|
||||
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
|
||||
sc: SparkContext,
|
||||
vertices: RDD[(K, V)],
|
||||
|
@ -110,7 +109,7 @@ object Bagel extends Logging {
|
|||
compute: (V, Option[C], Int) => (V, Array[M])
|
||||
): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
|
||||
|
||||
/** Runs a Bagel program with no [[spark.bagel.Aggregator]] */
|
||||
/** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] */
|
||||
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
|
||||
sc: SparkContext,
|
||||
vertices: RDD[(K, V)],
|
||||
|
@ -128,7 +127,7 @@ object Bagel extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]]
|
||||
* Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]]
|
||||
* and default storage level
|
||||
*/
|
||||
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
|
||||
|
@ -141,7 +140,7 @@ object Bagel extends Logging {
|
|||
compute: (V, Option[C], Int) => (V, Array[M])
|
||||
): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
|
||||
|
||||
/** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default [[spark.HashPartitioner]]*/
|
||||
/** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default [[org.apache.spark.HashPartitioner]]*/
|
||||
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
|
||||
sc: SparkContext,
|
||||
vertices: RDD[(K, V)],
|
||||
|
@ -159,8 +158,8 @@ object Bagel extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]],
|
||||
* [[spark.bagel.DefaultCombiner]] and the default storage level
|
||||
* Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]],
|
||||
* [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level
|
||||
*/
|
||||
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
|
||||
sc: SparkContext,
|
||||
|
@ -172,8 +171,8 @@ object Bagel extends Logging {
|
|||
): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
|
||||
|
||||
/**
|
||||
* Runs a Bagel program with no [[spark.bagel.Aggregator]], the default [[spark.HashPartitioner]]
|
||||
* and [[spark.bagel.DefaultCombiner]]
|
||||
* Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], the default [[org.apache.spark.HashPartitioner]]
|
||||
* and [[org.apache.spark.bagel.DefaultCombiner]]
|
||||
*/
|
||||
def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
|
||||
sc: SparkContext,
|
|
@ -15,16 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.bagel
|
||||
package org.apache.spark.bagel
|
||||
|
||||
import org.scalatest.{FunSuite, Assertions, BeforeAndAfter}
|
||||
import org.scalatest.{BeforeAndAfter, FunSuite, Assertions}
|
||||
import org.scalatest.concurrent.Timeouts
|
||||
import org.scalatest.time.SpanSugar._
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
import spark._
|
||||
import storage.StorageLevel
|
||||
import org.apache.spark._
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable
|
||||
class TestMessage(val targetId: String) extends Message[String] with Serializable
|
|
@ -49,4 +49,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
|
|||
fi
|
||||
fi
|
||||
|
||||
"$bin"/spark-daemon.sh start spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
|
||||
"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
|
||||
|
|
|
@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
|
|||
fi
|
||||
fi
|
||||
|
||||
"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@"
|
||||
"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
|
||||
|
|
|
@ -24,4 +24,4 @@ bin=`cd "$bin"; pwd`
|
|||
|
||||
. "$bin/spark-config.sh"
|
||||
|
||||
"$bin"/spark-daemon.sh stop spark.deploy.master.Master 1
|
||||
"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
|
||||
|
|
|
@ -29,9 +29,9 @@ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
|
|||
fi
|
||||
|
||||
if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
|
||||
"$bin"/spark-daemons.sh stop spark.deploy.worker.Worker 1
|
||||
"$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
|
||||
else
|
||||
for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
|
||||
"$bin"/spark-daemons.sh stop spark.deploy.worker.Worker $(( $i + 1 ))
|
||||
"$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
|
||||
done
|
||||
fi
|
||||
|
|
|
@ -19,13 +19,13 @@
|
|||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-parent</artifactId>
|
||||
<version>0.8.0-SNAPSHOT</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<groupId>org.spark-project</groupId>
|
||||
<groupId>org.apache.spark</groupId>
|
||||
<artifactId>spark-core</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<name>Spark Project Core</name>
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
import io.netty.channel.Channel;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
import io.netty.buffer.BufType;
|
||||
import io.netty.channel.ChannelInitializer;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
import io.netty.channel.ChannelHandlerContext;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
import io.netty.channel.socket.SocketChannel;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.network.netty;
|
||||
package org.apache.spark.network.netty;
|
||||
|
||||
|
||||
public interface PathResolver {
|
До Ширина: | Высота: | Размер: 3.5 KiB После Ширина: | Высота: | Размер: 3.5 KiB |
До Ширина: | Высота: | Размер: 14 KiB После Ширина: | Высота: | Размер: 14 KiB |
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
|
||||
|
@ -28,7 +28,7 @@ import scala.collection.generic.Growable
|
|||
*
|
||||
* You must define how to add data, and how to merge two of these together. For some datatypes,
|
||||
* such as a counter, these might be the same operation. In that case, you can use the simpler
|
||||
* [[spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are
|
||||
* [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are
|
||||
* accumulating a set. You will add items to the set, and you will union two sets together.
|
||||
*
|
||||
* @param initialValue initial value of accumulator
|
||||
|
@ -176,7 +176,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
|
|||
def zero(initialValue: R): R = {
|
||||
// We need to clone initialValue, but it's hard to specify that R should also be Cloneable.
|
||||
// Instead we'll serialize it to a buffer and load it back.
|
||||
val ser = (new spark.JavaSerializer).newInstance()
|
||||
val ser = new JavaSerializer().newInstance()
|
||||
val copy = ser.deserialize[R](ser.serialize(initialValue))
|
||||
copy.clear() // In case it contained stuff
|
||||
copy
|
||||
|
@ -184,7 +184,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
|
|||
}
|
||||
|
||||
/**
|
||||
* A simpler value of [[spark.Accumulable]] where the result type being accumulated is the same
|
||||
* A simpler value of [[org.apache.spark.Accumulable]] where the result type being accumulated is the same
|
||||
* as the types of elements being merged.
|
||||
*
|
||||
* @param initialValue initial value of accumulator
|
||||
|
@ -195,7 +195,7 @@ class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T])
|
|||
extends Accumulable[T,T](initialValue, param)
|
||||
|
||||
/**
|
||||
* A simpler version of [[spark.AccumulableParam]] where the only datatype you can add in is the same type
|
||||
* A simpler version of [[org.apache.spark.AccumulableParam]] where the only datatype you can add in is the same type
|
||||
* as the accumulated value. An implicit AccumulatorParam object needs to be available when you create
|
||||
* Accumulators of a specific type.
|
||||
*
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.util.{HashMap => JHashMap}
|
||||
|
|
@ -15,15 +15,15 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.collection.mutable.HashMap
|
||||
|
||||
import spark.executor.{ShuffleReadMetrics, TaskMetrics}
|
||||
import spark.serializer.Serializer
|
||||
import spark.storage.BlockManagerId
|
||||
import spark.util.CompletionIterator
|
||||
import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics}
|
||||
import org.apache.spark.serializer.Serializer
|
||||
import org.apache.spark.storage.BlockManagerId
|
||||
import org.apache.spark.util.CompletionIterator
|
||||
|
||||
|
||||
private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
|
|
@ -15,10 +15,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import scala.collection.mutable.{ArrayBuffer, HashSet}
|
||||
import spark.storage.{BlockManager, StorageLevel}
|
||||
import org.apache.spark.storage.{BlockManager, StorageLevel}
|
||||
|
||||
|
||||
/** Spark class responsible for passing RDDs split contents to the BlockManager and making
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.lang.reflect.Field
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
/**
|
||||
* Base class for dependencies.
|
|
@ -15,13 +15,13 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import spark.partial.BoundedDouble
|
||||
import spark.partial.MeanEvaluator
|
||||
import spark.partial.PartialResult
|
||||
import spark.partial.SumEvaluator
|
||||
import spark.util.StatCounter
|
||||
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.
|
||||
|
@ -34,7 +34,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Return a [[spark.util.StatCounter]] object that captures the mean, variance and count
|
||||
* 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 = {
|
|
@ -15,9 +15,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import spark.storage.BlockManagerId
|
||||
import org.apache.spark.storage.BlockManagerId
|
||||
|
||||
private[spark] class FetchFailedException(
|
||||
taskEndReason: TaskEndReason,
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io.{File}
|
||||
import com.google.common.io.Files
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io.File
|
||||
import java.net.InetAddress
|
|
@ -15,13 +15,13 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
import java.nio.ByteBuffer
|
||||
|
||||
import serializer.{Serializer, SerializerInstance, DeserializationStream, SerializationStream}
|
||||
import spark.util.ByteBufferInputStream
|
||||
import org.apache.spark.util.ByteBufferInputStream
|
||||
|
||||
private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream {
|
||||
val objOut = new ObjectOutputStream(out)
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
import java.nio.ByteBuffer
|
||||
|
@ -24,8 +24,8 @@ 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 spark.broadcast._
|
||||
import spark.storage._
|
||||
import org.apache.spark.broadcast._
|
||||
import org.apache.spark.storage._
|
||||
|
||||
private[spark]
|
||||
class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream {
|
||||
|
@ -104,7 +104,7 @@ trait KryoRegistrator {
|
|||
/**
|
||||
* A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]].
|
||||
*/
|
||||
class KryoSerializer extends spark.serializer.Serializer with Logging {
|
||||
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)
|
||||
|
@ -153,4 +153,4 @@ class KryoSerializer extends spark.serializer.Serializer with Logging {
|
|||
def newInstance(): SerializerInstance = {
|
||||
new KryoSerializerInstance(this)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
|
||||
|
@ -30,9 +30,9 @@ import akka.remote._
|
|||
import akka.util.Duration
|
||||
|
||||
|
||||
import spark.scheduler.MapStatus
|
||||
import spark.storage.BlockManagerId
|
||||
import spark.util.{MetadataCleaner, TimeStampedHashMap}
|
||||
import org.apache.spark.scheduler.MapStatus
|
||||
import org.apache.spark.storage.BlockManagerId
|
||||
import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
|
||||
|
||||
|
||||
private[spark] sealed trait MapOutputTrackerMessage
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.nio.ByteBuffer
|
||||
import java.util.{Date, HashMap => JHashMap}
|
||||
|
@ -40,11 +40,11 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat,
|
|||
RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil}
|
||||
import org.apache.hadoop.security.UserGroupInformation
|
||||
|
||||
import spark.partial.BoundedDouble
|
||||
import spark.partial.PartialResult
|
||||
import spark.rdd._
|
||||
import spark.SparkContext._
|
||||
import spark.Partitioner._
|
||||
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.
|
||||
|
@ -559,7 +559,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
|
|||
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
|
||||
val jobtrackerID = formatter.format(new Date())
|
||||
val stageId = self.id
|
||||
def writeShard(context: spark.TaskContext, iter: Iterator[(K,V)]): Int = {
|
||||
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
|
||||
|
@ -571,7 +571,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
|
|||
committer.setupTask(hadoopContext)
|
||||
val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]]
|
||||
while (iter.hasNext) {
|
||||
val (k, v) = iter.next
|
||||
val (k, v) = iter.next()
|
||||
writer.write(k, v)
|
||||
}
|
||||
writer.close(hadoopContext)
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
/**
|
||||
* A partition of an RDD.
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
/**
|
||||
* An object that defines how the elements in a key-value pair RDD are partitioned by key.
|
||||
|
@ -56,7 +56,7 @@ object Partitioner {
|
|||
}
|
||||
|
||||
/**
|
||||
* A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
|
||||
* A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
|
||||
*
|
||||
* Java arrays have hashCodes that are based on the arrays' identities rather than their contents,
|
||||
* so attempting to partition an RDD[Array[_]] or RDD[(Array[_], _)] using a HashPartitioner will
|
||||
|
@ -79,7 +79,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
|
|||
}
|
||||
|
||||
/**
|
||||
* A [[spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
|
||||
* A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
|
||||
* Determines the ranges by sampling the RDD passed in.
|
||||
*/
|
||||
class RangePartitioner[K <% Ordered[K]: ClassManifest, V](
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.util.Random
|
||||
|
||||
|
@ -31,30 +31,30 @@ import org.apache.hadoop.mapred.TextOutputFormat
|
|||
|
||||
import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
|
||||
|
||||
import spark.Partitioner._
|
||||
import spark.api.java.JavaRDD
|
||||
import spark.partial.BoundedDouble
|
||||
import spark.partial.CountEvaluator
|
||||
import spark.partial.GroupedCountEvaluator
|
||||
import spark.partial.PartialResult
|
||||
import spark.rdd.CoalescedRDD
|
||||
import spark.rdd.CartesianRDD
|
||||
import spark.rdd.FilteredRDD
|
||||
import spark.rdd.FlatMappedRDD
|
||||
import spark.rdd.GlommedRDD
|
||||
import spark.rdd.MappedRDD
|
||||
import spark.rdd.MapPartitionsRDD
|
||||
import spark.rdd.MapPartitionsWithIndexRDD
|
||||
import spark.rdd.PipedRDD
|
||||
import spark.rdd.SampledRDD
|
||||
import spark.rdd.ShuffledRDD
|
||||
import spark.rdd.UnionRDD
|
||||
import spark.rdd.ZippedRDD
|
||||
import spark.rdd.ZippedPartitionsRDD2
|
||||
import spark.rdd.ZippedPartitionsRDD3
|
||||
import spark.rdd.ZippedPartitionsRDD4
|
||||
import spark.storage.StorageLevel
|
||||
import spark.util.BoundedPriorityQueue
|
||||
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._
|
||||
|
||||
|
@ -62,12 +62,12 @@ 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,
|
||||
* [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such
|
||||
* as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations available only on
|
||||
* RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations available on RDDs
|
||||
* [[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 spark.SparkContext._`.
|
||||
* `import org.apache.spark.SparkContext._`.
|
||||
*
|
||||
* Internally, each RDD is characterized by five main properties:
|
||||
*
|
||||
|
@ -893,7 +893,7 @@ abstract class RDD[T: ClassManifest](
|
|||
dependencies.head.rdd.asInstanceOf[RDD[U]]
|
||||
}
|
||||
|
||||
/** The [[spark.SparkContext]] that this RDD was created on. */
|
||||
/** The [[org.apache.spark.SparkContext]] that this RDD was created on. */
|
||||
def context = sc
|
||||
|
||||
// Avoid handling doCheckpoint multiple times to prevent excessive recursion
|
||||
|
@ -929,7 +929,7 @@ abstract class RDD[T: ClassManifest](
|
|||
* 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 [[spark.rdd.UnionRDD]] for an example.
|
||||
* logic. See [[org.apache.spark.rdd.UnionRDD]] for an example.
|
||||
*/
|
||||
protected def clearDependencies() {
|
||||
dependencies_ = null
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.conf.Configuration
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io.EOFException
|
||||
import java.net.URL
|
||||
|
@ -41,7 +41,7 @@ import org.apache.hadoop.io.NullWritable
|
|||
import org.apache.hadoop.io.BytesWritable
|
||||
import org.apache.hadoop.io.Text
|
||||
|
||||
import spark.SparkContext._
|
||||
import org.apache.spark.SparkContext._
|
||||
|
||||
/**
|
||||
* Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile,
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
|
|
@ -15,10 +15,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import spark.executor.TaskMetrics
|
||||
import spark.serializer.Serializer
|
||||
import org.apache.spark.executor.TaskMetrics
|
||||
import org.apache.spark.serializer.Serializer
|
||||
|
||||
|
||||
private[spark] abstract class ShuffleFetcher {
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.lang.reflect.Field
|
||||
import java.lang.reflect.Modifier
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
import java.net.URI
|
||||
|
@ -52,22 +52,22 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
|
|||
|
||||
import org.apache.mesos.MesosNativeLibrary
|
||||
|
||||
import spark.deploy.LocalSparkCluster
|
||||
import spark.partial.{ApproximateEvaluator, PartialResult}
|
||||
import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD,
|
||||
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 spark.scheduler._
|
||||
import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
|
||||
import org.apache.spark.scheduler._
|
||||
import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
|
||||
ClusterScheduler, Schedulable, SchedulingMode}
|
||||
import spark.scheduler.local.LocalScheduler
|
||||
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
|
||||
import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource}
|
||||
import spark.ui.SparkUI
|
||||
import spark.util.{MetadataCleaner, TimeStampedHashMap}
|
||||
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.ui.SparkUI
|
||||
import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
|
||||
import scala.Some
|
||||
import spark.scheduler.StageInfo
|
||||
import spark.storage.RDDInfo
|
||||
import spark.storage.StorageStatus
|
||||
import org.apache.spark.scheduler.StageInfo
|
||||
import org.apache.spark.storage.RDDInfo
|
||||
import org.apache.spark.storage.StorageStatus
|
||||
|
||||
/**
|
||||
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
|
||||
|
@ -494,14 +494,14 @@ class SparkContext(
|
|||
// Methods for creating shared variables
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
|
||||
* Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values
|
||||
* to using the `+=` method. Only the driver can access the accumulator's `value`.
|
||||
*/
|
||||
def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
|
||||
new Accumulator(initialValue, param)
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulable]] shared variable, to which tasks can add values with `+=`.
|
||||
* Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values with `+=`.
|
||||
* Only the driver can access the accumuable's `value`.
|
||||
* @tparam T accumulator type
|
||||
* @tparam R type that can be added to the accumulator
|
||||
|
@ -521,7 +521,7 @@ class SparkContext(
|
|||
}
|
||||
|
||||
/**
|
||||
* Broadcast a read-only variable to the cluster, returning a [[spark.broadcast.Broadcast]] object for
|
||||
* Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.broadcast.Broadcast]] object for
|
||||
* reading it in distributed functions. The variable will be sent to each cluster only once.
|
||||
*/
|
||||
def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal)
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import collection.mutable
|
||||
import serializer.Serializer
|
||||
|
@ -23,15 +23,14 @@ import serializer.Serializer
|
|||
import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
|
||||
import akka.remote.RemoteActorRefProvider
|
||||
|
||||
import spark.broadcast.BroadcastManager
|
||||
import spark.metrics.MetricsSystem
|
||||
import spark.deploy.SparkHadoopUtil
|
||||
import spark.storage.BlockManager
|
||||
import spark.storage.BlockManagerMaster
|
||||
import spark.network.ConnectionManager
|
||||
import spark.serializer.{Serializer, SerializerManager}
|
||||
import spark.util.AkkaUtils
|
||||
import spark.api.python.PythonWorkerFactory
|
||||
import org.apache.spark.broadcast.BroadcastManager
|
||||
import org.apache.spark.metrics.MetricsSystem
|
||||
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.api.python.PythonWorkerFactory
|
||||
|
||||
|
||||
/**
|
||||
|
@ -156,10 +155,10 @@ object SparkEnv extends Logging {
|
|||
val serializerManager = new SerializerManager
|
||||
|
||||
val serializer = serializerManager.setDefault(
|
||||
System.getProperty("spark.serializer", "spark.JavaSerializer"))
|
||||
System.getProperty("spark.serializer", "org.apache.spark.JavaSerializer"))
|
||||
|
||||
val closureSerializer = serializerManager.get(
|
||||
System.getProperty("spark.closure.serializer", "spark.JavaSerializer"))
|
||||
System.getProperty("spark.closure.serializer", "org.apache.spark.JavaSerializer"))
|
||||
|
||||
def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
|
||||
if (isDriver) {
|
||||
|
@ -177,7 +176,7 @@ object SparkEnv extends Logging {
|
|||
|
||||
val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
|
||||
"BlockManagerMaster",
|
||||
new spark.storage.BlockManagerMasterActor(isLocal)))
|
||||
new BlockManagerMasterActor(isLocal)))
|
||||
val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer)
|
||||
|
||||
val connectionManager = blockManager.connectionManager
|
||||
|
@ -194,7 +193,7 @@ object SparkEnv extends Logging {
|
|||
new MapOutputTrackerActor(mapOutputTracker))
|
||||
|
||||
val shuffleFetcher = instantiateClass[ShuffleFetcher](
|
||||
"spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher")
|
||||
"spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher")
|
||||
|
||||
val httpFileServer = new HttpFileServer()
|
||||
httpFileServer.initialize()
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
class SparkException(message: String, cause: Throwable)
|
||||
extends Exception(message, cause) {
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark;
|
||||
package org.apache.spark;
|
||||
|
||||
import java.io.File;
|
||||
|
|
@ -25,8 +25,8 @@ import java.text.NumberFormat
|
|||
import java.io.IOException
|
||||
import java.util.Date
|
||||
|
||||
import spark.Logging
|
||||
import spark.SerializableWritable
|
||||
import org.apache.spark.Logging
|
||||
import org.apache.spark.SerializableWritable
|
||||
|
||||
/**
|
||||
* Internal helper class that saves an RDD using a Hadoop OutputFormat. This is only public
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import executor.TaskMetrics
|
||||
import scala.collection.mutable.ArrayBuffer
|
|
@ -15,10 +15,10 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import spark.executor.TaskMetrics
|
||||
import spark.storage.BlockManagerId
|
||||
import org.apache.spark.executor.TaskMetrics
|
||||
import org.apache.spark.storage.BlockManagerId
|
||||
|
||||
/**
|
||||
* Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import org.apache.mesos.Protos.{TaskState => MesosTaskState}
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark
|
||||
package org.apache.spark
|
||||
|
||||
import java.io._
|
||||
import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket}
|
||||
|
@ -33,8 +33,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
|
|||
|
||||
import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
|
||||
|
||||
import spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
|
||||
import spark.deploy.SparkHadoopUtil
|
||||
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
|
||||
import org.apache.spark.deploy.SparkHadoopUtil
|
||||
import java.nio.ByteBuffer
|
||||
|
||||
|
|
@ -15,16 +15,16 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java
|
||||
package org.apache.spark.api.java
|
||||
|
||||
import spark.RDD
|
||||
import spark.SparkContext.doubleRDDToDoubleRDDFunctions
|
||||
import spark.api.java.function.{Function => JFunction}
|
||||
import spark.util.StatCounter
|
||||
import spark.partial.{BoundedDouble, PartialResult}
|
||||
import spark.storage.StorageLevel
|
||||
import org.apache.spark.RDD
|
||||
import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions
|
||||
import org.apache.spark.api.java.function.{Function => JFunction}
|
||||
import org.apache.spark.util.StatCounter
|
||||
import org.apache.spark.partial.{BoundedDouble, PartialResult}
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
import java.lang.Double
|
||||
import spark.Partitioner
|
||||
import org.apache.spark.Partitioner
|
||||
|
||||
class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
|
||||
|
||||
|
@ -119,7 +119,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
|
|||
def sum(): Double = srdd.sum()
|
||||
|
||||
/**
|
||||
* Return a [[spark.util.StatCounter]] object that captures the mean, variance and count
|
||||
* 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 = srdd.stats()
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java
|
||||
package org.apache.spark.api.java
|
||||
|
||||
import java.util.{List => JList}
|
||||
import java.util.Comparator
|
||||
|
@ -30,17 +30,17 @@ import org.apache.hadoop.mapred.OutputFormat
|
|||
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
|
||||
import spark.HashPartitioner
|
||||
import spark.Partitioner
|
||||
import spark.Partitioner._
|
||||
import spark.RDD
|
||||
import spark.SparkContext.rddToPairRDDFunctions
|
||||
import spark.api.java.function.{Function2 => JFunction2}
|
||||
import spark.api.java.function.{Function => JFunction}
|
||||
import spark.partial.BoundedDouble
|
||||
import spark.partial.PartialResult
|
||||
import spark.rdd.OrderedRDDFunctions
|
||||
import spark.storage.StorageLevel
|
||||
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.OrderedRDDFunctions
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
|
||||
class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K],
|
|
@ -15,11 +15,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java
|
||||
package org.apache.spark.api.java
|
||||
|
||||
import spark._
|
||||
import spark.api.java.function.{Function => JFunction}
|
||||
import spark.storage.StorageLevel
|
||||
import org.apache.spark._
|
||||
import org.apache.spark.api.java.function.{Function => JFunction}
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
class JavaRDD[T](val rdd: RDD[T])(implicit val classManifest: ClassManifest[T]) extends
|
||||
JavaRDDLike[T, JavaRDD[T]] {
|
|
@ -15,18 +15,18 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java
|
||||
package org.apache.spark.api.java
|
||||
|
||||
import java.util.{List => JList, Comparator}
|
||||
import scala.Tuple2
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
import org.apache.hadoop.io.compress.CompressionCodec
|
||||
import spark.{SparkContext, Partition, RDD, TaskContext}
|
||||
import spark.api.java.JavaPairRDD._
|
||||
import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
|
||||
import spark.partial.{PartialResult, BoundedDouble}
|
||||
import spark.storage.StorageLevel
|
||||
import org.apache.spark.{SparkContext, Partition, RDD, TaskContext}
|
||||
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
|
||||
|
||||
|
||||
|
@ -40,7 +40,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
|
|||
/** Set of partitions in this RDD. */
|
||||
def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq)
|
||||
|
||||
/** The [[spark.SparkContext]] that this RDD was created on. */
|
||||
/** The [[org.apache.spark.SparkContext]] that this RDD was created on. */
|
||||
def context: SparkContext = rdd.context
|
||||
|
||||
/** A unique ID for this RDD (within its SparkContext). */
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java
|
||||
package org.apache.spark.api.java
|
||||
|
||||
import java.util.{Map => JMap}
|
||||
|
||||
|
@ -27,15 +27,15 @@ import org.apache.hadoop.mapred.InputFormat
|
|||
import org.apache.hadoop.mapred.JobConf
|
||||
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
|
||||
|
||||
import spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext}
|
||||
import spark.SparkContext.IntAccumulatorParam
|
||||
import spark.SparkContext.DoubleAccumulatorParam
|
||||
import spark.broadcast.Broadcast
|
||||
import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, 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
|
||||
|
||||
/**
|
||||
* A Java-friendly version of [[spark.SparkContext]] that returns [[spark.api.java.JavaRDD]]s and
|
||||
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and
|
||||
* works with Java collections instead of Scala ones.
|
||||
*/
|
||||
class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
|
||||
|
@ -283,48 +283,48 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
|
|||
}
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulator]] integer variable, which tasks can "add" values
|
||||
* Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values
|
||||
* to using the `add` method. Only the master can access the accumulator's `value`.
|
||||
*/
|
||||
def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] =
|
||||
sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]]
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulator]] double variable, which tasks can "add" values
|
||||
* Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values
|
||||
* to using the `add` method. Only the master can access the accumulator's `value`.
|
||||
*/
|
||||
def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] =
|
||||
sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]]
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulator]] integer variable, which tasks can "add" values
|
||||
* Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values
|
||||
* to using the `add` method. Only the master can access the accumulator's `value`.
|
||||
*/
|
||||
def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue)
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulator]] double variable, which tasks can "add" values
|
||||
* Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values
|
||||
* to using the `add` method. Only the master can access the accumulator's `value`.
|
||||
*/
|
||||
def accumulator(initialValue: Double): Accumulator[java.lang.Double] =
|
||||
doubleAccumulator(initialValue)
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
|
||||
* Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values
|
||||
* to using the `add` method. Only the master can access the accumulator's `value`.
|
||||
*/
|
||||
def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] =
|
||||
sc.accumulator(initialValue)(accumulatorParam)
|
||||
|
||||
/**
|
||||
* Create an [[spark.Accumulable]] shared variable of the given type, to which tasks can
|
||||
* Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks can
|
||||
* "add" values with `add`. Only the master can access the accumuable's `value`.
|
||||
*/
|
||||
def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] =
|
||||
sc.accumulable(initialValue)(param)
|
||||
|
||||
/**
|
||||
* Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for
|
||||
* Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.Broadcast]] object for
|
||||
* reading it in distributed functions. The variable will be sent to each cluster only once.
|
||||
*/
|
||||
def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java;
|
||||
package org.apache.spark.api.java;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java
|
||||
package org.apache.spark.api.java
|
||||
|
||||
import com.google.common.base.Optional
|
||||
|
|
@ -15,9 +15,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java;
|
||||
package org.apache.spark.api.java;
|
||||
|
||||
import spark.storage.StorageLevel;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
/**
|
||||
* Expose some commonly useful storage level constants.
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function;
|
||||
package org.apache.spark.api.java.function;
|
||||
|
||||
|
||||
import scala.runtime.AbstractFunction1;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function;
|
||||
package org.apache.spark.api.java.function;
|
||||
|
||||
|
||||
import scala.runtime.AbstractFunction1;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function
|
||||
package org.apache.spark.api.java.function
|
||||
|
||||
/**
|
||||
* A function that returns zero or more output records from each input record.
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function
|
||||
package org.apache.spark.api.java.function
|
||||
|
||||
/**
|
||||
* A function that takes two inputs and returns zero or more output records.
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function;
|
||||
package org.apache.spark.api.java.function;
|
||||
|
||||
import scala.reflect.ClassManifest;
|
||||
import scala.reflect.ClassManifest$;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function;
|
||||
package org.apache.spark.api.java.function;
|
||||
|
||||
import scala.reflect.ClassManifest;
|
||||
import scala.reflect.ClassManifest$;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function;
|
||||
package org.apache.spark.api.java.function;
|
||||
|
||||
import scala.Tuple2;
|
||||
import scala.reflect.ClassManifest;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function;
|
||||
package org.apache.spark.api.java.function;
|
||||
|
||||
import scala.Tuple2;
|
||||
import scala.reflect.ClassManifest;
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function
|
||||
package org.apache.spark.api.java.function
|
||||
|
||||
/**
|
||||
* A function with no return value.
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function
|
||||
package org.apache.spark.api.java.function
|
||||
|
||||
import scala.runtime.AbstractFunction1
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.java.function
|
||||
package org.apache.spark.api.java.function
|
||||
|
||||
import scala.runtime.AbstractFunction2
|
||||
|
|
@ -15,14 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.python
|
||||
package org.apache.spark.api.python
|
||||
|
||||
import spark.Partitioner
|
||||
import spark.Utils
|
||||
import org.apache.spark.Partitioner
|
||||
import org.apache.spark.Utils
|
||||
import java.util.Arrays
|
||||
|
||||
/**
|
||||
* A [[spark.Partitioner]] that performs handling of byte arrays, for use by the Python API.
|
||||
* A [[org.apache.spark.Partitioner]] that performs handling of byte arrays, for use by the Python API.
|
||||
*
|
||||
* Stores the unique id() of the Python-side partitioning function so that it is incorporated into
|
||||
* equality comparisons. Correctness requires that the id is a unique identifier for the
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.python
|
||||
package org.apache.spark.api.python
|
||||
|
||||
import java.io._
|
||||
import java.net._
|
||||
|
@ -23,10 +23,10 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio
|
|||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
|
||||
import spark.broadcast.Broadcast
|
||||
import spark._
|
||||
import spark.rdd.PipedRDD
|
||||
import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
|
||||
import org.apache.spark.broadcast.Broadcast
|
||||
import org.apache.spark._
|
||||
import org.apache.spark.rdd.PipedRDD
|
||||
|
||||
|
||||
private[spark] class PythonRDD[T: ClassManifest](
|
||||
|
@ -298,7 +298,7 @@ private object Pickle {
|
|||
val APPENDS: Byte = 'e'
|
||||
}
|
||||
|
||||
private class BytesToString extends spark.api.java.function.Function[Array[Byte], String] {
|
||||
private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] {
|
||||
override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8")
|
||||
}
|
||||
|
|
@ -15,14 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.api.python
|
||||
package org.apache.spark.api.python
|
||||
|
||||
import java.io.{File, DataInputStream, IOException}
|
||||
import java.net.{Socket, SocketException, InetAddress}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
import spark._
|
||||
import org.apache.spark._
|
||||
|
||||
private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String])
|
||||
extends Logging {
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
import java.io._
|
||||
import java.net._
|
||||
|
@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger
|
|||
import scala.collection.mutable.{ListBuffer, Map, Set}
|
||||
import scala.math
|
||||
|
||||
import spark._
|
||||
import spark.storage.StorageLevel
|
||||
import org.apache.spark._
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
|
||||
extends Broadcast[T](id)
|
|
@ -15,12 +15,12 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
import java.io._
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
import spark._
|
||||
import org.apache.spark._
|
||||
|
||||
abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
|
||||
def value: T
|
||||
|
@ -28,7 +28,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
|
|||
// We cannot have an abstract readObject here due to some weird issues with
|
||||
// readObject having to be 'private' in sub-classes.
|
||||
|
||||
override def toString = "spark.Broadcast(" + id + ")"
|
||||
override def toString = "Broadcast(" + id + ")"
|
||||
}
|
||||
|
||||
private[spark]
|
||||
|
@ -44,7 +44,7 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable
|
|||
synchronized {
|
||||
if (!initialized) {
|
||||
val broadcastFactoryClass = System.getProperty(
|
||||
"spark.broadcast.factory", "spark.broadcast.HttpBroadcastFactory")
|
||||
"spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
|
||||
|
||||
broadcastFactory =
|
||||
Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
/**
|
||||
* An interface for all the broadcast implementations in Spark (to allow
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream}
|
||||
import java.net.URL
|
||||
|
@ -23,10 +23,10 @@ import java.net.URL
|
|||
import it.unimi.dsi.fastutil.io.FastBufferedInputStream
|
||||
import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
|
||||
|
||||
import spark.{HttpServer, Logging, SparkEnv, Utils}
|
||||
import spark.io.CompressionCodec
|
||||
import spark.storage.StorageLevel
|
||||
import spark.util.{MetadataCleaner, TimeStampedHashSet}
|
||||
import org.apache.spark.{HttpServer, Logging, SparkEnv, Utils}
|
||||
import org.apache.spark.io.CompressionCodec
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
import org.apache.spark.util.{MetadataCleaner, TimeStampedHashSet}
|
||||
|
||||
|
||||
private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
import java.io._
|
||||
import java.net._
|
||||
|
@ -23,7 +23,7 @@ import java.util.Random
|
|||
|
||||
import scala.collection.mutable.Map
|
||||
|
||||
import spark._
|
||||
import org.apache.spark._
|
||||
|
||||
private object MultiTracker
|
||||
extends Logging {
|
|
@ -15,11 +15,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
import java.util.BitSet
|
||||
|
||||
import spark._
|
||||
import org.apache.spark._
|
||||
|
||||
/**
|
||||
* Used to keep and pass around information of peers involved in a broadcast
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.broadcast
|
||||
package org.apache.spark.broadcast
|
||||
|
||||
import java.io._
|
||||
import java.net._
|
||||
|
@ -24,8 +24,8 @@ import java.util.{Comparator, Random, UUID}
|
|||
import scala.collection.mutable.{ListBuffer, Map, Set}
|
||||
import scala.math
|
||||
|
||||
import spark._
|
||||
import spark.storage.StorageLevel
|
||||
import org.apache.spark._
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
|
||||
extends Broadcast[T](id) with Logging with Serializable {
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
private[spark] class ApplicationDescription(
|
||||
val name: String,
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
import scala.collection.Map
|
||||
|
|
@ -15,14 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
import scala.collection.immutable.List
|
||||
|
||||
import spark.Utils
|
||||
import spark.deploy.ExecutorState.ExecutorState
|
||||
import spark.deploy.master.{WorkerInfo, ApplicationInfo}
|
||||
import spark.deploy.worker.ExecutorRunner
|
||||
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
|
||||
|
||||
|
||||
private[deploy] sealed trait DeployMessage extends Serializable
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
private[spark] object ExecutorState
|
||||
extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") {
|
|
@ -15,13 +15,13 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
import net.liftweb.json.JsonDSL._
|
||||
|
||||
import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
|
||||
import spark.deploy.master.{ApplicationInfo, WorkerInfo}
|
||||
import spark.deploy.worker.ExecutorRunner
|
||||
import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
|
||||
import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
|
||||
import org.apache.spark.deploy.worker.ExecutorRunner
|
||||
|
||||
|
||||
private[spark] object JsonProtocol {
|
|
@ -15,14 +15,14 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
|
||||
|
||||
import spark.deploy.worker.Worker
|
||||
import spark.deploy.master.Master
|
||||
import spark.util.AkkaUtils
|
||||
import spark.{Logging, Utils}
|
||||
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 scala.collection.mutable.ArrayBuffer
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy
|
||||
package org.apache.spark.deploy
|
||||
|
||||
import java.text.SimpleDateFormat
|
||||
import java.util.Date
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.client
|
||||
package org.apache.spark.deploy.client
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
|
@ -28,10 +28,10 @@ import akka.remote.RemoteClientLifeCycleEvent
|
|||
import akka.remote.RemoteClientShutdown
|
||||
import akka.dispatch.Await
|
||||
|
||||
import spark.Logging
|
||||
import spark.deploy.{ApplicationDescription, ExecutorState}
|
||||
import spark.deploy.DeployMessages._
|
||||
import spark.deploy.master.Master
|
||||
import org.apache.spark.Logging
|
||||
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
|
||||
import org.apache.spark.deploy.DeployMessages._
|
||||
import org.apache.spark.deploy.master.Master
|
||||
|
||||
|
||||
/**
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.client
|
||||
package org.apache.spark.deploy.client
|
||||
|
||||
/**
|
||||
* Callbacks invoked by deploy client when various events happen. There are currently four events:
|
|
@ -15,11 +15,11 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.client
|
||||
package org.apache.spark.deploy.client
|
||||
|
||||
import spark.util.AkkaUtils
|
||||
import spark.{Logging, Utils}
|
||||
import spark.deploy.{Command, ApplicationDescription}
|
||||
import org.apache.spark.util.AkkaUtils
|
||||
import org.apache.spark.{Logging, Utils}
|
||||
import org.apache.spark.deploy.{Command, ApplicationDescription}
|
||||
|
||||
private[spark] object TestClient {
|
||||
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.client
|
||||
package org.apache.spark.deploy.client
|
||||
|
||||
private[spark] object TestExecutor {
|
||||
def main(args: Array[String]) {
|
|
@ -15,9 +15,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.master
|
||||
package org.apache.spark.deploy.master
|
||||
|
||||
import spark.deploy.ApplicationDescription
|
||||
import org.apache.spark.deploy.ApplicationDescription
|
||||
import java.util.Date
|
||||
import akka.actor.ActorRef
|
||||
import scala.collection.mutable
|
|
@ -1,8 +1,8 @@
|
|||
package spark.deploy.master
|
||||
package org.apache.spark.deploy.master
|
||||
|
||||
import com.codahale.metrics.{Gauge, MetricRegistry}
|
||||
|
||||
import spark.metrics.source.Source
|
||||
import org.apache.spark.metrics.source.Source
|
||||
|
||||
class ApplicationSource(val application: ApplicationInfo) extends Source {
|
||||
val metricRegistry = new MetricRegistry()
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.master
|
||||
package org.apache.spark.deploy.master
|
||||
|
||||
private[spark] object ApplicationState
|
||||
extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
|
|
@ -15,9 +15,9 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package spark.deploy.master
|
||||
package org.apache.spark.deploy.master
|
||||
|
||||
import spark.deploy.ExecutorState
|
||||
import org.apache.spark.deploy.ExecutorState
|
||||
|
||||
private[spark] class ExecutorInfo(
|
||||
val id: Int,
|
Некоторые файлы не были показаны из-за слишком большого количества измененных файлов Показать больше
Загрузка…
Ссылка в новой задаче