Merging changes between Reynold's branch and Joey's modifications.
This commit is contained in:
commit
a3fb29938c
2
.gitignore
vendored
2
.gitignore
vendored
|
@ -38,3 +38,5 @@ dependency-reduced-pom.xml
|
|||
.ensime_lucene
|
||||
checkpoint
|
||||
derby.log
|
||||
dist/
|
||||
spark-*-bin.tar.gz
|
||||
|
|
225
LICENSE
225
LICENSE
|
@ -1,27 +1,202 @@
|
|||
Copyright (c) 2010, Regents of the University of California.
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions
|
||||
are met:
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above copyright
|
||||
notice, this list of conditions and the following disclaimer in the
|
||||
documentation and/or other materials provided with the distribution.
|
||||
* Neither the name of the University of California, Berkeley nor the
|
||||
names of its contributors may be used to endorse or promote
|
||||
products derived from this software without specific prior written
|
||||
permission.
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
|
||||
TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
|
||||
PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
|
||||
LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
|
||||
NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
|
||||
SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
Licensed 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.
|
||||
|
|
5
NOTICE
Normal file
5
NOTICE
Normal file
|
@ -0,0 +1,5 @@
|
|||
Apache Spark
|
||||
Copyright 2013 The Apache Software Foundation.
|
||||
|
||||
This product includes software developed at
|
||||
The Apache Software Foundation (http://www.apache.org/).
|
77
README.md
77
README.md
|
@ -1 +1,78 @@
|
|||
|
||||
This is a preview of GraphX that we are actively working....
|
||||
|
||||
# Spark
|
||||
|
||||
Lightning-Fast Cluster Computing - <http://www.spark-project.org/>
|
||||
|
||||
|
||||
## Online Documentation
|
||||
|
||||
You can find the latest Spark documentation, including a programming
|
||||
guide, on the project webpage at <http://spark-project.org/documentation.html>.
|
||||
This README file only contains basic setup instructions.
|
||||
|
||||
|
||||
## Building
|
||||
|
||||
Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is
|
||||
built using Simple Build Tool (SBT), which is packaged with it. To build
|
||||
Spark and its example programs, run:
|
||||
|
||||
sbt/sbt package
|
||||
|
||||
Spark also supports building using Maven. If you would like to build using Maven,
|
||||
see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html)
|
||||
in the spark documentation..
|
||||
|
||||
To run Spark, you will need to have Scala's bin directory in your `PATH`, or
|
||||
you will need to set the `SCALA_HOME` environment variable to point to where
|
||||
you've installed Scala. Scala must be accessible through one of these
|
||||
methods on your cluster's worker nodes as well as its master.
|
||||
|
||||
To run one of the examples, use `./run <class> <params>`. For example:
|
||||
|
||||
./run spark.examples.SparkLR local[2]
|
||||
|
||||
will run the Logistic Regression example locally on 2 CPUs.
|
||||
|
||||
Each of the example programs prints usage help if no params are given.
|
||||
|
||||
All of the Spark samples take a `<host>` parameter that is the cluster URL
|
||||
to connect to. This can be a mesos:// or spark:// URL, or "local" to run
|
||||
locally with one thread, or "local[N]" to run locally with N threads.
|
||||
|
||||
|
||||
## A Note About Hadoop Versions
|
||||
|
||||
Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported
|
||||
storage systems. Because the HDFS API has changed in different versions of
|
||||
Hadoop, you must build Spark against the same version that your cluster runs.
|
||||
You can change the version by setting the `HADOOP_VERSION` variable at the top
|
||||
of `project/SparkBuild.scala`, then rebuilding Spark.
|
||||
|
||||
|
||||
## Configuration
|
||||
|
||||
Please refer to the "Configuration" guide in the online documentation for a
|
||||
full overview on how to configure Spark. At the minimum, you will need to
|
||||
create a `conf/spark-env.sh` script (copy `conf/spark-env.sh.template`) and
|
||||
set the following two variables:
|
||||
|
||||
- `SCALA_HOME`: Location where Scala is installed.
|
||||
|
||||
- `MESOS_NATIVE_LIBRARY`: Your Mesos library (only needed if you want to run
|
||||
on Mesos). For example, this might be `/usr/local/lib/libmesos.so` on Linux.
|
||||
|
||||
|
||||
## Contributing to Spark
|
||||
|
||||
Contributions via GitHub pull requests are gladly accepted from their original
|
||||
author. Along with any pull requests, please state that the contribution is
|
||||
your original work and that you license the work to the project under the
|
||||
project's open source license. Whether or not you state this explicitly, by
|
||||
submitting any copyrighted material via pull request, email, or other means
|
||||
you agree to license the material under the project's open source license and
|
||||
warrant that you have the legal authority to do so.
|
||||
|
||||
|
||||
|
|
13
assembly/README
Normal file
13
assembly/README
Normal file
|
@ -0,0 +1,13 @@
|
|||
This is an assembly module for Spark project.
|
||||
|
||||
It creates a single tar.gz file that includes all needed dependency of the project
|
||||
except for org.apache.hadoop.* jars that are supposed to be available from the
|
||||
deployed Hadoop cluster.
|
||||
|
||||
This module is off by default to avoid spending extra time on top of repl-bin
|
||||
module. To activate it specify the profile in the command line
|
||||
-Passembly
|
||||
|
||||
In case you want to avoid building time-expensive repl-bin module, that shaders
|
||||
all the dependency into a big flat jar supplement maven command with
|
||||
-DnoExpensive
|
92
assembly/pom.xml
Normal file
92
assembly/pom.xml
Normal file
|
@ -0,0 +1,92 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<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>
|
||||
<artifactId>spark-parent</artifactId>
|
||||
<version>0.8.0-SNAPSHOT</version>
|
||||
<relativePath>../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-assembly</artifactId>
|
||||
<name>Spark Project Assembly</name>
|
||||
<url>http://spark-project.org/</url>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-assembly-plugin</artifactId>
|
||||
<version>2.4</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>dist</id>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>single</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<descriptors>
|
||||
<descriptor>src/main/assembly/assembly.xml</descriptor>
|
||||
</descriptors>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<profiles>
|
||||
<profile>
|
||||
<id>hadoop1</id>
|
||||
<properties>
|
||||
<classifier.name>hadoop1</classifier.name>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>hadoop2</id>
|
||||
<properties>
|
||||
<classifier.name>hadoop2</classifier.name>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>hadoop2-yarn</id>
|
||||
<properties>
|
||||
<classifier.name>hadoop2-yarn</classifier.name>
|
||||
</properties>
|
||||
</profile>
|
||||
</profiles>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-core</artifactId>
|
||||
<classifier>${classifier.name}</classifier>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-bagel</artifactId>
|
||||
<classifier>${classifier.name}</classifier>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-mllib</artifactId>
|
||||
<classifier>${classifier.name}</classifier>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-repl</artifactId>
|
||||
<classifier>${classifier.name}</classifier>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.spark-project</groupId>
|
||||
<artifactId>spark-streaming</artifactId>
|
||||
<classifier>${classifier.name}</classifier>
|
||||
<version>${project.version}</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
68
assembly/src/main/assembly/assembly.xml
Normal file
68
assembly/src/main/assembly/assembly.xml
Normal file
|
@ -0,0 +1,68 @@
|
|||
<assembly>
|
||||
<id>dist</id>
|
||||
<formats>
|
||||
<format>tar.gz</format>
|
||||
<format>dir</format>
|
||||
</formats>
|
||||
<includeBaseDirectory>false</includeBaseDirectory>
|
||||
|
||||
<fileSets>
|
||||
<fileSet>
|
||||
<includes>
|
||||
<include>README</include>
|
||||
</includes>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>
|
||||
${project.parent.basedir}/core/src/main/resources/spark/ui/static/
|
||||
</directory>
|
||||
<outputDirectory>/ui-resources/spark/ui/static</outputDirectory>
|
||||
<includes>
|
||||
<include>**/*</include>
|
||||
</includes>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>
|
||||
${project.parent.basedir}/bin/
|
||||
</directory>
|
||||
<outputDirectory>/bin</outputDirectory>
|
||||
<includes>
|
||||
<include>**/*</include>
|
||||
</includes>
|
||||
</fileSet>
|
||||
<fileSet>
|
||||
<directory>
|
||||
${project.parent.basedir}
|
||||
</directory>
|
||||
<outputDirectory>/bin</outputDirectory>
|
||||
<includes>
|
||||
<include>run*</include>
|
||||
<include>spark-shell*</include>
|
||||
<include>spark-executor*</include>
|
||||
</includes>
|
||||
</fileSet>
|
||||
</fileSets>
|
||||
|
||||
<dependencySets>
|
||||
<dependencySet>
|
||||
<includes>
|
||||
<include>org.spark-project:*:jar</include>
|
||||
</includes>
|
||||
<excludes>
|
||||
<exclude>org.spark-project:spark-assembly:jar</exclude>
|
||||
</excludes>
|
||||
</dependencySet>
|
||||
<dependencySet>
|
||||
<outputDirectory>lib</outputDirectory>
|
||||
<useTransitiveDependencies>true</useTransitiveDependencies>
|
||||
<unpack>false</unpack>
|
||||
<scope>runtime</scope>
|
||||
<useProjectArtifact>false</useProjectArtifact>
|
||||
<excludes>
|
||||
<exclude>org.apache.hadoop:*:jar</exclude>
|
||||
<exclude>org.spark-project:*:jar</exclude>
|
||||
</excludes>
|
||||
</dependencySet>
|
||||
</dependencySets>
|
||||
|
||||
</assembly>
|
|
@ -1,4 +1,21 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<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>
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.bagel
|
||||
|
||||
import spark._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.bagel.examples
|
||||
|
||||
import spark._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.bagel.examples
|
||||
|
||||
import spark._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.bagel.examples
|
||||
|
||||
import spark._
|
||||
|
|
|
@ -1,4 +1,21 @@
|
|||
# Set everything to be logged to the file bagel/target/unit-tests.log
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Set everything to be logged to the file bagel/target/unit-tests.log
|
||||
log4j.rootCategory=INFO, file
|
||||
log4j.appender.file=org.apache.log4j.FileAppender
|
||||
log4j.appender.file.append=false
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.bagel
|
||||
|
||||
import org.scalatest.{FunSuite, Assertions, BeforeAndAfter}
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
@echo off
|
||||
|
||||
rem
|
||||
rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
rem contributor license agreements. See the NOTICE file distributed with
|
||||
rem this work for additional information regarding copyright ownership.
|
||||
rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
rem (the "License"); you may not use this file except in compliance with
|
||||
rem the License. You may obtain a copy of the License at
|
||||
rem
|
||||
rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
rem
|
||||
rem Unless required by applicable law or agreed to in writing, software
|
||||
rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
rem See the License for the specific language governing permissions and
|
||||
rem limitations under the License.
|
||||
rem
|
||||
|
||||
rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
|
||||
rem script and the ExecutorRunner in standalone cluster mode.
|
||||
|
||||
|
@ -15,6 +32,8 @@ set CORE_DIR=%FWDIR%core
|
|||
set REPL_DIR=%FWDIR%repl
|
||||
set EXAMPLES_DIR=%FWDIR%examples
|
||||
set BAGEL_DIR=%FWDIR%bagel
|
||||
set MLLIB_DIR=%FWDIR%mllib
|
||||
set TOOLS_DIR=%FWDIR%tools
|
||||
set STREAMING_DIR=%FWDIR%streaming
|
||||
set PYSPARK_DIR=%FWDIR%python
|
||||
|
||||
|
@ -29,6 +48,8 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\*
|
|||
set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\*
|
||||
set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\*
|
||||
set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes
|
||||
set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes
|
||||
set CLASSPATH=%CLASSPATH%;%TOOLS_DIR%\target\scala-%SCALA_VERSION%\classes
|
||||
|
||||
rem Add hadoop conf dir - else FileSystem.*, etc fail
|
||||
rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
|
||||
|
|
|
@ -1,4 +1,21 @@
|
|||
#!/bin/bash
|
||||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
|
||||
# script and the ExecutorRunner in standalone cluster mode.
|
||||
|
@ -18,52 +35,69 @@ REPL_DIR="$FWDIR/repl"
|
|||
REPL_BIN_DIR="$FWDIR/repl-bin"
|
||||
EXAMPLES_DIR="$FWDIR/examples"
|
||||
BAGEL_DIR="$FWDIR/bagel"
|
||||
MLLIB_DIR="$FWDIR/mllib"
|
||||
TOOLS_DIR="$FWDIR/tools"
|
||||
GRAPH_DIR="$FWDIR/graph"
|
||||
STREAMING_DIR="$FWDIR/streaming"
|
||||
PYSPARK_DIR="$FWDIR/python"
|
||||
|
||||
# Build up classpath
|
||||
CLASSPATH="$SPARK_CLASSPATH"
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/conf"
|
||||
CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
if [ -n "$SPARK_TESTING" ] ; then
|
||||
CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
|
||||
CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
|
||||
CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
|
||||
if [ -e "$FWDIR/lib_managed" ]; then
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*"
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
|
||||
# Add the shaded JAR for Maven builds
|
||||
if [ -e $REPL_BIN_DIR/target ]; then
|
||||
for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
|
||||
|
||||
function dev_classpath {
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/conf"
|
||||
CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
if [ -n "$SPARK_TESTING" ] ; then
|
||||
CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
|
||||
CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
|
||||
CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
|
||||
if [ -e "$FWDIR/lib_managed" ]; then
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*"
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
|
||||
# Add the shaded JAR for Maven builds
|
||||
if [ -e $REPL_BIN_DIR/target ]; then
|
||||
for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
|
||||
CLASSPATH="$CLASSPATH:$jar"
|
||||
done
|
||||
# The shaded JAR doesn't contain examples, so include those separately
|
||||
EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
|
||||
CLASSPATH+=":$EXAMPLES_JAR"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$GRAPH_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$TOOLS_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
|
||||
CLASSPATH="$CLASSPATH:$jar"
|
||||
done
|
||||
# The shaded JAR doesn't contain examples, so include those separately
|
||||
EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
|
||||
CLASSPATH+=":$EXAMPLES_JAR"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
CLASSPATH="$CLASSPATH:$GRAPH_DIR/target/scala-$SCALA_VERSION/classes"
|
||||
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
|
||||
CLASSPATH="$CLASSPATH:$jar"
|
||||
done
|
||||
# Add Scala standard library
|
||||
if [ -z "$SCALA_LIBRARY_PATH" ]; then
|
||||
if [ -z "$SCALA_HOME" ]; then
|
||||
echo "SCALA_HOME is not set" >&2
|
||||
exit 1
|
||||
fi
|
||||
SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar"
|
||||
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar"
|
||||
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar"
|
||||
}
|
||||
|
||||
# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
|
||||
# to avoid the -sources and -doc packages that are built by publish-local.
|
||||
if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then
|
||||
# Use the JAR from the SBT build
|
||||
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar`
|
||||
fi
|
||||
if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then
|
||||
# Use the JAR from the Maven build
|
||||
export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
|
||||
function release_classpath {
|
||||
CLASSPATH="$CLASSPATH:$FWDIR/jars/*"
|
||||
}
|
||||
|
||||
if [ -f "$FWDIR/RELEASE" ]; then
|
||||
release_classpath
|
||||
else
|
||||
dev_classpath
|
||||
fi
|
||||
|
||||
# Add hadoop conf dir - else FileSystem.*, etc fail !
|
||||
|
@ -76,16 +110,4 @@ if [ "x" != "x$YARN_CONF_DIR" ]; then
|
|||
CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
|
||||
fi
|
||||
|
||||
# Add Scala standard library
|
||||
if [ -z "$SCALA_LIBRARY_PATH" ]; then
|
||||
if [ -z "$SCALA_HOME" ]; then
|
||||
echo "SCALA_HOME is not set" >&2
|
||||
exit 1
|
||||
fi
|
||||
SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
|
||||
fi
|
||||
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar"
|
||||
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar"
|
||||
CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar"
|
||||
|
||||
echo "$CLASSPATH"
|
||||
|
|
|
@ -1,7 +1,5 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
# This Spark deploy script is a modified version of the Apache Hadoop deploy
|
||||
# script, available under the Apache 2 license:
|
||||
#
|
||||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
|
@ -10,13 +8,14 @@
|
|||
# (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
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Run a shell command on all slave hosts.
|
||||
#
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# included in all the spark scripts with source command
|
||||
# should not be executable directly
|
||||
# also should not be passed any arguments, since we need original $*
|
||||
|
@ -16,4 +33,4 @@ this="$config_bin/$script"
|
|||
|
||||
export SPARK_PREFIX=`dirname "$this"`/..
|
||||
export SPARK_HOME=${SPARK_PREFIX}
|
||||
export SPARK_CONF_DIR="$SPARK_HOME/conf"
|
||||
export SPARK_CONF_DIR="$SPARK_HOME/conf"
|
||||
|
|
|
@ -1,7 +1,5 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
# This Spark deploy script is a modified version of the Apache Hadoop deploy
|
||||
# script, available under the Apache 2 license:
|
||||
#
|
||||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
|
@ -10,13 +8,14 @@
|
|||
# (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
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Runs a Spark command as a daemon.
|
||||
#
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Run a Spark command on all slave hosts.
|
||||
|
||||
usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Start all spark daemons.
|
||||
# Starts the master on this node.
|
||||
# Starts a worker on each node specified in conf/slaves
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Starts the master on the machine this script is executed on.
|
||||
|
||||
bin=`dirname "$0"`
|
||||
|
|
|
@ -1,5 +1,25 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Usage: start-slave.sh <worker#> <master-spark-URL>
|
||||
# where <master-spark-URL> is like "spark://localhost:7077"
|
||||
|
||||
bin=`dirname "$0"`
|
||||
bin=`cd "$bin"; pwd`
|
||||
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
bin=`dirname "$0"`
|
||||
bin=`cd "$bin"; pwd`
|
||||
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Start all spark daemons.
|
||||
# Run this on the master nde
|
||||
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Starts the master on the machine this script is executed on.
|
||||
|
||||
bin=`dirname "$0"`
|
||||
|
|
|
@ -1,5 +1,22 @@
|
|||
#!/usr/bin/env bash
|
||||
|
||||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
# Starts the master on the machine this script is executed on.
|
||||
|
||||
bin=`dirname "$0"`
|
||||
|
|
15
conf/fairscheduler.xml.template
Normal file
15
conf/fairscheduler.xml.template
Normal file
|
@ -0,0 +1,15 @@
|
|||
<?xml version="1.0"?>
|
||||
<allocations>
|
||||
<pool name="production">
|
||||
<minShare>2</minShare>
|
||||
<weight>1</weight>
|
||||
<schedulingMode>FAIR</schedulingMode>
|
||||
</pool>
|
||||
<pool name="test">
|
||||
<minShare>3</minShare>
|
||||
<weight>2</weight>
|
||||
<schedulingMode>FIFO</schedulingMode>
|
||||
</pool>
|
||||
<pool name="data">
|
||||
</pool>
|
||||
</allocations>
|
84
conf/metrics.properties.template
Normal file
84
conf/metrics.properties.template
Normal file
|
@ -0,0 +1,84 @@
|
|||
# syntax: [instance].sink|source.[name].[options]=[value]
|
||||
|
||||
# This file configures Spark's internal metrics system. The metrics system is
|
||||
# divided into instances which correspond to internal components.
|
||||
# Each instance can be configured to report its metrics to one or more sinks.
|
||||
# Accepted values for [instance] are "master", "worker", "executor", "driver",
|
||||
# and "applications". A wild card "*" can be used as an instance name, in
|
||||
# which case all instances will inherit the supplied property.
|
||||
#
|
||||
# Within an instance, a "source" specifies a particular set of grouped metrics.
|
||||
# there are two kinds of sources:
|
||||
# 1. Spark internal sources, like MasterSource, WorkerSource, etc, which will
|
||||
# collect a Spark component's internal state. Each instance is paired with a
|
||||
# Spark source that is added automatically.
|
||||
# 2. Common sources, like JvmSource, which will collect low level state.
|
||||
# These can be added through configuration options and are then loaded
|
||||
# using reflection.
|
||||
#
|
||||
# A "sink" specifies where metrics are delivered to. Each instance can be
|
||||
# assigned one or more sinks.
|
||||
#
|
||||
# The sink|source field specifies whether the property relates to a sink or
|
||||
# source.
|
||||
#
|
||||
# The [name] field specifies the name of source or sink.
|
||||
#
|
||||
# The [options] field is the specific property of this source or sink. The
|
||||
# source or sink is responsible for parsing this property.
|
||||
#
|
||||
# Notes:
|
||||
# 1. To add a new sink, set the "class" option to a fully qualified class
|
||||
# name (see examples below).
|
||||
# 2. Some sinks involve a polling period. The minimum allowed polling period
|
||||
# is 1 second.
|
||||
# 3. Wild card properties can be overridden by more specific properties.
|
||||
# For example, master.sink.console.period takes precedence over
|
||||
# *.sink.console.period.
|
||||
# 4. A metrics specific configuration
|
||||
# "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be
|
||||
# added to Java properties using -Dspark.metrics.conf=xxx if you want to
|
||||
# customize metrics system. You can also put the file in ${SPARK_HOME}/conf
|
||||
# and it will be loaded automatically.
|
||||
|
||||
# Enable JmxSink for all instances by class name
|
||||
#*.sink.jmx.class=spark.metrics.sink.JmxSink
|
||||
|
||||
# Enable ConsoleSink for all instances by class name
|
||||
#*.sink.console.class=spark.metrics.sink.ConsoleSink
|
||||
|
||||
# Polling period for ConsoleSink
|
||||
#*.sink.console.period=10
|
||||
|
||||
#*.sink.console.unit=seconds
|
||||
|
||||
# Master instance overlap polling period
|
||||
#master.sink.console.period=15
|
||||
|
||||
#master.sink.console.unit=seconds
|
||||
|
||||
# Enable CsvSink for all instances
|
||||
#*.sink.csv.class=spark.metrics.sink.CsvSink
|
||||
|
||||
# Polling period for CsvSink
|
||||
#*.sink.csv.period=1
|
||||
|
||||
#*.sink.csv.unit=minutes
|
||||
|
||||
# Polling directory for CsvSink
|
||||
#*.sink.csv.directory=/tmp/
|
||||
|
||||
# Worker instance overlap polling period
|
||||
#worker.sink.csv.period=10
|
||||
|
||||
#worker.sink.csv.unit=minutes
|
||||
|
||||
# Enable jvm source for instance master, worker, driver and executor
|
||||
#master.source.jvm.class=spark.metrics.source.JvmSource
|
||||
|
||||
#worker.source.jvm.class=spark.metrics.source.JvmSource
|
||||
|
||||
#driver.source.jvm.class=spark.metrics.source.JvmSource
|
||||
|
||||
#executor.source.jvm.class=spark.metrics.source.JvmSource
|
||||
|
|
@ -3,8 +3,10 @@
|
|||
# This file contains environment variables required to run Spark. Copy it as
|
||||
# spark-env.sh and edit that to configure Spark for your site. At a minimum,
|
||||
# the following two variables should be set:
|
||||
# - MESOS_NATIVE_LIBRARY, to point to your Mesos native library (libmesos.so)
|
||||
# - SCALA_HOME, to point to your Scala installation
|
||||
# - SCALA_HOME, to point to your Scala installation, or SCALA_LIBRARY_PATH to
|
||||
# point to the directory for Scala library JARs (if you install Scala as a
|
||||
# Debian or RPM package, these are in a separate path, often /usr/share/java)
|
||||
# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos
|
||||
#
|
||||
# If using the standalone deploy mode, you can also set variables for it:
|
||||
# - SPARK_MASTER_IP, to bind the master to a different IP address
|
||||
|
@ -12,14 +14,6 @@
|
|||
# - SPARK_WORKER_CORES, to set the number of cores to use on this machine
|
||||
# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g)
|
||||
# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT
|
||||
# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes to be spawned on every slave machine
|
||||
#
|
||||
# Finally, Spark also relies on the following variables, but these can be set
|
||||
# on just the *master* (i.e. in your driver program), and will automatically
|
||||
# be propagated to workers:
|
||||
# - SPARK_MEM, to change the amount of memory used per node (this should
|
||||
# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g)
|
||||
# - SPARK_CLASSPATH, to add elements to Spark's classpath
|
||||
# - SPARK_JAVA_OPTS, to add JVM options
|
||||
# - SPARK_LIBRARY_PATH, to add extra search paths for native libraries.
|
||||
# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes
|
||||
# to be spawned on every slave machine
|
||||
|
||||
|
|
71
core/pom.xml
71
core/pom.xml
|
@ -1,4 +1,21 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<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>
|
||||
|
@ -31,6 +48,10 @@
|
|||
<groupId>com.ning</groupId>
|
||||
<artifactId>compress-lzf</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.xerial.snappy</groupId>
|
||||
<artifactId>snappy-java</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.ow2.asm</groupId>
|
||||
<artifactId>asm</artifactId>
|
||||
|
@ -40,8 +61,14 @@
|
|||
<artifactId>protobuf-java</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>de.javakaffee</groupId>
|
||||
<artifactId>kryo-serializers</artifactId>
|
||||
<groupId>com.twitter</groupId>
|
||||
<artifactId>chill_2.9.3</artifactId>
|
||||
<version>0.3.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.twitter</groupId>
|
||||
<artifactId>chill-java</artifactId>
|
||||
<version>0.3.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.typesafe.akka</groupId>
|
||||
|
@ -55,6 +82,18 @@
|
|||
<groupId>com.typesafe.akka</groupId>
|
||||
<artifactId>akka-slf4j</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.scala-lang</groupId>
|
||||
<artifactId>scalap</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.scala-lang</groupId>
|
||||
<artifactId>scala-library</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.liftweb</groupId>
|
||||
<artifactId>lift-json_2.9.2</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>it.unimi.dsi</groupId>
|
||||
<artifactId>fastutil</artifactId>
|
||||
|
@ -63,22 +102,6 @@
|
|||
<groupId>colt</groupId>
|
||||
<artifactId>colt</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>cc.spray</groupId>
|
||||
<artifactId>spray-can</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>cc.spray</groupId>
|
||||
<artifactId>spray-server</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>cc.spray</groupId>
|
||||
<artifactId>spray-json_2.9.2</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.tomdz.twirl</groupId>
|
||||
<artifactId>twirl-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.scala-incubator.io</groupId>
|
||||
<artifactId>scala-io-file_2.9.2</artifactId>
|
||||
|
@ -95,6 +118,14 @@
|
|||
<groupId>log4j</groupId>
|
||||
<artifactId>log4j</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.codahale.metrics</groupId>
|
||||
<artifactId>metrics-core</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.codahale.metrics</groupId>
|
||||
<artifactId>metrics-jvm</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.derby</groupId>
|
||||
|
@ -171,10 +202,6 @@
|
|||
</environmentVariables>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.tomdz.twirl</groupId>
|
||||
<artifactId>twirl-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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.hadoop.mapred
|
||||
|
||||
trait HadoopMapRedUtil {
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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.hadoop.mapreduce
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
|
|
|
@ -1,5 +1,23 @@
|
|||
/*
|
||||
* 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 spark.deploy
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
|
||||
|
||||
/**
|
||||
|
@ -20,4 +38,10 @@ object SparkHadoopUtil {
|
|||
|
||||
// Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
|
||||
def newConfiguration(): Configuration = new Configuration()
|
||||
|
||||
// add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
|
||||
def addCredentials(conf: JobConf) {}
|
||||
|
||||
def isYarnMode(): Boolean = { false }
|
||||
|
||||
}
|
||||
|
|
|
@ -1,4 +1,21 @@
|
|||
|
||||
/*
|
||||
* 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.hadoop.mapred
|
||||
|
||||
import org.apache.hadoop.mapreduce.TaskType
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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.hadoop.mapreduce
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
|
|
|
@ -1,6 +1,24 @@
|
|||
/*
|
||||
* 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 spark.deploy
|
||||
|
||||
import collection.mutable.HashMap
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
import org.apache.hadoop.security.UserGroupInformation
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
|
@ -27,18 +45,7 @@ object SparkHadoopUtil {
|
|||
}
|
||||
|
||||
def runAsUser(func: (Product) => Unit, args: Product, user: String) {
|
||||
|
||||
// println("running as user " + jobUserName)
|
||||
|
||||
UserGroupInformation.setConfiguration(yarnConf)
|
||||
val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(user)
|
||||
appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] {
|
||||
def run: AnyRef = {
|
||||
func(args)
|
||||
// no return value ...
|
||||
null
|
||||
}
|
||||
})
|
||||
func(args)
|
||||
}
|
||||
|
||||
// Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true.
|
||||
|
@ -60,4 +67,10 @@ object SparkHadoopUtil {
|
|||
// Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
|
||||
// Always create a new config, dont reuse yarnConf.
|
||||
def newConfiguration(): Configuration = new YarnConfiguration(new Configuration())
|
||||
|
||||
// add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
|
||||
def addCredentials(conf: JobConf) {
|
||||
val jobCreds = conf.getCredentials();
|
||||
jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.deploy.yarn
|
||||
|
||||
import java.net.Socket
|
||||
|
@ -27,26 +44,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
|
|||
private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
|
||||
|
||||
private var yarnAllocator: YarnAllocationHandler = null
|
||||
private var isFinished:Boolean = false
|
||||
|
||||
def run() {
|
||||
|
||||
// Initialization
|
||||
val jobUserName = Utils.getUserNameFromEnvironment()
|
||||
logInfo("running as user " + jobUserName)
|
||||
|
||||
// run as user ...
|
||||
UserGroupInformation.setConfiguration(yarnConf)
|
||||
val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(jobUserName)
|
||||
appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] {
|
||||
def run: AnyRef = {
|
||||
runImpl()
|
||||
return null
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
private def runImpl() {
|
||||
|
||||
appAttemptId = getApplicationAttemptId()
|
||||
resourceManager = registerWithResourceManager()
|
||||
val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
|
||||
|
@ -85,10 +86,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
|
|||
|
||||
// Wait for the user class to Finish
|
||||
userThread.join()
|
||||
|
||||
// Finish the ApplicationMaster
|
||||
finishApplicationMaster()
|
||||
// TODO: Exit based on success/failure
|
||||
|
||||
System.exit(0)
|
||||
}
|
||||
|
||||
|
@ -141,17 +139,30 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private def startUserClass(): Thread = {
|
||||
logInfo("Starting the user JAR in a separate Thread")
|
||||
val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader)
|
||||
.getMethod("main", classOf[Array[String]])
|
||||
val t = new Thread {
|
||||
override def run() {
|
||||
// Copy
|
||||
var mainArgs: Array[String] = new Array[String](args.userArgs.size())
|
||||
args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size())
|
||||
mainMethod.invoke(null, mainArgs)
|
||||
var successed = false
|
||||
try {
|
||||
// Copy
|
||||
var mainArgs: Array[String] = new Array[String](args.userArgs.size())
|
||||
args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size())
|
||||
mainMethod.invoke(null, mainArgs)
|
||||
// some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR
|
||||
// userThread will stop here unless it has uncaught exception thrown out
|
||||
// It need shutdown hook to set SUCCEEDED
|
||||
successed = true
|
||||
} finally {
|
||||
if (successed) {
|
||||
ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
|
||||
} else {
|
||||
ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
t.start()
|
||||
|
@ -196,7 +207,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
|
|||
logInfo("All workers have launched.")
|
||||
|
||||
// Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
|
||||
if (userThread.isAlive){
|
||||
if (userThread.isAlive) {
|
||||
// ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
|
||||
|
||||
val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
|
||||
|
@ -214,7 +225,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
|
|||
|
||||
val t = new Thread {
|
||||
override def run() {
|
||||
while (userThread.isAlive){
|
||||
while (userThread.isAlive) {
|
||||
val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
|
||||
if (missingWorkerCount > 0) {
|
||||
logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
|
||||
|
@ -252,14 +263,23 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
|
|||
}
|
||||
}
|
||||
*/
|
||||
|
||||
def finishApplicationMaster() {
|
||||
|
||||
def finishApplicationMaster(status: FinalApplicationStatus) {
|
||||
|
||||
synchronized {
|
||||
if (isFinished) {
|
||||
return
|
||||
}
|
||||
isFinished = true
|
||||
}
|
||||
|
||||
logInfo("finishApplicationMaster with " + status)
|
||||
val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest])
|
||||
.asInstanceOf[FinishApplicationMasterRequest]
|
||||
finishReq.setAppAttemptId(appAttemptId)
|
||||
// TODO: Check if the application has failed or succeeded
|
||||
finishReq.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED)
|
||||
finishReq.setFinishApplicationStatus(status)
|
||||
resourceManager.finishApplicationMaster(finishReq)
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -273,7 +293,7 @@ object ApplicationMaster {
|
|||
private val ALLOCATOR_LOOP_WAIT_COUNT = 30
|
||||
def incrementAllocatorLoop(by: Int) {
|
||||
val count = yarnAllocatorLoop.getAndAdd(by)
|
||||
if (count >= ALLOCATOR_LOOP_WAIT_COUNT){
|
||||
if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
|
||||
yarnAllocatorLoop.synchronized {
|
||||
// to wake threads off wait ...
|
||||
yarnAllocatorLoop.notifyAll()
|
||||
|
@ -308,14 +328,16 @@ object ApplicationMaster {
|
|||
logInfo("Invoking sc stop from shutdown hook")
|
||||
sc.stop()
|
||||
// best case ...
|
||||
for (master <- applicationMasters) master.finishApplicationMaster
|
||||
for (master <- applicationMasters) {
|
||||
master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
|
||||
}
|
||||
}
|
||||
} )
|
||||
}
|
||||
|
||||
// Wait for initialization to complete and atleast 'some' nodes can get allocated
|
||||
yarnAllocatorLoop.synchronized {
|
||||
while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT){
|
||||
while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
|
||||
yarnAllocatorLoop.wait(1000L)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.deploy.yarn
|
||||
|
||||
import spark.util.IntParam
|
||||
|
|
|
@ -1,9 +1,30 @@
|
|||
/*
|
||||
* 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 spark.deploy.yarn
|
||||
|
||||
import java.net.{InetSocketAddress, URI}
|
||||
import java.nio.ByteBuffer
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
|
||||
import org.apache.hadoop.mapred.Master
|
||||
import org.apache.hadoop.net.NetUtils
|
||||
import org.apache.hadoop.io.DataOutputBuffer
|
||||
import org.apache.hadoop.security.UserGroupInformation
|
||||
import org.apache.hadoop.yarn.api._
|
||||
import org.apache.hadoop.yarn.api.records._
|
||||
import org.apache.hadoop.yarn.api.protocolrecords._
|
||||
|
@ -23,6 +44,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
|
||||
var rpc: YarnRPC = YarnRPC.create(conf)
|
||||
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
|
||||
val credentials = UserGroupInformation.getCurrentUser().getCredentials();
|
||||
|
||||
def run() {
|
||||
init(yarnConf)
|
||||
|
@ -40,8 +62,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
|
||||
appContext.setQueue(args.amQueue)
|
||||
appContext.setAMContainerSpec(amContainer)
|
||||
appContext.setUser(args.amUser)
|
||||
|
||||
appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
|
||||
|
||||
submitApp(appContext)
|
||||
|
||||
monitorApplication(appId)
|
||||
|
@ -62,14 +84,21 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
|
||||
def verifyClusterResources(app: GetNewApplicationResponse) = {
|
||||
val maxMem = app.getMaximumResourceCapability().getMemory()
|
||||
logInfo("Max mem capabililty of resources in this cluster " + maxMem)
|
||||
logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
|
||||
|
||||
// If the cluster does not have enough memory resources, exit.
|
||||
val requestedMem = (args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + args.numWorkers * args.workerMemory
|
||||
if (requestedMem > maxMem) {
|
||||
logError("Cluster cannot satisfy memory resource request of " + requestedMem)
|
||||
// if we have requested more then the clusters max for a single resource then exit.
|
||||
if (args.workerMemory > maxMem) {
|
||||
logError("the worker size is to large to run on this cluster " + args.workerMemory);
|
||||
System.exit(1)
|
||||
}
|
||||
val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
|
||||
if (amMem > maxMem) {
|
||||
logError("AM size is to large to run on this cluster " + amMem)
|
||||
System.exit(1)
|
||||
}
|
||||
|
||||
// We could add checks to make sure the entire cluster has enough resources but that involves getting
|
||||
// all the node reports and computing ourselves
|
||||
}
|
||||
|
||||
def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
|
||||
|
@ -86,6 +115,15 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
// Upload Spark and the application JAR to the remote file system
|
||||
// Add them as local resources to the AM
|
||||
val fs = FileSystem.get(conf)
|
||||
|
||||
val delegTokenRenewer = Master.getMasterPrincipal(conf);
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
|
||||
logError("Can't get Master Kerberos principal for use as renewer")
|
||||
System.exit(1)
|
||||
}
|
||||
}
|
||||
|
||||
Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF"))
|
||||
.foreach { case(destName, _localPath) =>
|
||||
val localPath: String = if (_localPath != null) _localPath.trim() else ""
|
||||
|
@ -97,6 +135,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
fs.copyFromLocalFile(false, true, src, dst)
|
||||
val destStatus = fs.getFileStatus(dst)
|
||||
|
||||
// get tokens for anything we upload to hdfs
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
fs.addDelegationTokens(delegTokenRenewer, credentials);
|
||||
}
|
||||
|
||||
val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
|
||||
amJarRsrc.setType(LocalResourceType.FILE)
|
||||
amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
|
||||
|
@ -106,6 +149,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
locaResources(destName) = amJarRsrc
|
||||
}
|
||||
}
|
||||
UserGroupInformation.getCurrentUser().addCredentials(credentials);
|
||||
return locaResources
|
||||
}
|
||||
|
||||
|
@ -114,7 +158,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
val log4jConfLocalRes = localResources.getOrElse("log4j.properties", null)
|
||||
|
||||
val env = new HashMap[String, String]()
|
||||
Apps.addToEnvironment(env, Environment.USER.name, args.amUser)
|
||||
|
||||
// If log4j present, ensure ours overrides all others
|
||||
if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
|
||||
|
@ -142,6 +185,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
env("SPARK_YARN_LOG4J_SIZE") = log4jConfLocalRes.getSize().toString()
|
||||
}
|
||||
|
||||
|
||||
// Add each SPARK-* key to the environment
|
||||
System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
|
||||
return env
|
||||
|
@ -195,7 +239,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
}
|
||||
|
||||
// Command for the ApplicationMaster
|
||||
val commands = List[String]("java " +
|
||||
var javaCommand = "java";
|
||||
val javaHome = System.getenv("JAVA_HOME")
|
||||
if (javaHome != null && !javaHome.isEmpty()) {
|
||||
javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
|
||||
}
|
||||
|
||||
val commands = List[String](javaCommand +
|
||||
" -server " +
|
||||
JAVA_OPTS +
|
||||
" spark.deploy.yarn.ApplicationMaster" +
|
||||
|
@ -214,7 +264,12 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
|
|||
// Memory for the ApplicationMaster
|
||||
capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
|
||||
amContainer.setResource(capability)
|
||||
|
||||
|
||||
// Setup security tokens
|
||||
val dob = new DataOutputBuffer()
|
||||
credentials.writeTokenStorageToStream(dob)
|
||||
amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData()))
|
||||
|
||||
return amContainer
|
||||
}
|
||||
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.deploy.yarn
|
||||
|
||||
import spark.util.MemoryParam
|
||||
|
@ -13,7 +30,6 @@ class ClientArguments(val args: Array[String]) {
|
|||
var workerMemory = 1024
|
||||
var workerCores = 1
|
||||
var numWorkers = 2
|
||||
var amUser = System.getProperty("user.name")
|
||||
var amQueue = System.getProperty("QUEUE", "default")
|
||||
var amMemory: Int = 512
|
||||
// TODO
|
||||
|
@ -58,10 +74,6 @@ class ClientArguments(val args: Array[String]) {
|
|||
workerCores = value
|
||||
args = tail
|
||||
|
||||
case ("--user") :: value :: tail =>
|
||||
amUser = value
|
||||
args = tail
|
||||
|
||||
case ("--queue") :: value :: tail =>
|
||||
amQueue = value
|
||||
args = tail
|
||||
|
@ -96,8 +108,7 @@ class ClientArguments(val args: Array[String]) {
|
|||
" --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" +
|
||||
" --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
|
||||
" --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
|
||||
" --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" +
|
||||
" --user USERNAME Run the ApplicationMaster (and slaves) as a different user\n"
|
||||
" --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')"
|
||||
)
|
||||
System.exit(exitCode)
|
||||
}
|
||||
|
|
|
@ -1,9 +1,29 @@
|
|||
/*
|
||||
* 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 spark.deploy.yarn
|
||||
|
||||
import java.net.URI
|
||||
import java.nio.ByteBuffer
|
||||
import java.security.PrivilegedExceptionAction
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
|
||||
import org.apache.hadoop.io.DataOutputBuffer
|
||||
import org.apache.hadoop.net.NetUtils
|
||||
import org.apache.hadoop.security.UserGroupInformation
|
||||
import org.apache.hadoop.yarn.api._
|
||||
|
@ -11,7 +31,7 @@ import org.apache.hadoop.yarn.api.records._
|
|||
import org.apache.hadoop.yarn.api.protocolrecords._
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC
|
||||
import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
|
||||
import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
|
||||
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
@ -76,7 +96,19 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
|
|||
*/
|
||||
|
||||
ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
|
||||
val commands = List[String]("java " +
|
||||
|
||||
val credentials = UserGroupInformation.getCurrentUser().getCredentials()
|
||||
val dob = new DataOutputBuffer()
|
||||
credentials.writeTokenStorageToStream(dob)
|
||||
ctx.setContainerTokens(ByteBuffer.wrap(dob.getData()))
|
||||
|
||||
var javaCommand = "java";
|
||||
val javaHome = System.getenv("JAVA_HOME")
|
||||
if (javaHome != null && !javaHome.isEmpty()) {
|
||||
javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
|
||||
}
|
||||
|
||||
val commands = List[String](javaCommand +
|
||||
" -server " +
|
||||
// Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
|
||||
// Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state.
|
||||
|
@ -143,8 +175,6 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
|
|||
|
||||
def prepareEnvironment: HashMap[String, String] = {
|
||||
val env = new HashMap[String, String]()
|
||||
// should we add this ?
|
||||
Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment())
|
||||
|
||||
// If log4j present, ensure ours overrides all others
|
||||
if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
|
||||
|
@ -165,7 +195,23 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
|
|||
val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort()
|
||||
val cmAddress = NetUtils.createSocketAddr(cmHostPortStr)
|
||||
logInfo("Connecting to ContainerManager at " + cmHostPortStr)
|
||||
return rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager]
|
||||
|
||||
// use doAs and remoteUser here so we can add the container token and not
|
||||
// pollute the current users credentials with all of the individual container tokens
|
||||
val user = UserGroupInformation.createRemoteUser(container.getId().toString());
|
||||
val containerToken = container.getContainerToken();
|
||||
if (containerToken != null) {
|
||||
user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress))
|
||||
}
|
||||
|
||||
val proxy = user
|
||||
.doAs(new PrivilegedExceptionAction[ContainerManager] {
|
||||
def run: ContainerManager = {
|
||||
return rpc.getProxy(classOf[ContainerManager],
|
||||
cmAddress, conf).asInstanceOf[ContainerManager]
|
||||
}
|
||||
});
|
||||
return proxy;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.deploy.yarn
|
||||
|
||||
import spark.{Logging, Utils}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.scheduler.cluster
|
||||
|
||||
import spark._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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.hadoop.mapred
|
||||
|
||||
trait HadoopMapRedUtil {
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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.hadoop.mapreduce
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
|
|
|
@ -1,5 +1,23 @@
|
|||
/*
|
||||
* 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 spark.deploy
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
|
||||
|
||||
/**
|
||||
|
@ -20,4 +38,10 @@ object SparkHadoopUtil {
|
|||
|
||||
// Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
|
||||
def newConfiguration(): Configuration = new Configuration()
|
||||
|
||||
// add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
|
||||
def addCredentials(conf: JobConf) {}
|
||||
|
||||
def isYarnMode(): Boolean = { false }
|
||||
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
import io.netty.bootstrap.Bootstrap;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
import io.netty.buffer.BufType;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
import io.netty.buffer.ByteBuf;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
import io.netty.channel.ChannelInitializer;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
import java.io.File;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.network.netty;
|
||||
|
||||
|
||||
|
|
File diff suppressed because one or more lines are too long
File diff suppressed because one or more lines are too long
9
core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css
vendored
Normal file
9
core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css
vendored
Normal file
File diff suppressed because one or more lines are too long
9
core/src/main/resources/spark/ui/static/bootstrap.min.css
vendored
Normal file
9
core/src/main/resources/spark/ui/static/bootstrap.min.css
vendored
Normal file
File diff suppressed because one or more lines are too long
Binary file not shown.
After Width: | Height: | Size: 3.5 KiB |
Before Width: | Height: | Size: 14 KiB After Width: | Height: | Size: 14 KiB |
53
core/src/main/resources/spark/ui/static/webui.css
Normal file
53
core/src/main/resources/spark/ui/static/webui.css
Normal file
|
@ -0,0 +1,53 @@
|
|||
.navbar .brand {
|
||||
height: 50px;
|
||||
width: 110px;
|
||||
margin-left: 1px;
|
||||
padding: 0;
|
||||
}
|
||||
|
||||
.version {
|
||||
line-height: 30px;
|
||||
vertical-align: bottom;
|
||||
font-size: 12px;
|
||||
padding: 0;
|
||||
margin: 0;
|
||||
font-weight: bold;
|
||||
color: #777;
|
||||
}
|
||||
|
||||
.navbar-inner {
|
||||
padding-top: 2px;
|
||||
height: 50px;
|
||||
}
|
||||
|
||||
.navbar-inner .nav {
|
||||
margin-top: 5px;
|
||||
font-size: 15px;
|
||||
}
|
||||
|
||||
|
||||
#infolist {
|
||||
margin-left: 400px;
|
||||
margin-top: 14px;
|
||||
}
|
||||
|
||||
#infolist li {
|
||||
display: inline;
|
||||
list-style-type: none;
|
||||
list-style-position: outside;
|
||||
padding-right: 20px;
|
||||
padding-top: 10px;
|
||||
padding-bottom: 10px;
|
||||
}
|
||||
|
||||
.progress-cell {
|
||||
width: 134px;
|
||||
border-right: 0;
|
||||
padding: 0;
|
||||
padding-top: 7px;
|
||||
padding-left: 4px;
|
||||
}
|
||||
|
||||
.table td {
|
||||
vertical-align: middle !important;
|
||||
}
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.util.{HashMap => JHashMap}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
|
|
@ -1,63 +0,0 @@
|
|||
package spark
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
private[spark] sealed trait CachePutResponse
|
||||
private[spark] case class CachePutSuccess(size: Long) extends CachePutResponse
|
||||
private[spark] case class CachePutFailure() extends CachePutResponse
|
||||
|
||||
/**
|
||||
* An interface for caches in Spark, to allow for multiple implementations. Caches are used to store
|
||||
* both partitions of cached RDDs and broadcast variables on Spark executors. Caches are also aware
|
||||
* of which entries are part of the same dataset (for example, partitions in the same RDD). The key
|
||||
* for each value in a cache is a (datasetID, partition) pair.
|
||||
*
|
||||
* A single Cache instance gets created on each machine and is shared by all caches (i.e. both the
|
||||
* RDD split cache and the broadcast variable cache), to enable global replacement policies.
|
||||
* However, because these several independent modules all perform caching, it is important to give
|
||||
* them separate key namespaces, so that an RDD and a broadcast variable (for example) do not use
|
||||
* the same key. For this purpose, Cache has the notion of KeySpaces. Each client module must first
|
||||
* ask for a KeySpace, and then call get() and put() on that space using its own keys.
|
||||
*
|
||||
* This abstract class handles the creation of key spaces, so that subclasses need only deal with
|
||||
* keys that are unique across modules.
|
||||
*/
|
||||
private[spark] abstract class Cache {
|
||||
private val nextKeySpaceId = new AtomicInteger(0)
|
||||
private def newKeySpaceId() = nextKeySpaceId.getAndIncrement()
|
||||
|
||||
def newKeySpace() = new KeySpace(this, newKeySpaceId())
|
||||
|
||||
/**
|
||||
* Get the value for a given (datasetId, partition), or null if it is not
|
||||
* found.
|
||||
*/
|
||||
def get(datasetId: Any, partition: Int): Any
|
||||
|
||||
/**
|
||||
* Attempt to put a value in the cache; returns CachePutFailure if this was
|
||||
* not successful (e.g. because the cache replacement policy forbids it), and
|
||||
* CachePutSuccess if successful. If size estimation is available, the cache
|
||||
* implementation should set the size field in CachePutSuccess.
|
||||
*/
|
||||
def put(datasetId: Any, partition: Int, value: Any): CachePutResponse
|
||||
|
||||
/**
|
||||
* Report the capacity of the cache partition. By default this just reports
|
||||
* zero. Specific implementations can choose to provide the capacity number.
|
||||
*/
|
||||
def getCapacity: Long = 0L
|
||||
}
|
||||
|
||||
/**
|
||||
* A key namespace in a Cache.
|
||||
*/
|
||||
private[spark] class KeySpace(cache: Cache, val keySpaceId: Int) {
|
||||
def get(datasetId: Any, partition: Int): Any =
|
||||
cache.get((keySpaceId, datasetId), partition)
|
||||
|
||||
def put(datasetId: Any, partition: Int, value: Any): CachePutResponse =
|
||||
cache.put((keySpaceId, datasetId), partition, value)
|
||||
|
||||
def getCapacity: Long = cache.getCapacity
|
||||
}
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import scala.collection.mutable.{ArrayBuffer, HashSet}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.lang.reflect.Field
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import spark.partial.BoundedDouble
|
||||
|
@ -37,7 +54,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
|
|||
* Compute the sample standard deviation of this RDD's elements (which corrects for bias in
|
||||
* estimating the standard deviation by dividing by N-1 instead of N).
|
||||
*/
|
||||
def sampleStdev(): Double = stats().stdev
|
||||
def sampleStdev(): Double = stats().sampleStdev
|
||||
|
||||
/**
|
||||
* Compute the sample variance of this RDD's elements (which corrects for bias in
|
||||
* estimating the variance by dividing by N-1 instead of N).
|
||||
*/
|
||||
def sampleVariance(): Double = stats().sampleVariance
|
||||
|
||||
/** (Experimental) Approximate operation to return the mean within a timeout. */
|
||||
def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import spark.storage.BlockManagerId
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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.hadoop.mapred
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io.{File}
|
||||
|
@ -42,4 +59,4 @@ private[spark] class HttpFileServer extends Logging {
|
|||
return dir + "/" + file.getName
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io.File
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
|
|
|
@ -1,25 +1,34 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
import java.nio.ByteBuffer
|
||||
import java.nio.channels.Channels
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.mutable
|
||||
|
||||
import com.esotericsoftware.kryo._
|
||||
import com.esotericsoftware.kryo.{Serializer => KSerializer}
|
||||
import com.esotericsoftware.kryo.{Kryo, KryoException}
|
||||
import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
|
||||
import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer}
|
||||
import de.javakaffee.kryoserializers.KryoReflectionFactorySupport
|
||||
|
||||
import com.twitter.chill.ScalaKryoInstantiator
|
||||
import serializer.{SerializerInstance, DeserializationStream, SerializationStream}
|
||||
import spark.broadcast._
|
||||
import spark.storage._
|
||||
|
||||
private[spark]
|
||||
class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream {
|
||||
|
||||
val output = new KryoOutput(outStream)
|
||||
|
||||
def writeObject[T](t: T): SerializationStream = {
|
||||
|
@ -33,7 +42,6 @@ class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends Seria
|
|||
|
||||
private[spark]
|
||||
class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream {
|
||||
|
||||
val input = new KryoInput(inStream)
|
||||
|
||||
def readObject[T](): T = {
|
||||
|
@ -41,7 +49,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser
|
|||
kryo.readClassAndObject(input).asInstanceOf[T]
|
||||
} catch {
|
||||
// DeserializationStream uses the EOF exception to indicate stopping condition.
|
||||
case e: com.esotericsoftware.kryo.KryoException => throw new java.io.EOFException
|
||||
case _: KryoException => throw new EOFException
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -52,10 +60,9 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser
|
|||
}
|
||||
|
||||
private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance {
|
||||
|
||||
val kryo = ks.kryo.get()
|
||||
val output = ks.output.get()
|
||||
val input = ks.input.get()
|
||||
val kryo = ks.newKryo()
|
||||
val output = ks.newKryoOutput()
|
||||
val input = ks.newKryoInput()
|
||||
|
||||
def serialize[T](t: T): ByteBuffer = {
|
||||
output.clear()
|
||||
|
@ -91,130 +98,59 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ
|
|||
* serialization.
|
||||
*/
|
||||
trait KryoRegistrator {
|
||||
def registerClasses(kryo: Kryo): Unit
|
||||
def registerClasses(kryo: Kryo)
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
|
||||
|
||||
val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
|
||||
def newKryoOutput() = new KryoOutput(bufferSize)
|
||||
|
||||
val kryo = new ThreadLocal[Kryo] {
|
||||
override def initialValue = createKryo()
|
||||
}
|
||||
def newKryoInput() = new KryoInput(bufferSize)
|
||||
|
||||
val output = new ThreadLocal[KryoOutput] {
|
||||
override def initialValue = new KryoOutput(bufferSize)
|
||||
}
|
||||
|
||||
val input = new ThreadLocal[KryoInput] {
|
||||
override def initialValue = new KryoInput(bufferSize)
|
||||
}
|
||||
|
||||
def createKryo(): Kryo = {
|
||||
val kryo = new KryoReflectionFactorySupport()
|
||||
def newKryo(): Kryo = {
|
||||
val instantiator = new ScalaKryoInstantiator
|
||||
val kryo = instantiator.newKryo()
|
||||
val classLoader = Thread.currentThread.getContextClassLoader
|
||||
|
||||
// Register some commonly used classes
|
||||
val toRegister: Seq[AnyRef] = Seq(
|
||||
// Arrays
|
||||
Array(1), Array(1.0), Array(1.0f), Array(1L), Array(""), Array(("", "")),
|
||||
Array(new java.lang.Object), Array(1.toByte), Array(true), Array('c'),
|
||||
// Specialized Tuple2s
|
||||
("", ""), ("", 1), (1, 1), (1.0, 1.0), (1L, 1L),
|
||||
(1, 1.0), (1.0, 1), (1L, 1.0), (1.0, 1L), (1, 1L), (1L, 1),
|
||||
// Scala collections
|
||||
List(1), mutable.ArrayBuffer(1),
|
||||
// Options and Either
|
||||
Some(1), Left(1), Right(1),
|
||||
// Higher-dimensional tuples
|
||||
(1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1, 1),
|
||||
None,
|
||||
ByteBuffer.allocate(1),
|
||||
StorageLevel.MEMORY_ONLY,
|
||||
PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY),
|
||||
GotBlock("1", ByteBuffer.allocate(1)),
|
||||
GetBlock("1")
|
||||
)
|
||||
for (obj <- toRegister) {
|
||||
kryo.register(obj.getClass)
|
||||
}
|
||||
|
||||
for (obj <- toRegister) kryo.register(obj.getClass)
|
||||
|
||||
// Allow sending SerializableWritable
|
||||
kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer())
|
||||
kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer())
|
||||
|
||||
// Register some commonly used Scala singleton objects. Because these
|
||||
// are singletons, we must return the exact same local object when we
|
||||
// deserialize rather than returning a clone as FieldSerializer would.
|
||||
class SingletonSerializer[T](obj: T) extends KSerializer[T] {
|
||||
override def write(kryo: Kryo, output: KryoOutput, obj: T) {}
|
||||
override def read(kryo: Kryo, input: KryoInput, cls: java.lang.Class[T]): T = obj
|
||||
}
|
||||
kryo.register(None.getClass, new SingletonSerializer[AnyRef](None))
|
||||
kryo.register(Nil.getClass, new SingletonSerializer[AnyRef](Nil))
|
||||
|
||||
// Register maps with a special serializer since they have complex internal structure
|
||||
class ScalaMapSerializer(buildMap: Array[(Any, Any)] => scala.collection.Map[Any, Any])
|
||||
extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] {
|
||||
|
||||
//hack, look at https://groups.google.com/forum/#!msg/kryo-users/Eu5V4bxCfws/k-8UQ22y59AJ
|
||||
private final val FAKE_REFERENCE = new Object()
|
||||
override def write(
|
||||
kryo: Kryo,
|
||||
output: KryoOutput,
|
||||
obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) {
|
||||
val map = obj.asInstanceOf[scala.collection.Map[Any, Any]]
|
||||
output.writeInt(map.size)
|
||||
for ((k, v) <- map) {
|
||||
kryo.writeClassAndObject(output, k)
|
||||
kryo.writeClassAndObject(output, v)
|
||||
}
|
||||
}
|
||||
override def read (
|
||||
kryo: Kryo,
|
||||
input: KryoInput,
|
||||
cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]])
|
||||
: Array[(Any, Any)] => scala.collection.Map[Any, Any] = {
|
||||
kryo.reference(FAKE_REFERENCE)
|
||||
val size = input.readInt()
|
||||
val elems = new Array[(Any, Any)](size)
|
||||
for (i <- 0 until size) {
|
||||
val k = kryo.readClassAndObject(input)
|
||||
val v = kryo.readClassAndObject(input)
|
||||
elems(i)=(k,v)
|
||||
}
|
||||
buildMap(elems).asInstanceOf[Array[(Any, Any)] => scala.collection.Map[Any, Any]]
|
||||
}
|
||||
}
|
||||
kryo.register(mutable.HashMap().getClass, new ScalaMapSerializer(mutable.HashMap() ++ _))
|
||||
// TODO: add support for immutable maps too; this is more annoying because there are many
|
||||
// subclasses of immutable.Map for small maps (with <= 4 entries)
|
||||
val map1 = Map[Any, Any](1 -> 1)
|
||||
val map2 = Map[Any, Any](1 -> 1, 2 -> 2)
|
||||
val map3 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3)
|
||||
val map4 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4)
|
||||
val map5 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4, 5 -> 5)
|
||||
kryo.register(map1.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap))
|
||||
kryo.register(map2.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap))
|
||||
kryo.register(map3.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap))
|
||||
kryo.register(map4.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap))
|
||||
kryo.register(map5.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap))
|
||||
|
||||
// Allow the user to register their own classes by setting spark.kryo.registrator
|
||||
val regCls = System.getProperty("spark.kryo.registrator")
|
||||
if (regCls != null) {
|
||||
logInfo("Running user registrator: " + regCls)
|
||||
val classLoader = Thread.currentThread.getContextClassLoader
|
||||
val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
|
||||
reg.registerClasses(kryo)
|
||||
try {
|
||||
Option(System.getProperty("spark.kryo.registrator")).foreach { regCls =>
|
||||
logDebug("Running user registrator: " + regCls)
|
||||
val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
|
||||
reg.registerClasses(kryo)
|
||||
}
|
||||
} catch {
|
||||
case _: Exception => println("Failed to register spark.kryo.registrator")
|
||||
}
|
||||
|
||||
kryo.setClassLoader(classLoader)
|
||||
|
||||
// Allow disabling Kryo reference tracking if user knows their object graphs don't have loops
|
||||
kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean)
|
||||
|
||||
kryo
|
||||
}
|
||||
|
||||
def newInstance(): SerializerInstance = {
|
||||
this.kryo.get().setClassLoader(Thread.currentThread().getContextClassLoader)
|
||||
new KryoSerializerInstance(this)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import org.slf4j.Logger
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.nio.ByteBuffer
|
||||
|
@ -21,6 +38,7 @@ import org.apache.hadoop.mapred.OutputFormat
|
|||
|
||||
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
|
||||
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil}
|
||||
import org.apache.hadoop.security.UserGroupInformation
|
||||
|
||||
import spark.partial.BoundedDouble
|
||||
import spark.partial.PartialResult
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
/**
|
||||
|
@ -48,17 +65,9 @@ object Partitioner {
|
|||
class HashPartitioner(partitions: Int) extends Partitioner {
|
||||
def numPartitions = partitions
|
||||
|
||||
def getPartition(key: Any): Int = {
|
||||
if (key == null) {
|
||||
return 0
|
||||
} else {
|
||||
val mod = key.hashCode % partitions
|
||||
if (mod < 0) {
|
||||
mod + partitions
|
||||
} else {
|
||||
mod // Guard against negative hash codes
|
||||
}
|
||||
}
|
||||
def getPartition(key: Any): Int = key match {
|
||||
case null => 0
|
||||
case _ => Utils.nonNegativeMod(key.hashCode, numPartitions)
|
||||
}
|
||||
|
||||
override def equals(other: Any): Boolean = other match {
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.util.Random
|
||||
|
@ -105,6 +122,9 @@ abstract class RDD[T: ClassManifest](
|
|||
// Methods and fields available on all RDDs
|
||||
// =======================================================================
|
||||
|
||||
/** The SparkContext that created this RDD. */
|
||||
def sparkContext: SparkContext = sc
|
||||
|
||||
/** A unique ID for this RDD (within its SparkContext). */
|
||||
val id: Int = sc.newRddId()
|
||||
|
||||
|
@ -119,7 +139,7 @@ abstract class RDD[T: ClassManifest](
|
|||
|
||||
/** User-defined generator of this RDD*/
|
||||
var generator = Utils.getCallSiteInfo.firstUserClass
|
||||
|
||||
|
||||
/** Reset generator*/
|
||||
def setGenerator(_generator: String) = {
|
||||
generator = _generator
|
||||
|
@ -281,31 +301,35 @@ abstract class RDD[T: ClassManifest](
|
|||
def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = {
|
||||
var fraction = 0.0
|
||||
var total = 0
|
||||
val multiplier = 3.0
|
||||
val initialCount = count()
|
||||
var multiplier = 3.0
|
||||
var initialCount = this.count()
|
||||
var maxSelected = 0
|
||||
|
||||
if (num < 0) {
|
||||
throw new IllegalArgumentException("Negative number of elements requested")
|
||||
}
|
||||
|
||||
if (initialCount > Integer.MAX_VALUE - 1) {
|
||||
maxSelected = Integer.MAX_VALUE - 1
|
||||
} else {
|
||||
maxSelected = initialCount.toInt
|
||||
}
|
||||
|
||||
if (num > initialCount) {
|
||||
if (num > initialCount && !withReplacement) {
|
||||
total = maxSelected
|
||||
fraction = math.min(multiplier * (maxSelected + 1) / initialCount, 1.0)
|
||||
} else if (num < 0) {
|
||||
throw(new IllegalArgumentException("Negative number of elements requested"))
|
||||
fraction = multiplier * (maxSelected + 1) / initialCount
|
||||
} else {
|
||||
fraction = math.min(multiplier * (num + 1) / initialCount, 1.0)
|
||||
fraction = multiplier * (num + 1) / initialCount
|
||||
total = num
|
||||
}
|
||||
|
||||
val rand = new Random(seed)
|
||||
var samples = this.sample(withReplacement, fraction, rand.nextInt).collect()
|
||||
var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect()
|
||||
|
||||
// If the first sample didn't turn out large enough, keep trying to take samples;
|
||||
// this shouldn't happen often because we use a big multiplier for thei initial size
|
||||
while (samples.length < total) {
|
||||
samples = this.sample(withReplacement, fraction, rand.nextInt).collect()
|
||||
samples = this.sample(withReplacement, fraction, rand.nextInt()).collect()
|
||||
}
|
||||
|
||||
Utils.randomizeInPlace(samples, rand).take(total)
|
||||
|
@ -363,7 +387,7 @@ abstract class RDD[T: ClassManifest](
|
|||
/**
|
||||
* Return an RDD created by piping elements to a forked external process.
|
||||
*/
|
||||
def pipe(command: String, env: Map[String, String]): RDD[String] =
|
||||
def pipe(command: String, env: Map[String, String]): RDD[String] =
|
||||
new PipedRDD(this, command, env)
|
||||
|
||||
|
||||
|
@ -374,24 +398,24 @@ abstract class RDD[T: ClassManifest](
|
|||
* @param command command to run in forked process.
|
||||
* @param env environment variables to set.
|
||||
* @param printPipeContext Before piping elements, this function is called as an oppotunity
|
||||
* to pipe context data. Print line function (like out.println) will be
|
||||
* to pipe context data. Print line function (like out.println) will be
|
||||
* passed as printPipeContext's parameter.
|
||||
* @param printRDDElement Use this function to customize how to pipe elements. This function
|
||||
* will be called with each RDD element as the 1st parameter, and the
|
||||
* @param printRDDElement Use this function to customize how to pipe elements. This function
|
||||
* will be called with each RDD element as the 1st parameter, and the
|
||||
* print line function (like out.println()) as the 2nd parameter.
|
||||
* An example of pipe the RDD data of groupBy() in a streaming way,
|
||||
* instead of constructing a huge String to concat all the elements:
|
||||
* def printRDDElement(record:(String, Seq[String]), f:String=>Unit) =
|
||||
* def printRDDElement(record:(String, Seq[String]), f:String=>Unit) =
|
||||
* for (e <- record._2){f(e)}
|
||||
* @return the result RDD
|
||||
*/
|
||||
def pipe(
|
||||
command: Seq[String],
|
||||
env: Map[String, String] = Map(),
|
||||
command: Seq[String],
|
||||
env: Map[String, String] = Map(),
|
||||
printPipeContext: (String => Unit) => Unit = null,
|
||||
printRDDElement: (T, String => Unit) => Unit = null): RDD[String] =
|
||||
new PipedRDD(this, command, env,
|
||||
if (printPipeContext ne null) sc.clean(printPipeContext) else null,
|
||||
printRDDElement: (T, String => Unit) => Unit = null): RDD[String] =
|
||||
new PipedRDD(this, command, env,
|
||||
if (printPipeContext ne null) sc.clean(printPipeContext) else null,
|
||||
if (printRDDElement ne null) sc.clean(printRDDElement) else null)
|
||||
|
||||
/**
|
||||
|
@ -771,9 +795,19 @@ abstract class RDD[T: ClassManifest](
|
|||
}.reduce { (queue1, queue2) =>
|
||||
queue1 ++= queue2
|
||||
queue1
|
||||
}.toArray
|
||||
}.toArray.sorted(ord.reverse)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the first K elements from this RDD as defined by
|
||||
* the specified implicit Ordering[T] and maintains the
|
||||
* ordering.
|
||||
* @param num the number of top elements to return
|
||||
* @param ord the implicit ordering for T
|
||||
* @return an array of top elements
|
||||
*/
|
||||
def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse)
|
||||
|
||||
/**
|
||||
* Save this RDD as a text file, using string representations of elements.
|
||||
*/
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io.EOFException
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import spark.executor.TaskMetrics
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.lang.reflect.Field
|
||||
|
|
|
@ -1,18 +0,0 @@
|
|||
package spark
|
||||
|
||||
import com.google.common.collect.MapMaker
|
||||
|
||||
/**
|
||||
* An implementation of Cache that uses soft references.
|
||||
*/
|
||||
private[spark] class SoftReferenceCache extends Cache {
|
||||
val map = new MapMaker().softValues().makeMap[Any, Any]()
|
||||
|
||||
override def get(datasetId: Any, partition: Int): Any =
|
||||
map.get((datasetId, partition))
|
||||
|
||||
override def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = {
|
||||
map.put((datasetId, partition), value)
|
||||
return CachePutSuccess(0)
|
||||
}
|
||||
}
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
|
@ -10,6 +27,7 @@ import scala.collection.JavaConversions._
|
|||
import scala.collection.Map
|
||||
import scala.collection.generic.Growable
|
||||
import scala.collection.mutable.HashMap
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.util.DynamicVariable
|
||||
import scala.collection.mutable.{ConcurrentMap, HashMap}
|
||||
|
@ -36,18 +54,23 @@ import org.apache.hadoop.mapred.TextInputFormat
|
|||
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
|
||||
import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
|
||||
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
|
||||
import org.apache.hadoop.security.UserGroupInformation
|
||||
|
||||
import org.apache.mesos.MesosNativeLibrary
|
||||
|
||||
import spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
|
||||
import spark.partial.{ApproximateEvaluator, PartialResult}
|
||||
import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
|
||||
import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler}
|
||||
import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler}
|
||||
import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener,
|
||||
SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob}
|
||||
import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
|
||||
ClusterScheduler, Schedulable, SchedulingMode}
|
||||
import spark.scheduler.local.LocalScheduler
|
||||
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
|
||||
import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo}
|
||||
import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource}
|
||||
import spark.util.{MetadataCleaner, TimeStampedHashMap}
|
||||
import ui.{SparkUI}
|
||||
import spark.metrics._
|
||||
|
||||
/**
|
||||
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
|
||||
|
@ -93,11 +116,6 @@ class SparkContext(
|
|||
isLocal)
|
||||
SparkEnv.set(env)
|
||||
|
||||
// Start the BlockManager UI
|
||||
private[spark] val ui = new BlockManagerUI(
|
||||
env.actorSystem, env.blockManager.master.driverActor, this)
|
||||
ui.start()
|
||||
|
||||
// Used to store a URL for each static file/jar together with the file's local timestamp
|
||||
private[spark] val addedFiles = HashMap[String, Long]()
|
||||
private[spark] val addedJars = HashMap[String, Long]()
|
||||
|
@ -106,6 +124,11 @@ class SparkContext(
|
|||
private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
|
||||
private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup)
|
||||
|
||||
// Initalize the Spark UI
|
||||
private[spark] val ui = new SparkUI(this)
|
||||
ui.bind()
|
||||
|
||||
val startTime = System.currentTimeMillis()
|
||||
|
||||
// Add each JAR given through the constructor
|
||||
if (jars != null) {
|
||||
|
@ -115,13 +138,14 @@ class SparkContext(
|
|||
// Environment variables to pass to our executors
|
||||
private[spark] val executorEnvs = HashMap[String, String]()
|
||||
// Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
|
||||
for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS",
|
||||
"SPARK_TESTING")) {
|
||||
for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) {
|
||||
val value = System.getenv(key)
|
||||
if (value != null) {
|
||||
executorEnvs(key) = value
|
||||
}
|
||||
}
|
||||
// Since memory can be set with a system property too, use that
|
||||
executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m"
|
||||
if (environment != null) {
|
||||
executorEnvs ++= environment
|
||||
}
|
||||
|
@ -156,14 +180,12 @@ class SparkContext(
|
|||
scheduler
|
||||
|
||||
case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
|
||||
// Check to make sure SPARK_MEM <= memoryPerSlave. Otherwise Spark will just hang.
|
||||
// Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
|
||||
val memoryPerSlaveInt = memoryPerSlave.toInt
|
||||
val sparkMemEnv = System.getenv("SPARK_MEM")
|
||||
val sparkMemEnvInt = if (sparkMemEnv != null) Utils.memoryStringToMb(sparkMemEnv) else 512
|
||||
if (sparkMemEnvInt > memoryPerSlaveInt) {
|
||||
if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
|
||||
throw new SparkException(
|
||||
"Slave memory (%d MB) cannot be smaller than SPARK_MEM (%d MB)".format(
|
||||
memoryPerSlaveInt, sparkMemEnvInt))
|
||||
"Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
|
||||
memoryPerSlaveInt, SparkContext.executorMemoryRequested))
|
||||
}
|
||||
|
||||
val scheduler = new ClusterScheduler(this)
|
||||
|
@ -215,6 +237,8 @@ class SparkContext(
|
|||
@volatile private var dagScheduler = new DAGScheduler(taskScheduler)
|
||||
dagScheduler.start()
|
||||
|
||||
ui.start()
|
||||
|
||||
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
|
||||
val hadoopConfiguration = {
|
||||
val conf = SparkHadoopUtil.newConfiguration()
|
||||
|
@ -243,15 +267,31 @@ class SparkContext(
|
|||
localProperties.value = new Properties()
|
||||
}
|
||||
|
||||
def addLocalProperties(key: String, value: String) {
|
||||
def addLocalProperty(key: String, value: String) {
|
||||
if(localProperties.value == null) {
|
||||
localProperties.value = new Properties()
|
||||
}
|
||||
localProperties.value.setProperty(key,value)
|
||||
}
|
||||
|
||||
/** Set a human readable description of the current job. */
|
||||
def setDescription(value: String) {
|
||||
addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value)
|
||||
}
|
||||
|
||||
// Post init
|
||||
taskScheduler.postStartHook()
|
||||
|
||||
val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler)
|
||||
val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager)
|
||||
|
||||
def initDriverMetrics() {
|
||||
SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource)
|
||||
SparkEnv.get.metricsSystem.registerSource(blockManagerSource)
|
||||
}
|
||||
|
||||
initDriverMetrics()
|
||||
|
||||
// Methods for creating RDDs
|
||||
|
||||
/** Distribute a local Scala collection to form an RDD. */
|
||||
|
@ -528,6 +568,12 @@ class SparkContext(
|
|||
StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an immutable map of RDDs that have marked themselves as persistent via cache() call.
|
||||
* Note that this does not necessarily mean the caching or computation was successful.
|
||||
*/
|
||||
def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap
|
||||
|
||||
def getStageInfo: Map[Stage,StageInfo] = {
|
||||
dagScheduler.stageToInfos
|
||||
}
|
||||
|
@ -539,6 +585,28 @@ class SparkContext(
|
|||
env.blockManager.master.getStorageStatus
|
||||
}
|
||||
|
||||
/**
|
||||
* Return pools for fair scheduler
|
||||
* TODO(xiajunluan): We should take nested pools into account
|
||||
*/
|
||||
def getAllPools: ArrayBuffer[Schedulable] = {
|
||||
taskScheduler.rootPool.schedulableQueue
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the pool associated with the given name, if one exists
|
||||
*/
|
||||
def getPoolForName(pool: String): Option[Schedulable] = {
|
||||
taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)
|
||||
}
|
||||
|
||||
/**
|
||||
* Return current scheduling mode
|
||||
*/
|
||||
def getSchedulingMode: SchedulingMode.SchedulingMode = {
|
||||
taskScheduler.schedulingMode
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear the job's list of files added by `addFile` so that they do not get downloaded to
|
||||
* any new nodes.
|
||||
|
@ -559,7 +627,12 @@ class SparkContext(
|
|||
} else {
|
||||
val uri = new URI(path)
|
||||
val key = uri.getScheme match {
|
||||
case null | "file" => env.httpFileServer.addJar(new File(uri.getPath))
|
||||
case null | "file" =>
|
||||
if (SparkHadoopUtil.isYarnMode()) {
|
||||
logWarning("local jar specified as parameter to addJar under Yarn mode")
|
||||
return
|
||||
}
|
||||
env.httpFileServer.addJar(new File(uri.getPath))
|
||||
case _ => path
|
||||
}
|
||||
addedJars(key) = System.currentTimeMillis
|
||||
|
@ -577,6 +650,7 @@ class SparkContext(
|
|||
|
||||
/** Shut down the SparkContext. */
|
||||
def stop() {
|
||||
ui.stop()
|
||||
// Do this only if not stopped already - best case effort.
|
||||
// prevent NPE if stopped more than once.
|
||||
val dagSchedulerCopy = dagScheduler
|
||||
|
@ -775,6 +849,7 @@ class SparkContext(
|
|||
* various Spark features.
|
||||
*/
|
||||
object SparkContext {
|
||||
val SPARK_JOB_DESCRIPTION = "spark.job.description"
|
||||
|
||||
implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
|
||||
def addInPlace(t1: Double, t2: Double): Double = t1 + t2
|
||||
|
@ -881,8 +956,16 @@ object SparkContext {
|
|||
|
||||
/** Find the JAR that contains the class of a particular object */
|
||||
def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass)
|
||||
}
|
||||
|
||||
/** Get the amount of memory per executor requested through system properties or SPARK_MEM */
|
||||
private[spark] val executorMemoryRequested = {
|
||||
// TODO: Might need to add some extra memory for the non-heap parts of the JVM
|
||||
Option(System.getProperty("spark.executor.memory"))
|
||||
.orElse(Option(System.getenv("SPARK_MEM")))
|
||||
.map(Utils.memoryStringToMb)
|
||||
.getOrElse(512)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A class encapsulating how to convert some type T to Writable. It stores both the Writable class
|
||||
|
@ -895,3 +978,4 @@ private[spark] class WritableConverter[T](
|
|||
val writableClass: ClassManifest[T] => Class[_ <: Writable],
|
||||
val convert: Writable => T)
|
||||
extends Serializable
|
||||
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import collection.mutable
|
||||
|
@ -7,6 +24,7 @@ import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
|
|||
import akka.remote.RemoteActorRefProvider
|
||||
|
||||
import spark.broadcast.BroadcastManager
|
||||
import spark.metrics.MetricsSystem
|
||||
import spark.storage.BlockManager
|
||||
import spark.storage.BlockManagerMaster
|
||||
import spark.network.ConnectionManager
|
||||
|
@ -36,6 +54,7 @@ class SparkEnv (
|
|||
val connectionManager: ConnectionManager,
|
||||
val httpFileServer: HttpFileServer,
|
||||
val sparkFilesDir: String,
|
||||
val metricsSystem: MetricsSystem,
|
||||
// To be set only as part of initialization of SparkContext.
|
||||
// (executorId, defaultHostPort) => executorHostPort
|
||||
// If executorId is NOT found, return defaultHostPort
|
||||
|
@ -51,6 +70,7 @@ class SparkEnv (
|
|||
broadcastManager.stop()
|
||||
blockManager.stop()
|
||||
blockManager.master.stop()
|
||||
metricsSystem.stop()
|
||||
actorSystem.shutdown()
|
||||
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut
|
||||
// down, but let's call it anyway in case it gets fixed in a later release
|
||||
|
@ -59,7 +79,8 @@ class SparkEnv (
|
|||
|
||||
def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = {
|
||||
synchronized {
|
||||
pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create()
|
||||
val key = (pythonExec, envVars)
|
||||
pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -166,6 +187,13 @@ object SparkEnv extends Logging {
|
|||
httpFileServer.initialize()
|
||||
System.setProperty("spark.fileserver.uri", httpFileServer.serverUri)
|
||||
|
||||
val metricsSystem = if (isDriver) {
|
||||
MetricsSystem.createMetricsSystem("driver")
|
||||
} else {
|
||||
MetricsSystem.createMetricsSystem("executor")
|
||||
}
|
||||
metricsSystem.start()
|
||||
|
||||
// Set the sparkFiles directory, used when downloading dependencies. In local mode,
|
||||
// this is a temporary directory; in distributed mode, this is the executor's current working
|
||||
// directory.
|
||||
|
@ -195,6 +223,7 @@ object SparkEnv extends Logging {
|
|||
connectionManager,
|
||||
httpFileServer,
|
||||
sparkFilesDir,
|
||||
metricsSystem,
|
||||
None)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
class SparkException(message: String, cause: Throwable)
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark;
|
||||
|
||||
import java.io.File;
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import executor.TaskMetrics
|
||||
|
|
|
@ -1,5 +1,23 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import spark.executor.TaskMetrics
|
||||
import spark.storage.BlockManagerId
|
||||
|
||||
/**
|
||||
|
@ -24,7 +42,8 @@ private[spark] case class FetchFailed(
|
|||
private[spark] case class ExceptionFailure(
|
||||
className: String,
|
||||
description: String,
|
||||
stackTrace: Array[StackTraceElement])
|
||||
stackTrace: Array[StackTraceElement],
|
||||
metrics: Option[TaskMetrics])
|
||||
extends TaskEndReason
|
||||
|
||||
private[spark] case class OtherFailure(message: String) extends TaskEndReason
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import org.apache.mesos.Protos.{TaskState => MesosTaskState}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark
|
||||
|
||||
import java.io._
|
||||
|
@ -6,6 +23,7 @@ import java.util.{Locale, Random, UUID}
|
|||
import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor}
|
||||
import java.util.regex.Pattern
|
||||
|
||||
import scala.collection.Map
|
||||
import scala.collection.mutable.{ArrayBuffer, HashMap}
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.io.Source
|
||||
|
@ -485,6 +503,26 @@ private object Utils extends Logging {
|
|||
"%.1f %s".formatLocal(Locale.US, value, unit)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a human-readable string representing a duration such as "35ms"
|
||||
*/
|
||||
def msDurationToString(ms: Long): String = {
|
||||
val second = 1000
|
||||
val minute = 60 * second
|
||||
val hour = 60 * minute
|
||||
|
||||
ms match {
|
||||
case t if t < second =>
|
||||
"%d ms".format(t)
|
||||
case t if t < minute =>
|
||||
"%.1f s".format(t.toFloat / second)
|
||||
case t if t < hour =>
|
||||
"%.1f m".format(t.toFloat / minute)
|
||||
case t =>
|
||||
"%.2f h".format(t.toFloat / hour)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a memory quantity in megabytes to a human-readable string such as "4.0 MB".
|
||||
*/
|
||||
|
@ -525,10 +563,15 @@ private object Utils extends Logging {
|
|||
/**
|
||||
* Execute a command and get its output, throwing an exception if it yields a code other than 0.
|
||||
*/
|
||||
def executeAndGetOutput(command: Seq[String], workingDir: File = new File(".")): String = {
|
||||
val process = new ProcessBuilder(command: _*)
|
||||
def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."),
|
||||
extraEnvironment: Map[String, String] = Map.empty): String = {
|
||||
val builder = new ProcessBuilder(command: _*)
|
||||
.directory(workingDir)
|
||||
.start()
|
||||
val environment = builder.environment()
|
||||
for ((key, value) <- extraEnvironment) {
|
||||
environment.put(key, value)
|
||||
}
|
||||
val process = builder.start()
|
||||
new Thread("read stderr for " + command(0)) {
|
||||
override def run() {
|
||||
for (line <- Source.fromInputStream(process.getErrorStream).getLines) {
|
||||
|
@ -553,8 +596,15 @@ private object Utils extends Logging {
|
|||
output.toString
|
||||
}
|
||||
|
||||
/**
|
||||
* A regular expression to match classes of the "core" Spark API that we want to skip when
|
||||
* finding the call site of a method.
|
||||
*/
|
||||
private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r
|
||||
|
||||
private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String,
|
||||
val firstUserLine: Int, val firstUserClass: String)
|
||||
|
||||
/**
|
||||
* When called inside a class in the spark package, returns the name of the user code class
|
||||
* (outside the spark package) that called into Spark, as well as which Spark method they called.
|
||||
|
@ -576,7 +626,7 @@ private object Utils extends Logging {
|
|||
|
||||
for (el <- trace) {
|
||||
if (!finished) {
|
||||
if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) {
|
||||
if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) {
|
||||
lastSparkMethod = if (el.getMethodName == "<init>") {
|
||||
// Spark method is a constructor; get its class name
|
||||
el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1)
|
||||
|
@ -600,18 +650,20 @@ private object Utils extends Logging {
|
|||
"%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile,
|
||||
callSiteInfo.firstUserLine)
|
||||
}
|
||||
/**
|
||||
* Try to find a free port to bind to on the local host. This should ideally never be needed,
|
||||
* except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray)
|
||||
* don't let users bind to port 0 and then figure out which free port they actually bound to.
|
||||
* We work around this by binding a ServerSocket and immediately unbinding it. This is *not*
|
||||
* necessarily guaranteed to work, but it's the best we can do.
|
||||
*/
|
||||
def findFreePort(): Int = {
|
||||
val socket = new ServerSocket(0)
|
||||
val portBound = socket.getLocalPort
|
||||
socket.close()
|
||||
portBound
|
||||
|
||||
/** Return a string containing part of a file from byte 'start' to 'end'. */
|
||||
def offsetBytes(path: String, start: Long, end: Long): String = {
|
||||
val file = new File(path)
|
||||
val length = file.length()
|
||||
val effectiveEnd = math.min(length, end)
|
||||
val effectiveStart = math.max(0, start)
|
||||
val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt)
|
||||
val stream = new FileInputStream(file)
|
||||
|
||||
stream.skip(effectiveStart)
|
||||
stream.read(buff)
|
||||
stream.close()
|
||||
Source.fromBytes(buff).mkString
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -704,4 +756,13 @@ private object Utils extends Logging {
|
|||
}
|
||||
return buf
|
||||
}
|
||||
|
||||
/* Calculates 'x' modulo 'mod', takes to consideration sign of x,
|
||||
* i.e. if 'x' is negative, than 'x' % 'mod' is negative too
|
||||
* so function return (x % mod) + mod in that case.
|
||||
*/
|
||||
def nonNegativeMod(x: Int, mod: Int): Int = {
|
||||
val rawMod = x % mod
|
||||
rawMod + (if (rawMod < 0) mod else 0)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.api.java
|
||||
|
||||
import spark.RDD
|
||||
|
@ -98,33 +115,48 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
|
|||
|
||||
// Double RDD functions
|
||||
|
||||
/** Return the sum of the elements in this RDD. */
|
||||
/** Add up the elements in this RDD. */
|
||||
def sum(): Double = srdd.sum()
|
||||
|
||||
/** Return a [[spark.StatCounter]] describing the elements in this RDD. */
|
||||
/**
|
||||
* Return a [[spark.util.StatCounter]] object that captures the mean, variance and count
|
||||
* of the RDD's elements in one operation.
|
||||
*/
|
||||
def stats(): StatCounter = srdd.stats()
|
||||
|
||||
/** Return the mean of the elements in this RDD. */
|
||||
/** Compute the mean of this RDD's elements. */
|
||||
def mean(): Double = srdd.mean()
|
||||
|
||||
/** Return the variance of the elements in this RDD. */
|
||||
/** Compute the variance of this RDD's elements. */
|
||||
def variance(): Double = srdd.variance()
|
||||
|
||||
/** Return the standard deviation of the elements in this RDD. */
|
||||
/** Compute the standard deviation of this RDD's elements. */
|
||||
def stdev(): Double = srdd.stdev()
|
||||
|
||||
/**
|
||||
* Compute the sample standard deviation of this RDD's elements (which corrects for bias in
|
||||
* estimating the standard deviation by dividing by N-1 instead of N).
|
||||
*/
|
||||
def sampleStdev(): Double = srdd.sampleStdev()
|
||||
|
||||
/**
|
||||
* Compute the sample variance of this RDD's elements (which corrects for bias in
|
||||
* estimating the standard variance by dividing by N-1 instead of N).
|
||||
*/
|
||||
def sampleVariance(): Double = srdd.sampleVariance()
|
||||
|
||||
/** Return the approximate mean of the elements in this RDD. */
|
||||
def meanApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] =
|
||||
srdd.meanApprox(timeout, confidence)
|
||||
|
||||
/** Return the approximate mean of the elements in this RDD. */
|
||||
/** (Experimental) Approximate operation to return the mean within a timeout. */
|
||||
def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout)
|
||||
|
||||
/** Return the approximate sum of the elements in this RDD. */
|
||||
/** (Experimental) Approximate operation to return the sum within a timeout. */
|
||||
def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] =
|
||||
srdd.sumApprox(timeout, confidence)
|
||||
|
||||
/** Return the approximate sum of the elements in this RDD. */
|
||||
|
||||
/** (Experimental) Approximate operation to return the sum within a timeout. */
|
||||
def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout)
|
||||
}
|
||||
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.api.java
|
||||
|
||||
import java.util.{List => JList}
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.api.java
|
||||
|
||||
import spark._
|
||||
|
|
|
@ -1,3 +1,20 @@
|
|||
/*
|
||||
* 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 spark.api.java
|
||||
|
||||
import java.util.{List => JList, Comparator}
|
||||
|
@ -384,4 +401,29 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
|
|||
val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]]
|
||||
top(num, comp)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the first K elements from this RDD as defined by
|
||||
* the specified Comparator[T] and maintains the order.
|
||||
* @param num the number of top elements to return
|
||||
* @param comp the comparator that defines the order
|
||||
* @return an array of top elements
|
||||
*/
|
||||
def takeOrdered(num: Int, comp: Comparator[T]): JList[T] = {
|
||||
import scala.collection.JavaConversions._
|
||||
val topElems = rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp))
|
||||
val arr: java.util.Collection[T] = topElems.toSeq
|
||||
new java.util.ArrayList(arr)
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the first K elements from this RDD using the
|
||||
* natural ordering for T while maintain the order.
|
||||
* @param num the number of top elements to return
|
||||
* @return an array of top elements
|
||||
*/
|
||||
def takeOrdered(num: Int): JList[T] = {
|
||||
val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]]
|
||||
takeOrdered(num, comp)
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue