StructType |
- org.apache.spark.sql.api.java.Row |
+ org.apache.spark.sql.Row |
DataTypes.createStructType(fields)
Note: fields is a List or an array of StructFields.
@@ -1478,10 +2000,10 @@ please use factory methods provided in
-All data types of Spark SQL are located in the package of `pyspark.sql`.
+All data types of Spark SQL are located in the package of `pyspark.sql.types`.
You can access them by doing
{% highlight python %}
-from pyspark.sql import *
+from pyspark.sql.types import *
{% endhighlight %}
diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
index 740c267fd9866..0857657152ec7 100644
--- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
+++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
@@ -26,9 +26,10 @@ export SPARK_LOCAL_DIRS="{{spark_local_dirs}}"
export MODULES="{{modules}}"
export SPARK_VERSION="{{spark_version}}"
export SHARK_VERSION="{{shark_version}}"
+export TACHYON_VERSION="{{tachyon_version}}"
export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}"
export SWAP_MB="{{swap}}"
export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}"
export SPARK_MASTER_OPTS="{{spark_master_opts}}"
export AWS_ACCESS_KEY_ID="{{aws_access_key_id}}"
-export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}"
\ No newline at end of file
+export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}"
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index c59ab565c6862..f848874b0c775 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -22,6 +22,7 @@
from __future__ import with_statement
import hashlib
+import itertools
import logging
import os
import os.path
@@ -61,6 +62,16 @@
"1.2.1",
])
+SPARK_TACHYON_MAP = {
+ "1.0.0": "0.4.1",
+ "1.0.1": "0.4.1",
+ "1.0.2": "0.4.1",
+ "1.1.0": "0.5.0",
+ "1.1.1": "0.5.0",
+ "1.2.0": "0.5.0",
+ "1.2.1": "0.5.0",
+}
+
DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION
DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark"
@@ -69,34 +80,60 @@
DEFAULT_SPARK_EC2_BRANCH = "branch-1.3"
-def setup_boto():
- # Download Boto if it's not already present in the SPARK_EC2_DIR/lib folder:
- version = "boto-2.34.0"
- md5 = "5556223d2d0cc4d06dd4829e671dcecd"
- url = "https://pypi.python.org/packages/source/b/boto/%s.tar.gz" % version
- lib_dir = os.path.join(SPARK_EC2_DIR, "lib")
- if not os.path.exists(lib_dir):
- os.mkdir(lib_dir)
- boto_lib_dir = os.path.join(lib_dir, version)
- if not os.path.isdir(boto_lib_dir):
- tgz_file_path = os.path.join(lib_dir, "%s.tar.gz" % version)
- print "Downloading Boto from PyPi"
- download_stream = urllib2.urlopen(url)
- with open(tgz_file_path, "wb") as tgz_file:
- tgz_file.write(download_stream.read())
- with open(tgz_file_path) as tar:
- if hashlib.md5(tar.read()).hexdigest() != md5:
- print >> stderr, "ERROR: Got wrong md5sum for Boto"
- sys.exit(1)
- tar = tarfile.open(tgz_file_path)
- tar.extractall(path=lib_dir)
- tar.close()
- os.remove(tgz_file_path)
- print "Finished downloading Boto"
- sys.path.insert(0, boto_lib_dir)
+def setup_external_libs(libs):
+ """
+ Download external libraries from PyPI to SPARK_EC2_DIR/lib/ and prepend them to our PATH.
+ """
+ PYPI_URL_PREFIX = "https://pypi.python.org/packages/source"
+ SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib")
+
+ if not os.path.exists(SPARK_EC2_LIB_DIR):
+ print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format(
+ path=SPARK_EC2_LIB_DIR
+ )
+ print "This should be a one-time operation."
+ os.mkdir(SPARK_EC2_LIB_DIR)
+
+ for lib in libs:
+ versioned_lib_name = "{n}-{v}".format(n=lib["name"], v=lib["version"])
+ lib_dir = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name)
+
+ if not os.path.isdir(lib_dir):
+ tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz")
+ print " - Downloading {lib}...".format(lib=lib["name"])
+ download_stream = urllib2.urlopen(
+ "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format(
+ prefix=PYPI_URL_PREFIX,
+ first_letter=lib["name"][:1],
+ lib_name=lib["name"],
+ lib_version=lib["version"]
+ )
+ )
+ with open(tgz_file_path, "wb") as tgz_file:
+ tgz_file.write(download_stream.read())
+ with open(tgz_file_path) as tar:
+ if hashlib.md5(tar.read()).hexdigest() != lib["md5"]:
+ print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"])
+ sys.exit(1)
+ tar = tarfile.open(tgz_file_path)
+ tar.extractall(path=SPARK_EC2_LIB_DIR)
+ tar.close()
+ os.remove(tgz_file_path)
+ print " - Finished downloading {lib}.".format(lib=lib["name"])
+ sys.path.insert(1, lib_dir)
+
+
+# Only PyPI libraries are supported.
+external_libs = [
+ {
+ "name": "boto",
+ "version": "2.34.0",
+ "md5": "5556223d2d0cc4d06dd4829e671dcecd"
+ }
+]
+setup_external_libs(external_libs)
-setup_boto()
import boto
from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType
from boto import ec2
@@ -135,7 +172,7 @@ def parse_args():
help="Master instance type (leave empty for same as instance-type)")
parser.add_option(
"-r", "--region", default="us-east-1",
- help="EC2 region used to launch instances in, or to find them in")
+ help="EC2 region used to launch instances in, or to find them in (default: %default)")
parser.add_option(
"-z", "--zone", default="",
help="Availability zone to launch instances in, or 'all' to spread " +
@@ -159,6 +196,15 @@ def parse_args():
"--spark-ec2-git-branch",
default=DEFAULT_SPARK_EC2_BRANCH,
help="Github repo branch of spark-ec2 to use (default: %default)")
+ parser.add_option(
+ "--deploy-root-dir",
+ default=None,
+ help="A directory to copy into / on the first master. " +
+ "Must be absolute. Note that a trailing slash is handled as per rsync: " +
+ "If you omit it, the last directory of the --deploy-root-dir path will be created " +
+ "in / before copying its contents. If you append the trailing slash, " +
+ "the directory is not created and its contents are copied directly into /. " +
+ "(default: %default).")
parser.add_option(
"--hadoop-major-version", default="1",
help="Major version of Hadoop (default: %default)")
@@ -220,7 +266,7 @@ def parse_args():
"(e.g -Dspark.worker.timeout=180)")
parser.add_option(
"--user-data", type="string", default="",
- help="Path to a user-data file (most AMI's interpret this as an initialization script)")
+ help="Path to a user-data file (most AMIs interpret this as an initialization script)")
parser.add_option(
"--authorized-address", type="string", default="0.0.0.0/0",
help="Address to authorize on created security groups (default: %default)")
@@ -290,13 +336,6 @@ def get_validate_spark_version(version, repo):
return version
-# Check whether a given EC2 instance object is in a state we consider active,
-# i.e. not terminating or terminated. We count both stopping and stopped as
-# active since we can restart stopped clusters.
-def is_active(instance):
- return (instance.state in ['pending', 'running', 'stopping', 'stopped'])
-
-
# Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/
# Last Updated: 2014-06-20
# For easy maintainability, please keep this manually-inputted dictionary sorted by key.
@@ -341,6 +380,10 @@ def is_active(instance):
}
+def get_tachyon_version(spark_version):
+ return SPARK_TACHYON_MAP.get(spark_version, "")
+
+
# Attempt to resolve an appropriate AMI given the architecture and region of the request.
def get_spark_ami(opts):
if opts.instance_type in EC2_INSTANCE_TYPES:
@@ -564,8 +607,11 @@ def launch_cluster(conn, opts, cluster_name):
placement_group=opts.placement_group,
user_data=user_data_content)
slave_nodes += slave_res.instances
- print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone,
- zone, slave_res.id)
+ print "Launched {s} slave{plural_s} in {z}, regid = {r}".format(
+ s=num_slaves_this_zone,
+ plural_s=('' if num_slaves_this_zone == 1 else 's'),
+ z=zone,
+ r=slave_res.id)
i += 1
# Launch or resume masters
@@ -612,40 +658,47 @@ def launch_cluster(conn, opts, cluster_name):
return (master_nodes, slave_nodes)
-# Get the EC2 instances in an existing cluster if available.
-# Returns a tuple of lists of EC2 instance objects for the masters and slaves
def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
- print "Searching for existing cluster " + cluster_name + " in region " \
- + opts.region + "..."
- reservations = conn.get_all_reservations()
- master_nodes = []
- slave_nodes = []
- for res in reservations:
- active = [i for i in res.instances if is_active(i)]
- for inst in active:
- group_names = [g.name for g in inst.groups]
- if (cluster_name + "-master") in group_names:
- master_nodes.append(inst)
- elif (cluster_name + "-slaves") in group_names:
- slave_nodes.append(inst)
- if any((master_nodes, slave_nodes)):
- print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))
- if master_nodes != [] or not die_on_error:
- return (master_nodes, slave_nodes)
- else:
- if master_nodes == [] and slave_nodes != []:
- print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name \
- + "-master" + " in region " + opts.region
- else:
- print >> sys.stderr, "ERROR: Could not find any existing cluster" \
- + " in region " + opts.region
+ """
+ Get the EC2 instances in an existing cluster if available.
+ Returns a tuple of lists of EC2 instance objects for the masters and slaves.
+ """
+ print "Searching for existing cluster {c} in region {r}...".format(
+ c=cluster_name, r=opts.region)
+
+ def get_instances(group_names):
+ """
+ Get all non-terminated instances that belong to any of the provided security groups.
+
+ EC2 reservation filters and instance states are documented here:
+ http://docs.aws.amazon.com/cli/latest/reference/ec2/describe-instances.html#options
+ """
+ reservations = conn.get_all_reservations(
+ filters={"instance.group-name": group_names})
+ instances = itertools.chain.from_iterable(r.instances for r in reservations)
+ return [i for i in instances if i.state not in ["shutting-down", "terminated"]]
+
+ master_instances = get_instances([cluster_name + "-master"])
+ slave_instances = get_instances([cluster_name + "-slaves"])
+
+ if any((master_instances, slave_instances)):
+ print "Found {m} master{plural_m}, {s} slave{plural_s}.".format(
+ m=len(master_instances),
+ plural_m=('' if len(master_instances) == 1 else 's'),
+ s=len(slave_instances),
+ plural_s=('' if len(slave_instances) == 1 else 's'))
+
+ if not master_instances and die_on_error:
+ print >> sys.stderr, \
+ "ERROR: Could not find a master for cluster {c} in region {r}.".format(
+ c=cluster_name, r=opts.region)
sys.exit(1)
+ return (master_instances, slave_instances)
+
# Deploy configuration files and run setup scripts on a newly launched
# or started EC2 cluster.
-
-
def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
master = master_nodes[0].public_dns_name
if deploy_ssh_key:
@@ -694,6 +747,14 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
modules=modules
)
+ if opts.deploy_root_dir is not None:
+ print "Deploying {s} to master...".format(s=opts.deploy_root_dir)
+ deploy_user_files(
+ root_dir=opts.deploy_root_dir,
+ opts=opts,
+ master_nodes=master_nodes
+ )
+
print "Running setup on master..."
setup_spark_cluster(master, opts)
print "Done!"
@@ -872,9 +933,13 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
if "." in opts.spark_version:
# Pre-built Spark deploy
spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo)
+ tachyon_v = get_tachyon_version(spark_v)
else:
# Spark-only custom deploy
spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version)
+ tachyon_v = ""
+ print "Deploying Spark via git hash; Tachyon won't be set up"
+ modules = filter(lambda x: x != "tachyon", modules)
template_vars = {
"master_list": '\n'.join([i.public_dns_name for i in master_nodes]),
@@ -887,6 +952,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
"swap": str(opts.swap),
"modules": '\n'.join(modules),
"spark_version": spark_v,
+ "tachyon_version": tachyon_v,
"hadoop_major_version": opts.hadoop_major_version,
"spark_worker_instances": "%d" % opts.worker_instances,
"spark_master_opts": opts.master_opts
@@ -931,6 +997,23 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
shutil.rmtree(tmp_dir)
+# Deploy a given local directory to a cluster, WITHOUT parameter substitution.
+# Note that unlike deploy_files, this works for binary files.
+# Also, it is up to the user to add (or not) the trailing slash in root_dir.
+# Files are only deployed to the first master instance in the cluster.
+#
+# root_dir should be an absolute path.
+def deploy_user_files(root_dir, opts, master_nodes):
+ active_master = master_nodes[0].public_dns_name
+ command = [
+ 'rsync', '-rv',
+ '-e', stringify_command(ssh_command(opts)),
+ "%s" % root_dir,
+ "%s@%s:/" % (opts.user, active_master)
+ ]
+ subprocess.check_call(command)
+
+
def stringify_command(parts):
if isinstance(parts, str):
return parts
@@ -1099,6 +1182,14 @@ def real_main():
"Furthermore, we currently only support forks named spark-ec2."
sys.exit(1)
+ if not (opts.deploy_root_dir is None or
+ (os.path.isabs(opts.deploy_root_dir) and
+ os.path.isdir(opts.deploy_root_dir) and
+ os.path.exists(opts.deploy_root_dir))):
+ print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \
+ "on the local file system"
+ sys.exit(1)
+
try:
conn = ec2.connect_to_region(opts.region)
except Exception as e:
@@ -1126,14 +1217,16 @@ def real_main():
setup_cluster(conn, master_nodes, slave_nodes, opts, True)
elif action == "destroy":
- print "Are you sure you want to destroy the cluster %s?" % cluster_name
- print "The following instances will be terminated:"
(master_nodes, slave_nodes) = get_existing_cluster(
conn, opts, cluster_name, die_on_error=False)
- for inst in master_nodes + slave_nodes:
- print "> %s" % inst.public_dns_name
- msg = "ALL DATA ON ALL NODES WILL BE LOST!!\nDestroy cluster %s (y/N): " % cluster_name
+ if any(master_nodes + slave_nodes):
+ print "The following instances will be terminated:"
+ for inst in master_nodes + slave_nodes:
+ print "> %s" % inst.public_dns_name
+ print "ALL DATA ON ALL NODES WILL BE LOST!!"
+
+ msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name)
response = raw_input(msg)
if response == "y":
print "Terminating master..."
@@ -1145,7 +1238,6 @@ def real_main():
# Delete security groups as well
if opts.delete_groups:
- print "Deleting security groups (this will take some time)..."
group_names = [cluster_name + "-master", cluster_name + "-slaves"]
wait_for_cluster_state(
conn=conn,
@@ -1153,6 +1245,7 @@ def real_main():
cluster_instances=(master_nodes + slave_nodes),
cluster_state='terminated'
)
+ print "Deleting security groups (this will take some time)..."
attempt = 1
while attempt <= 3:
print "Attempt %d" % attempt
@@ -1259,6 +1352,17 @@ def real_main():
cluster_instances=(master_nodes + slave_nodes),
cluster_state='ssh-ready'
)
+
+ # Determine types of running instances
+ existing_master_type = master_nodes[0].instance_type
+ existing_slave_type = slave_nodes[0].instance_type
+ # Setting opts.master_instance_type to the empty string indicates we
+ # have the same instance type for the master and the slaves
+ if existing_master_type == existing_slave_type:
+ existing_master_type = ""
+ opts.master_instance_type = existing_master_type
+ opts.instance_type = existing_slave_type
+
setup_cluster(conn, master_nodes, slave_nodes, opts, False)
else:
diff --git a/examples/pom.xml b/examples/pom.xml
index 8caad2bc2e27a..994071d94d0ad 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
index e809a65b79975..f6f8d9f90c275 100644
--- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala
@@ -17,11 +17,6 @@
package org.apache.spark.examples.graphx
-import org.apache.spark.SparkContext._
-import org.apache.spark._
-import org.apache.spark.graphx._
-
-
/**
* Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from
* http://snap.stanford.edu/data/soc-LiveJournal1.html.
@@ -31,13 +26,13 @@ object LiveJournalPageRank {
if (args.length < 1) {
System.err.println(
"Usage: LiveJournalPageRank \n" +
+ " --numEPart=\n" +
+ " The number of partitions for the graph's edge RDD.\n" +
" [--tol=]\n" +
" The tolerance allowed at convergence (smaller => more accurate). Default is " +
"0.001.\n" +
" [--output=]\n" +
" If specified, the file to write the ranks to.\n" +
- " [--numEPart=]\n" +
- " The number of partitions for the graph's edge RDD. Default is 4.\n" +
" [--partStrategy=RandomVertexCut | EdgePartition1D | EdgePartition2D | " +
"CanonicalRandomVertexCut]\n" +
" The way edges are assigned to edge partitions. Default is RandomVertexCut.")
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
index 11399a7633638..08a93595a2e17 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
@@ -173,7 +173,9 @@ object LDAExample {
stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = {
// Get dataset of document texts
- // One document per line in each text file.
+ // One document per line in each text file. If the input consists of many small files,
+ // this can result in a large number of small partitions, which can degrade performance.
+ // In this case, consider using coalesce() to create fewer, larger partitions.
val textRDD: RDD[String] = sc.textFile(paths.mkString(","))
// Split text into words
diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml
index 0706f1ebf66e2..96c2787e35cd0 100644
--- a/external/flume-sink/pom.xml
+++ b/external/flume-sink/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/flume/pom.xml b/external/flume/pom.xml
index 1f2681394c583..172d447b77cda 100644
--- a/external/flume/pom.xml
+++ b/external/flume/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml
index 8daa7ed608f6a..5109b8ed87524 100644
--- a/external/kafka-assembly/pom.xml
+++ b/external/kafka-assembly/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
index af96138d79405..369856187a244 100644
--- a/external/kafka/pom.xml
+++ b/external/kafka/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
index 560c8b9d18276..a344f000c5002 100644
--- a/external/mqtt/pom.xml
+++ b/external/mqtt/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
index da6ffe7662f63..e95853f005ce2 100644
--- a/external/twitter/pom.xml
+++ b/external/twitter/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
index e919c2c9b19ea..9b3475d7c3dc2 100644
--- a/external/zeromq/pom.xml
+++ b/external/zeromq/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml
index 0fb431808bacd..bc2f8be10c9ce 100644
--- a/extras/java8-tests/pom.xml
+++ b/extras/java8-tests/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml
index 216661b8bc73a..7e49a71907336 100644
--- a/extras/kinesis-asl/pom.xml
+++ b/extras/kinesis-asl/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml
index f2f0aa78b0a4b..6eb29af03f833 100644
--- a/extras/spark-ganglia-lgpl/pom.xml
+++ b/extras/spark-ganglia-lgpl/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/graphx/pom.xml b/graphx/pom.xml
index 8fac24b6ed86d..57e338c03ecf9 100644
--- a/graphx/pom.xml
+++ b/graphx/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/launcher/pom.xml b/launcher/pom.xml
new file mode 100644
index 0000000000000..ccbd9d0419a98
--- /dev/null
+++ b/launcher/pom.xml
@@ -0,0 +1,83 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.10
+ 1.3.0-SNAPSHOT
+ ../pom.xml
+
+
+ org.apache.spark
+ spark-launcher_2.10
+ jar
+ Spark Launcher Project
+ http://spark.apache.org/
+
+ launcher
+
+
+
+
+
+ log4j
+ log4j
+ test
+
+
+ junit
+ junit
+ test
+
+
+ org.mockito
+ mockito-all
+ test
+
+
+ org.scalatest
+ scalatest_${scala.binary.version}
+ test
+
+
+ org.slf4j
+ slf4j-api
+ test
+
+
+ org.slf4j
+ slf4j-log4j12
+ test
+
+
+
+
+ org.apache.hadoop
+ hadoop-client
+ test
+
+
+
+
+ target/scala-${scala.binary.version}/classes
+ target/scala-${scala.binary.version}/test-classes
+
+
diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
new file mode 100644
index 0000000000000..dc90e9e987234
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
@@ -0,0 +1,362 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.jar.JarFile;
+import java.util.regex.Pattern;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Abstract Spark command builder that defines common functionality.
+ */
+abstract class AbstractCommandBuilder {
+
+ boolean verbose;
+ String appName;
+ String appResource;
+ String deployMode;
+ String javaHome;
+ String mainClass;
+ String master;
+ String propertiesFile;
+ final List appArgs;
+ final List jars;
+ final List files;
+ final List pyFiles;
+ final Map childEnv;
+ final Map conf;
+
+ public AbstractCommandBuilder() {
+ this.appArgs = new ArrayList();
+ this.childEnv = new HashMap();
+ this.conf = new HashMap();
+ this.files = new ArrayList();
+ this.jars = new ArrayList();
+ this.pyFiles = new ArrayList();
+ }
+
+ /**
+ * Builds the command to execute.
+ *
+ * @param env A map containing environment variables for the child process. It may already contain
+ * entries defined by the user (such as SPARK_HOME, or those defined by the
+ * SparkLauncher constructor that takes an environment), and may be modified to
+ * include other variables needed by the process to be executed.
+ */
+ abstract List buildCommand(Map env) throws IOException;
+
+ /**
+ * Builds a list of arguments to run java.
+ *
+ * This method finds the java executable to use and appends JVM-specific options for running a
+ * class with Spark in the classpath. It also loads options from the "java-opts" file in the
+ * configuration directory being used.
+ *
+ * Callers should still add at least the class to run, as well as any arguments to pass to the
+ * class.
+ */
+ List buildJavaCommand(String extraClassPath) throws IOException {
+ List cmd = new ArrayList();
+ if (javaHome == null) {
+ cmd.add(join(File.separator, System.getProperty("java.home"), "bin", "java"));
+ } else {
+ cmd.add(join(File.separator, javaHome, "bin", "java"));
+ }
+
+ // Load extra JAVA_OPTS from conf/java-opts, if it exists.
+ File javaOpts = new File(join(File.separator, getConfDir(), "java-opts"));
+ if (javaOpts.isFile()) {
+ BufferedReader br = new BufferedReader(new InputStreamReader(
+ new FileInputStream(javaOpts), "UTF-8"));
+ try {
+ String line;
+ while ((line = br.readLine()) != null) {
+ addOptionString(cmd, line);
+ }
+ } finally {
+ br.close();
+ }
+ }
+
+ cmd.add("-cp");
+ cmd.add(join(File.pathSeparator, buildClassPath(extraClassPath)));
+ return cmd;
+ }
+
+ /**
+ * Adds the default perm gen size option for Spark if the VM requires it and the user hasn't
+ * set it.
+ */
+ void addPermGenSizeOpt(List cmd) {
+ // Don't set MaxPermSize for Java 8 and later.
+ String[] version = System.getProperty("java.version").split("\\.");
+ if (Integer.parseInt(version[0]) > 1 || Integer.parseInt(version[1]) > 7) {
+ return;
+ }
+
+ for (String arg : cmd) {
+ if (arg.startsWith("-XX:MaxPermSize=")) {
+ return;
+ }
+ }
+
+ cmd.add("-XX:MaxPermSize=128m");
+ }
+
+ void addOptionString(List cmd, String options) {
+ if (!isEmpty(options)) {
+ for (String opt : parseOptionString(options)) {
+ cmd.add(opt);
+ }
+ }
+ }
+
+ /**
+ * Builds the classpath for the application. Returns a list with one classpath entry per element;
+ * each entry is formatted in the way expected by java.net.URLClassLoader (more
+ * specifically, with trailing slashes for directories).
+ */
+ List buildClassPath(String appClassPath) throws IOException {
+ String sparkHome = getSparkHome();
+ String scala = getScalaVersion();
+
+ List cp = new ArrayList();
+ addToClassPath(cp, getenv("SPARK_CLASSPATH"));
+ addToClassPath(cp, appClassPath);
+
+ addToClassPath(cp, getConfDir());
+
+ boolean prependClasses = !isEmpty(getenv("SPARK_PREPEND_CLASSES"));
+ boolean isTesting = "1".equals(getenv("SPARK_TESTING"));
+ if (prependClasses || isTesting) {
+ List projects = Arrays.asList("core", "repl", "mllib", "bagel", "graphx",
+ "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver",
+ "yarn", "launcher");
+ if (prependClasses) {
+ System.err.println(
+ "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " +
+ "assembly.");
+ for (String project : projects) {
+ addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project,
+ scala));
+ }
+ }
+ if (isTesting) {
+ for (String project : projects) {
+ addToClassPath(cp, String.format("%s/%s/target/scala-%s/test-classes", sparkHome,
+ project, scala));
+ }
+ }
+
+ // Add this path to include jars that are shaded in the final deliverable created during
+ // the maven build. These jars are copied to this directory during the build.
+ addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome));
+ }
+
+ String assembly = findAssembly(scala);
+ addToClassPath(cp, assembly);
+
+ // When Hive support is needed, Datanucleus jars must be included on the classpath. Datanucleus
+ // jars do not work if only included in the uber jar as plugin.xml metadata is lost. Both sbt
+ // and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is built
+ // with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark
+ // assembly is built for Hive, before actually populating the CLASSPATH with the jars.
+ //
+ // This block also serves as a check for SPARK-1703, when the assembly jar is built with
+ // Java 7 and ends up with too many files, causing issues with other JDK versions.
+ boolean needsDataNucleus = false;
+ JarFile assemblyJar = null;
+ try {
+ assemblyJar = new JarFile(assembly);
+ needsDataNucleus = assemblyJar.getEntry("org/apache/hadoop/hive/ql/exec/") != null;
+ } catch (IOException ioe) {
+ if (ioe.getMessage().indexOf("invalid CEN header") >= 0) {
+ System.err.println(
+ "Loading Spark jar failed.\n" +
+ "This is likely because Spark was compiled with Java 7 and run\n" +
+ "with Java 6 (see SPARK-1703). Please use Java 7 to run Spark\n" +
+ "or build Spark with Java 6.");
+ System.exit(1);
+ } else {
+ throw ioe;
+ }
+ } finally {
+ if (assemblyJar != null) {
+ try {
+ assemblyJar.close();
+ } catch (IOException e) {
+ // Ignore.
+ }
+ }
+ }
+
+ if (needsDataNucleus) {
+ System.err.println("Spark assembly has been built with Hive, including Datanucleus jars " +
+ "in classpath.");
+ File libdir;
+ if (new File(sparkHome, "RELEASE").isFile()) {
+ libdir = new File(sparkHome, "lib");
+ } else {
+ libdir = new File(sparkHome, "lib_managed/jars");
+ }
+
+ checkState(libdir.isDirectory(), "Library directory '%s' does not exist.",
+ libdir.getAbsolutePath());
+ for (File jar : libdir.listFiles()) {
+ if (jar.getName().startsWith("datanucleus-")) {
+ addToClassPath(cp, jar.getAbsolutePath());
+ }
+ }
+ }
+
+ addToClassPath(cp, getenv("HADOOP_CONF_DIR"));
+ addToClassPath(cp, getenv("YARN_CONF_DIR"));
+ addToClassPath(cp, getenv("SPARK_DIST_CLASSPATH"));
+ return cp;
+ }
+
+ /**
+ * Adds entries to the classpath.
+ *
+ * @param cp List to which the new entries are appended.
+ * @param entries New classpath entries (separated by File.pathSeparator).
+ */
+ private void addToClassPath(List cp, String entries) {
+ if (isEmpty(entries)) {
+ return;
+ }
+ String[] split = entries.split(Pattern.quote(File.pathSeparator));
+ for (String entry : split) {
+ if (!isEmpty(entry)) {
+ if (new File(entry).isDirectory() && !entry.endsWith(File.separator)) {
+ entry += File.separator;
+ }
+ cp.add(entry);
+ }
+ }
+ }
+
+ String getScalaVersion() {
+ String scala = getenv("SPARK_SCALA_VERSION");
+ if (scala != null) {
+ return scala;
+ }
+
+ String sparkHome = getSparkHome();
+ File scala210 = new File(sparkHome, "assembly/target/scala-2.10");
+ File scala211 = new File(sparkHome, "assembly/target/scala-2.11");
+ checkState(!scala210.isDirectory() || !scala211.isDirectory(),
+ "Presence of build for both scala versions (2.10 and 2.11) detected.\n" +
+ "Either clean one of them or set SPARK_SCALA_VERSION in your environment.");
+ if (scala210.isDirectory()) {
+ return "2.10";
+ } else {
+ checkState(scala211.isDirectory(), "Cannot find any assembly build directories.");
+ return "2.11";
+ }
+ }
+
+ String getSparkHome() {
+ String path = getenv(ENV_SPARK_HOME);
+ checkState(path != null,
+ "Spark home not found; set it explicitly or use the SPARK_HOME environment variable.");
+ return path;
+ }
+
+ /**
+ * Loads the configuration file for the application, if it exists. This is either the
+ * user-specified properties file, or the spark-defaults.conf file under the Spark configuration
+ * directory.
+ */
+ Properties loadPropertiesFile() throws IOException {
+ Properties props = new Properties();
+ File propsFile;
+ if (propertiesFile != null) {
+ propsFile = new File(propertiesFile);
+ checkArgument(propsFile.isFile(), "Invalid properties file '%s'.", propertiesFile);
+ } else {
+ propsFile = new File(getConfDir(), DEFAULT_PROPERTIES_FILE);
+ }
+
+ if (propsFile.isFile()) {
+ FileInputStream fd = null;
+ try {
+ fd = new FileInputStream(propsFile);
+ props.load(new InputStreamReader(fd, "UTF-8"));
+ } finally {
+ if (fd != null) {
+ try {
+ fd.close();
+ } catch (IOException e) {
+ // Ignore.
+ }
+ }
+ }
+ }
+
+ return props;
+ }
+
+ String getenv(String key) {
+ return firstNonEmpty(childEnv.get(key), System.getenv(key));
+ }
+
+ private String findAssembly(String scalaVersion) {
+ String sparkHome = getSparkHome();
+ File libdir;
+ if (new File(sparkHome, "RELEASE").isFile()) {
+ libdir = new File(sparkHome, "lib");
+ checkState(libdir.isDirectory(), "Library directory '%s' does not exist.",
+ libdir.getAbsolutePath());
+ } else {
+ libdir = new File(sparkHome, String.format("assembly/target/scala-%s", scalaVersion));
+ }
+
+ final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar");
+ FileFilter filter = new FileFilter() {
+ @Override
+ public boolean accept(File file) {
+ return file.isFile() && re.matcher(file.getName()).matches();
+ }
+ };
+ File[] assemblies = libdir.listFiles(filter);
+ checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir);
+ checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir);
+ return assemblies[0].getAbsolutePath();
+ }
+
+ private String getConfDir() {
+ String confDir = getenv("SPARK_CONF_DIR");
+ return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf");
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
new file mode 100644
index 0000000000000..9b04732afee14
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Helper methods for command builders.
+ */
+class CommandBuilderUtils {
+
+ static final String DEFAULT_MEM = "512m";
+ static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf";
+ static final String ENV_SPARK_HOME = "SPARK_HOME";
+
+ /** Returns whether the given string is null or empty. */
+ static boolean isEmpty(String s) {
+ return s == null || s.isEmpty();
+ }
+
+ /** Joins a list of strings using the given separator. */
+ static String join(String sep, String... elements) {
+ StringBuilder sb = new StringBuilder();
+ for (String e : elements) {
+ if (e != null) {
+ if (sb.length() > 0) {
+ sb.append(sep);
+ }
+ sb.append(e);
+ }
+ }
+ return sb.toString();
+ }
+
+ /** Joins a list of strings using the given separator. */
+ static String join(String sep, Iterable elements) {
+ StringBuilder sb = new StringBuilder();
+ for (String e : elements) {
+ if (e != null) {
+ if (sb.length() > 0) {
+ sb.append(sep);
+ }
+ sb.append(e);
+ }
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Returns the first non-empty value mapped to the given key in the given maps, or null otherwise.
+ */
+ static String firstNonEmptyValue(String key, Map, ?>... maps) {
+ for (Map, ?> map : maps) {
+ String value = (String) map.get(key);
+ if (!isEmpty(value)) {
+ return value;
+ }
+ }
+ return null;
+ }
+
+ /** Returns the first non-empty, non-null string in the given list, or null otherwise. */
+ static String firstNonEmpty(String... candidates) {
+ for (String s : candidates) {
+ if (!isEmpty(s)) {
+ return s;
+ }
+ }
+ return null;
+ }
+
+ /** Returns the name of the env variable that holds the native library path. */
+ static String getLibPathEnvName() {
+ if (isWindows()) {
+ return "PATH";
+ }
+
+ String os = System.getProperty("os.name");
+ if (os.startsWith("Mac OS X")) {
+ return "DYLD_LIBRARY_PATH";
+ } else {
+ return "LD_LIBRARY_PATH";
+ }
+ }
+
+ /** Returns whether the OS is Windows. */
+ static boolean isWindows() {
+ String os = System.getProperty("os.name");
+ return os.startsWith("Windows");
+ }
+
+ /**
+ * Updates the user environment, appending the given pathList to the existing value of the given
+ * environment variable (or setting it if it hasn't yet been set).
+ */
+ static void mergeEnvPathList(Map userEnv, String envKey, String pathList) {
+ if (!isEmpty(pathList)) {
+ String current = firstNonEmpty(userEnv.get(envKey), System.getenv(envKey));
+ userEnv.put(envKey, join(File.pathSeparator, current, pathList));
+ }
+ }
+
+ /**
+ * Parse a string as if it were a list of arguments, following bash semantics.
+ * For example:
+ *
+ * Input: "\"ab cd\" efgh 'i \" j'"
+ * Output: [ "ab cd", "efgh", "i \" j" ]
+ */
+ static List parseOptionString(String s) {
+ List opts = new ArrayList();
+ StringBuilder opt = new StringBuilder();
+ boolean inOpt = false;
+ boolean inSingleQuote = false;
+ boolean inDoubleQuote = false;
+ boolean escapeNext = false;
+
+ // This is needed to detect when a quoted empty string is used as an argument ("" or '').
+ boolean hasData = false;
+
+ for (int i = 0; i < s.length(); i++) {
+ int c = s.codePointAt(i);
+ if (escapeNext) {
+ opt.appendCodePoint(c);
+ escapeNext = false;
+ } else if (inOpt) {
+ switch (c) {
+ case '\\':
+ if (inSingleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ escapeNext = true;
+ }
+ break;
+ case '\'':
+ if (inDoubleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ inSingleQuote = !inSingleQuote;
+ }
+ break;
+ case '"':
+ if (inSingleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ inDoubleQuote = !inDoubleQuote;
+ }
+ break;
+ default:
+ if (!Character.isWhitespace(c) || inSingleQuote || inDoubleQuote) {
+ opt.appendCodePoint(c);
+ } else {
+ opts.add(opt.toString());
+ opt.setLength(0);
+ inOpt = false;
+ hasData = false;
+ }
+ }
+ } else {
+ switch (c) {
+ case '\'':
+ inSingleQuote = true;
+ inOpt = true;
+ hasData = true;
+ break;
+ case '"':
+ inDoubleQuote = true;
+ inOpt = true;
+ hasData = true;
+ break;
+ case '\\':
+ escapeNext = true;
+ inOpt = true;
+ hasData = true;
+ break;
+ default:
+ if (!Character.isWhitespace(c)) {
+ inOpt = true;
+ hasData = true;
+ opt.appendCodePoint(c);
+ }
+ }
+ }
+ }
+
+ checkArgument(!inSingleQuote && !inDoubleQuote && !escapeNext, "Invalid option string: %s", s);
+ if (hasData) {
+ opts.add(opt.toString());
+ }
+ return opts;
+ }
+
+ /** Throws IllegalArgumentException if the given object is null. */
+ static void checkNotNull(Object o, String arg) {
+ if (o == null) {
+ throw new IllegalArgumentException(String.format("'%s' must not be null.", arg));
+ }
+ }
+
+ /** Throws IllegalArgumentException with the given message if the check is false. */
+ static void checkArgument(boolean check, String msg, Object... args) {
+ if (!check) {
+ throw new IllegalArgumentException(String.format(msg, args));
+ }
+ }
+
+ /** Throws IllegalStateException with the given message if the check is false. */
+ static void checkState(boolean check, String msg, Object... args) {
+ if (!check) {
+ throw new IllegalStateException(String.format(msg, args));
+ }
+ }
+
+ /**
+ * Quote a command argument for a command to be run by a Windows batch script, if the argument
+ * needs quoting. Arguments only seem to need quotes in batch scripts if they have certain
+ * special characters, some of which need extra (and different) escaping.
+ *
+ * For example:
+ * original single argument: ab="cde fgh"
+ * quoted: "ab^=""cde fgh"""
+ */
+ static String quoteForBatchScript(String arg) {
+
+ boolean needsQuotes = false;
+ for (int i = 0; i < arg.length(); i++) {
+ int c = arg.codePointAt(i);
+ if (Character.isWhitespace(c) || c == '"' || c == '=') {
+ needsQuotes = true;
+ break;
+ }
+ }
+ if (!needsQuotes) {
+ return arg;
+ }
+ StringBuilder quoted = new StringBuilder();
+ quoted.append("\"");
+ for (int i = 0; i < arg.length(); i++) {
+ int cp = arg.codePointAt(i);
+ switch (cp) {
+ case '"':
+ quoted.append('"');
+ break;
+
+ case '=':
+ quoted.append('^');
+ break;
+
+ default:
+ break;
+ }
+ quoted.appendCodePoint(cp);
+ }
+ quoted.append("\"");
+ return quoted.toString();
+ }
+
+ /**
+ * Quotes a string so that it can be used in a command string and be parsed back into a single
+ * argument by python's "shlex.split()" function.
+ *
+ * Basically, just add simple escapes. E.g.:
+ * original single argument : ab "cd" ef
+ * after: "ab \"cd\" ef"
+ */
+ static String quoteForPython(String s) {
+ StringBuilder quoted = new StringBuilder().append('"');
+ for (int i = 0; i < s.length(); i++) {
+ int cp = s.codePointAt(i);
+ if (cp == '"' || cp == '\\') {
+ quoted.appendCodePoint('\\');
+ }
+ quoted.appendCodePoint(cp);
+ }
+ return quoted.append('"').toString();
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/Main.java b/launcher/src/main/java/org/apache/spark/launcher/Main.java
new file mode 100644
index 0000000000000..206acfb514d86
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/Main.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Command line interface for the Spark launcher. Used internally by Spark scripts.
+ */
+class Main {
+
+ /**
+ * Usage: Main [class] [class args]
+ *
+ * This CLI works in two different modes:
+ *
+ * - "spark-submit": if class is "org.apache.spark.deploy.SparkSubmit", the
+ * {@link SparkLauncher} class is used to launch a Spark application.
+ * - "spark-class": if another class is provided, an internal Spark class is run.
+ *
+ *
+ * This class works in tandem with the "bin/spark-class" script on Unix-like systems, and
+ * "bin/spark-class2.cmd" batch script on Windows to execute the final command.
+ *
+ * On Unix-like systems, the output is a list of command arguments, separated by the NULL
+ * character. On Windows, the output is a command line suitable for direct execution from the
+ * script.
+ */
+ public static void main(String[] argsArray) throws Exception {
+ checkArgument(argsArray.length > 0, "Not enough arguments: missing class name.");
+
+ List args = new ArrayList(Arrays.asList(argsArray));
+ String className = args.remove(0);
+
+ boolean printLaunchCommand;
+ boolean printUsage;
+ AbstractCommandBuilder builder;
+ try {
+ if (className.equals("org.apache.spark.deploy.SparkSubmit")) {
+ builder = new SparkSubmitCommandBuilder(args);
+ } else {
+ builder = new SparkClassCommandBuilder(className, args);
+ }
+ printLaunchCommand = !isEmpty(System.getenv("SPARK_PRINT_LAUNCH_COMMAND"));
+ printUsage = false;
+ } catch (IllegalArgumentException e) {
+ builder = new UsageCommandBuilder(e.getMessage());
+ printLaunchCommand = false;
+ printUsage = true;
+ }
+
+ Map env = new HashMap();
+ List cmd = builder.buildCommand(env);
+ if (printLaunchCommand) {
+ System.err.println("Spark Command: " + join(" ", cmd));
+ System.err.println("========================================");
+ }
+
+ if (isWindows()) {
+ // When printing the usage message, we can't use "cmd /v" since that prevents the env
+ // variable from being seen in the caller script. So do not call prepareWindowsCommand().
+ if (printUsage) {
+ System.out.println(join(" ", cmd));
+ } else {
+ System.out.println(prepareWindowsCommand(cmd, env));
+ }
+ } else {
+ // In bash, use NULL as the arg separator since it cannot be used in an argument.
+ List bashCmd = prepareBashCommand(cmd, env);
+ for (String c : bashCmd) {
+ System.out.print(c);
+ System.out.print('\0');
+ }
+ }
+ }
+
+ /**
+ * Prepare a command line for execution from a Windows batch script.
+ *
+ * The method quotes all arguments so that spaces are handled as expected. Quotes within arguments
+ * are "double quoted" (which is batch for escaping a quote). This page has more details about
+ * quoting and other batch script fun stuff: http://ss64.com/nt/syntax-esc.html
+ *
+ * The command is executed using "cmd /c" and formatted in single line, since that's the
+ * easiest way to consume this from a batch script (see spark-class2.cmd).
+ */
+ private static String prepareWindowsCommand(List cmd, Map childEnv) {
+ StringBuilder cmdline = new StringBuilder("cmd /c \"");
+ for (Map.Entry e : childEnv.entrySet()) {
+ cmdline.append(String.format("set %s=%s", e.getKey(), e.getValue()));
+ cmdline.append(" && ");
+ }
+ for (String arg : cmd) {
+ cmdline.append(quoteForBatchScript(arg));
+ cmdline.append(" ");
+ }
+ cmdline.append("\"");
+ return cmdline.toString();
+ }
+
+ /**
+ * Prepare the command for execution from a bash script. The final command will have commands to
+ * set up any needed environment variables needed by the child process.
+ */
+ private static List prepareBashCommand(List cmd, Map childEnv) {
+ if (childEnv.isEmpty()) {
+ return cmd;
+ }
+
+ List newCmd = new ArrayList();
+ newCmd.add("env");
+
+ for (Map.Entry e : childEnv.entrySet()) {
+ newCmd.add(String.format("%s=%s", e.getKey(), e.getValue()));
+ }
+ newCmd.addAll(cmd);
+ return newCmd;
+ }
+
+ /**
+ * Internal builder used when command line parsing fails. This will behave differently depending
+ * on the platform:
+ *
+ * - On Unix-like systems, it will print a call to the "usage" function with two arguments: the
+ * the error string, and the exit code to use. The function is expected to print the command's
+ * usage and exit with the provided exit code. The script should use "export -f usage" after
+ * declaring a function called "usage", so that the function is available to downstream scripts.
+ *
+ * - On Windows it will set the variable "SPARK_LAUNCHER_USAGE_ERROR" to the usage error message.
+ * The batch script should check for this variable and print its usage, since batch scripts
+ * don't really support the "export -f" functionality used in bash.
+ */
+ private static class UsageCommandBuilder extends AbstractCommandBuilder {
+
+ private final String message;
+
+ UsageCommandBuilder(String message) {
+ this.message = message;
+ }
+
+ @Override
+ public List buildCommand(Map env) {
+ if (isWindows()) {
+ return Arrays.asList("set", "SPARK_LAUNCHER_USAGE_ERROR=" + message);
+ } else {
+ return Arrays.asList("usage", message, "1");
+ }
+ }
+
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
new file mode 100644
index 0000000000000..e601a0a19f368
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkClassCommandBuilder.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Command builder for internal Spark classes.
+ *
+ * This class handles building the command to launch all internal Spark classes except for
+ * SparkSubmit (which is handled by {@link SparkSubmitCommandBuilder} class.
+ */
+class SparkClassCommandBuilder extends AbstractCommandBuilder {
+
+ private final String className;
+ private final List classArgs;
+
+ SparkClassCommandBuilder(String className, List classArgs) {
+ this.className = className;
+ this.classArgs = classArgs;
+ }
+
+ @Override
+ public List buildCommand(Map env) throws IOException {
+ List javaOptsKeys = new ArrayList();
+ String memKey = null;
+ String extraClassPath = null;
+
+ // Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) +
+ // SPARK_DAEMON_MEMORY.
+ if (className.equals("org.apache.spark.deploy.master.Master")) {
+ javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_MASTER_OPTS");
+ memKey = "SPARK_DAEMON_MEMORY";
+ } else if (className.equals("org.apache.spark.deploy.worker.Worker")) {
+ javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_WORKER_OPTS");
+ memKey = "SPARK_DAEMON_MEMORY";
+ } else if (className.equals("org.apache.spark.deploy.history.HistoryServer")) {
+ javaOptsKeys.add("SPARK_DAEMON_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_HISTORY_OPTS");
+ memKey = "SPARK_DAEMON_MEMORY";
+ } else if (className.equals("org.apache.spark.executor.CoarseGrainedExecutorBackend")) {
+ javaOptsKeys.add("SPARK_JAVA_OPTS");
+ javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
+ memKey = "SPARK_EXECUTOR_MEMORY";
+ } else if (className.equals("org.apache.spark.executor.MesosExecutorBackend")) {
+ javaOptsKeys.add("SPARK_EXECUTOR_OPTS");
+ memKey = "SPARK_EXECUTOR_MEMORY";
+ } else if (className.startsWith("org.apache.spark.tools.")) {
+ String sparkHome = getSparkHome();
+ File toolsDir = new File(join(File.separator, sparkHome, "tools", "target",
+ "scala-" + getScalaVersion()));
+ checkState(toolsDir.isDirectory(), "Cannot find tools build directory.");
+
+ Pattern re = Pattern.compile("spark-tools_.*\\.jar");
+ for (File f : toolsDir.listFiles()) {
+ if (re.matcher(f.getName()).matches()) {
+ extraClassPath = f.getAbsolutePath();
+ break;
+ }
+ }
+
+ checkState(extraClassPath != null,
+ "Failed to find Spark Tools Jar in %s.\n" +
+ "You need to run \"build/sbt tools/package\" before running %s.",
+ toolsDir.getAbsolutePath(), className);
+
+ javaOptsKeys.add("SPARK_JAVA_OPTS");
+ }
+
+ List cmd = buildJavaCommand(extraClassPath);
+ for (String key : javaOptsKeys) {
+ addOptionString(cmd, System.getenv(key));
+ }
+
+ String mem = firstNonEmpty(memKey != null ? System.getenv(memKey) : null, DEFAULT_MEM);
+ cmd.add("-Xms" + mem);
+ cmd.add("-Xmx" + mem);
+ addPermGenSizeOpt(cmd);
+ cmd.add(className);
+ cmd.addAll(classArgs);
+ return cmd;
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
new file mode 100644
index 0000000000000..b566507ee6061
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java
@@ -0,0 +1,279 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Launcher for Spark applications.
+ *
+ * Use this class to start Spark applications programmatically. The class uses a builder pattern
+ * to allow clients to configure the Spark application and launch it as a child process.
+ */
+public class SparkLauncher {
+
+ /** The Spark master. */
+ public static final String SPARK_MASTER = "spark.master";
+
+ /** Configuration key for the driver memory. */
+ public static final String DRIVER_MEMORY = "spark.driver.memory";
+ /** Configuration key for the driver class path. */
+ public static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath";
+ /** Configuration key for the driver VM options. */
+ public static final String DRIVER_EXTRA_JAVA_OPTIONS = "spark.driver.extraJavaOptions";
+ /** Configuration key for the driver native library path. */
+ public static final String DRIVER_EXTRA_LIBRARY_PATH = "spark.driver.extraLibraryPath";
+
+ /** Configuration key for the executor memory. */
+ public static final String EXECUTOR_MEMORY = "spark.executor.memory";
+ /** Configuration key for the executor class path. */
+ public static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath";
+ /** Configuration key for the executor VM options. */
+ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions";
+ /** Configuration key for the executor native library path. */
+ public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions";
+ /** Configuration key for the number of executor CPU cores. */
+ public static final String EXECUTOR_CORES = "spark.executor.cores";
+
+ private final SparkSubmitCommandBuilder builder;
+
+ public SparkLauncher() {
+ this(null);
+ }
+
+ /**
+ * Creates a launcher that will set the given environment variables in the child.
+ *
+ * @param env Environment variables to set.
+ */
+ public SparkLauncher(Map env) {
+ this.builder = new SparkSubmitCommandBuilder();
+ if (env != null) {
+ this.builder.childEnv.putAll(env);
+ }
+ }
+
+ /**
+ * Set a custom JAVA_HOME for launching the Spark application.
+ *
+ * @param javaHome Path to the JAVA_HOME to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setJavaHome(String javaHome) {
+ checkNotNull(javaHome, "javaHome");
+ builder.javaHome = javaHome;
+ return this;
+ }
+
+ /**
+ * Set a custom Spark installation location for the application.
+ *
+ * @param sparkHome Path to the Spark installation to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setSparkHome(String sparkHome) {
+ checkNotNull(sparkHome, "sparkHome");
+ builder.childEnv.put(ENV_SPARK_HOME, sparkHome);
+ return this;
+ }
+
+ /**
+ * Set a custom properties file with Spark configuration for the application.
+ *
+ * @param path Path to custom properties file to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setPropertiesFile(String path) {
+ checkNotNull(path, "path");
+ builder.propertiesFile = path;
+ return this;
+ }
+
+ /**
+ * Set a single configuration value for the application.
+ *
+ * @param key Configuration key.
+ * @param value The value to use.
+ * @return This launcher.
+ */
+ public SparkLauncher setConf(String key, String value) {
+ checkNotNull(key, "key");
+ checkNotNull(value, "value");
+ checkArgument(key.startsWith("spark."), "'key' must start with 'spark.'");
+ builder.conf.put(key, value);
+ return this;
+ }
+
+ /**
+ * Set the application name.
+ *
+ * @param appName Application name.
+ * @return This launcher.
+ */
+ public SparkLauncher setAppName(String appName) {
+ checkNotNull(appName, "appName");
+ builder.appName = appName;
+ return this;
+ }
+
+ /**
+ * Set the Spark master for the application.
+ *
+ * @param master Spark master.
+ * @return This launcher.
+ */
+ public SparkLauncher setMaster(String master) {
+ checkNotNull(master, "master");
+ builder.master = master;
+ return this;
+ }
+
+ /**
+ * Set the deploy mode for the application.
+ *
+ * @param mode Deploy mode.
+ * @return This launcher.
+ */
+ public SparkLauncher setDeployMode(String mode) {
+ checkNotNull(mode, "mode");
+ builder.deployMode = mode;
+ return this;
+ }
+
+ /**
+ * Set the main application resource. This should be the location of a jar file for Scala/Java
+ * applications, or a python script for PySpark applications.
+ *
+ * @param resource Path to the main application resource.
+ * @return This launcher.
+ */
+ public SparkLauncher setAppResource(String resource) {
+ checkNotNull(resource, "resource");
+ builder.appResource = resource;
+ return this;
+ }
+
+ /**
+ * Sets the application class name for Java/Scala applications.
+ *
+ * @param mainClass Application's main class.
+ * @return This launcher.
+ */
+ public SparkLauncher setMainClass(String mainClass) {
+ checkNotNull(mainClass, "mainClass");
+ builder.mainClass = mainClass;
+ return this;
+ }
+
+ /**
+ * Adds command line arguments for the application.
+ *
+ * @param args Arguments to pass to the application's main class.
+ * @return This launcher.
+ */
+ public SparkLauncher addAppArgs(String... args) {
+ for (String arg : args) {
+ checkNotNull(arg, "arg");
+ builder.appArgs.add(arg);
+ }
+ return this;
+ }
+
+ /**
+ * Adds a jar file to be submitted with the application.
+ *
+ * @param jar Path to the jar file.
+ * @return This launcher.
+ */
+ public SparkLauncher addJar(String jar) {
+ checkNotNull(jar, "jar");
+ builder.jars.add(jar);
+ return this;
+ }
+
+ /**
+ * Adds a file to be submitted with the application.
+ *
+ * @param file Path to the file.
+ * @return This launcher.
+ */
+ public SparkLauncher addFile(String file) {
+ checkNotNull(file, "file");
+ builder.files.add(file);
+ return this;
+ }
+
+ /**
+ * Adds a python file / zip / egg to be submitted with the application.
+ *
+ * @param file Path to the file.
+ * @return This launcher.
+ */
+ public SparkLauncher addPyFile(String file) {
+ checkNotNull(file, "file");
+ builder.pyFiles.add(file);
+ return this;
+ }
+
+ /**
+ * Enables verbose reporting for SparkSubmit.
+ *
+ * @param verbose Whether to enable verbose output.
+ * @return This launcher.
+ */
+ public SparkLauncher setVerbose(boolean verbose) {
+ builder.verbose = verbose;
+ return this;
+ }
+
+ /**
+ * Launches a sub-process that will start the configured Spark application.
+ *
+ * @return A process handle for the Spark app.
+ */
+ public Process launch() throws IOException {
+ List cmd = new ArrayList();
+ String script = isWindows() ? "spark-submit.cmd" : "spark-submit";
+ cmd.add(join(File.separator, builder.getSparkHome(), "bin", script));
+ cmd.addAll(builder.buildSparkSubmitArgs());
+
+ // Since the child process is a batch script, let's quote things so that special characters are
+ // preserved, otherwise the batch interpreter will mess up the arguments. Batch scripts are
+ // weird.
+ if (isWindows()) {
+ List winCmd = new ArrayList();
+ for (String arg : cmd) {
+ winCmd.add(quoteForBatchScript(arg));
+ }
+ cmd = winCmd;
+ }
+
+ ProcessBuilder pb = new ProcessBuilder(cmd.toArray(new String[cmd.size()]));
+ for (Map.Entry e : builder.childEnv.entrySet()) {
+ pb.environment().put(e.getKey(), e.getValue());
+ }
+ return pb.start();
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
new file mode 100644
index 0000000000000..6ffdff63d3c78
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -0,0 +1,327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+/**
+ * Special command builder for handling a CLI invocation of SparkSubmit.
+ *
+ * This builder adds command line parsing compatible with SparkSubmit. It handles setting
+ * driver-side options and special parsing behavior needed for the special-casing certain internal
+ * Spark applications.
+ *
+ * This class has also some special features to aid launching pyspark.
+ */
+class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
+
+ /**
+ * Name of the app resource used to identify the PySpark shell. The command line parser expects
+ * the resource name to be the very first argument to spark-submit in this case.
+ *
+ * NOTE: this cannot be "pyspark-shell" since that identifies the PySpark shell to SparkSubmit
+ * (see java_gateway.py), and can cause this code to enter into an infinite loop.
+ */
+ static final String PYSPARK_SHELL = "pyspark-shell-main";
+
+ /**
+ * This is the actual resource name that identifies the PySpark shell to SparkSubmit.
+ */
+ static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell";
+
+ /**
+ * This map must match the class names for available special classes, since this modifies the way
+ * command line parsing works. This maps the class name to the resource to use when calling
+ * spark-submit.
+ */
+ private static final Map specialClasses = new HashMap();
+ static {
+ specialClasses.put("org.apache.spark.repl.Main", "spark-shell");
+ specialClasses.put("org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver",
+ "spark-internal");
+ specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2",
+ "spark-internal");
+ }
+
+ private final List sparkArgs;
+
+ /**
+ * Controls whether mixing spark-submit arguments with app arguments is allowed. This is needed
+ * to parse the command lines for things like bin/spark-shell, which allows users to mix and
+ * match arguments (e.g. "bin/spark-shell SparkShellArg --master foo").
+ */
+ private boolean allowsMixedArguments;
+
+ SparkSubmitCommandBuilder() {
+ this.sparkArgs = new ArrayList();
+ }
+
+ SparkSubmitCommandBuilder(List args) {
+ this();
+ List submitArgs = args;
+ if (args.size() > 0 && args.get(0).equals(PYSPARK_SHELL)) {
+ this.allowsMixedArguments = true;
+ appResource = PYSPARK_SHELL_RESOURCE;
+ submitArgs = args.subList(1, args.size());
+ } else {
+ this.allowsMixedArguments = false;
+ }
+
+ new OptionParser().parse(submitArgs);
+ }
+
+ @Override
+ public List buildCommand(Map env) throws IOException {
+ if (PYSPARK_SHELL_RESOURCE.equals(appResource)) {
+ return buildPySparkShellCommand(env);
+ } else {
+ return buildSparkSubmitCommand(env);
+ }
+ }
+
+ List buildSparkSubmitArgs() {
+ List args = new ArrayList();
+ SparkSubmitOptionParser parser = new SparkSubmitOptionParser();
+
+ if (verbose) {
+ args.add(parser.VERBOSE);
+ }
+
+ if (master != null) {
+ args.add(parser.MASTER);
+ args.add(master);
+ }
+
+ if (deployMode != null) {
+ args.add(parser.DEPLOY_MODE);
+ args.add(deployMode);
+ }
+
+ if (appName != null) {
+ args.add(parser.NAME);
+ args.add(appName);
+ }
+
+ for (Map.Entry e : conf.entrySet()) {
+ args.add(parser.CONF);
+ args.add(String.format("%s=%s", e.getKey(), e.getValue()));
+ }
+
+ if (propertiesFile != null) {
+ args.add(parser.PROPERTIES_FILE);
+ args.add(propertiesFile);
+ }
+
+ if (!jars.isEmpty()) {
+ args.add(parser.JARS);
+ args.add(join(",", jars));
+ }
+
+ if (!files.isEmpty()) {
+ args.add(parser.FILES);
+ args.add(join(",", files));
+ }
+
+ if (!pyFiles.isEmpty()) {
+ args.add(parser.PY_FILES);
+ args.add(join(",", pyFiles));
+ }
+
+ if (mainClass != null) {
+ args.add(parser.CLASS);
+ args.add(mainClass);
+ }
+
+ args.addAll(sparkArgs);
+ if (appResource != null) {
+ args.add(appResource);
+ }
+ args.addAll(appArgs);
+
+ return args;
+ }
+
+ private List buildSparkSubmitCommand(Map env) throws IOException {
+ // Load the properties file and check whether spark-submit will be running the app's driver
+ // or just launching a cluster app. When running the driver, the JVM's argument will be
+ // modified to cover the driver's configuration.
+ Properties props = loadPropertiesFile();
+ boolean isClientMode = isClientMode(props);
+ String extraClassPath = isClientMode ?
+ firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_CLASSPATH, conf, props) : null;
+
+ List cmd = buildJavaCommand(extraClassPath);
+ addOptionString(cmd, System.getenv("SPARK_SUBMIT_OPTS"));
+ addOptionString(cmd, System.getenv("SPARK_JAVA_OPTS"));
+
+ if (isClientMode) {
+ // Figuring out where the memory value come from is a little tricky due to precedence.
+ // Precedence is observed in the following order:
+ // - explicit configuration (setConf()), which also covers --driver-memory cli argument.
+ // - properties file.
+ // - SPARK_DRIVER_MEMORY env variable
+ // - SPARK_MEM env variable
+ // - default value (512m)
+ String memory = firstNonEmpty(firstNonEmptyValue(SparkLauncher.DRIVER_MEMORY, conf, props),
+ System.getenv("SPARK_DRIVER_MEMORY"), System.getenv("SPARK_MEM"), DEFAULT_MEM);
+ cmd.add("-Xms" + memory);
+ cmd.add("-Xmx" + memory);
+ addOptionString(cmd, firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, conf, props));
+ mergeEnvPathList(env, getLibPathEnvName(),
+ firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props));
+ }
+
+ addPermGenSizeOpt(cmd);
+ cmd.add("org.apache.spark.deploy.SparkSubmit");
+ cmd.addAll(buildSparkSubmitArgs());
+ return cmd;
+ }
+
+ private List buildPySparkShellCommand(Map env) throws IOException {
+ // For backwards compatibility, if a script is specified in
+ // the pyspark command line, then run it using spark-submit.
+ if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) {
+ System.err.println(
+ "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" +
+ "Use ./bin/spark-submit ");
+ appResource = appArgs.get(0);
+ appArgs.remove(0);
+ return buildCommand(env);
+ }
+
+ // When launching the pyspark shell, the spark-submit arguments should be stored in the
+ // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable
+ // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS.
+ checkArgument(appArgs.isEmpty(), "pyspark does not support any application options.");
+
+ Properties props = loadPropertiesFile();
+ mergeEnvPathList(env, getLibPathEnvName(),
+ firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props));
+
+ // Store spark-submit arguments in an environment variable, since there's no way to pass
+ // them to shell.py on the comand line.
+ StringBuilder submitArgs = new StringBuilder();
+ for (String arg : buildSparkSubmitArgs()) {
+ if (submitArgs.length() > 0) {
+ submitArgs.append(" ");
+ }
+ submitArgs.append(quoteForPython(arg));
+ }
+ env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString());
+
+ List pyargs = new ArrayList();
+ pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python"));
+ String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS");
+ if (!isEmpty(pyOpts)) {
+ pyargs.addAll(parseOptionString(pyOpts));
+ }
+
+ return pyargs;
+ }
+
+ private boolean isClientMode(Properties userProps) {
+ String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER));
+ // Default master is "local[*]", so assume client mode in that case.
+ return userMaster == null ||
+ "client".equals(deployMode) ||
+ (!userMaster.equals("yarn-cluster") && deployMode == null);
+ }
+
+ private class OptionParser extends SparkSubmitOptionParser {
+
+ private final List driverJvmKeys = Arrays.asList(
+ SparkLauncher.DRIVER_EXTRA_CLASSPATH,
+ SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS,
+ SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH,
+ SparkLauncher.DRIVER_MEMORY);
+
+ @Override
+ protected boolean handle(String opt, String value) {
+ if (opt.equals(MASTER)) {
+ master = value;
+ } else if (opt.equals(DEPLOY_MODE)) {
+ deployMode = value;
+ } else if (opt.equals(PROPERTIES_FILE)) {
+ propertiesFile = value;
+ } else if (opt.equals(DRIVER_MEMORY)) {
+ conf.put(SparkLauncher.DRIVER_MEMORY, value);
+ } else if (opt.equals(DRIVER_JAVA_OPTIONS)) {
+ conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, value);
+ } else if (opt.equals(DRIVER_LIBRARY_PATH)) {
+ conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, value);
+ } else if (opt.equals(DRIVER_CLASS_PATH)) {
+ conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, value);
+ } else if (opt.equals(CONF)) {
+ String[] setConf = value.split("=", 2);
+ checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value);
+ if (driverJvmKeys.contains(setConf[0])) {
+ conf.put(setConf[0], setConf[1]);
+ }
+ } else if (opt.equals(CLASS)) {
+ // The special classes require some special command line handling, since they allow
+ // mixing spark-submit arguments with arguments that should be propagated to the shell
+ // itself. Note that for this to work, the "--class" argument must come before any
+ // non-spark-submit arguments.
+ mainClass = value;
+ if (specialClasses.containsKey(value)) {
+ allowsMixedArguments = true;
+ appResource = specialClasses.get(value);
+ }
+ } else {
+ sparkArgs.add(opt);
+ if (value != null) {
+ sparkArgs.add(value);
+ }
+ }
+ return true;
+ }
+
+ @Override
+ protected boolean handleUnknown(String opt) {
+ // When mixing arguments, add unrecognized parameters directly to the user arguments list. In
+ // normal mode, any unrecognized parameter triggers the end of command line parsing, and the
+ // parameter itself will be interpreted by SparkSubmit as the application resource. The
+ // remaining params will be appended to the list of SparkSubmit arguments.
+ if (allowsMixedArguments) {
+ appArgs.add(opt);
+ return true;
+ } else {
+ sparkArgs.add(opt);
+ return false;
+ }
+ }
+
+ @Override
+ protected void handleExtraArgs(List extra) {
+ for (String arg : extra) {
+ sparkArgs.add(arg);
+ }
+ }
+
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java
new file mode 100644
index 0000000000000..8526d2e7cfa3f
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java
@@ -0,0 +1,224 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Parser for spark-submit command line options.
+ *
+ * This class encapsulates the parsing code for spark-submit command line options, so that there
+ * is a single list of options that needs to be maintained (well, sort of, but it makes it harder
+ * to break things).
+ */
+class SparkSubmitOptionParser {
+
+ // The following constants define the "main" name for the available options. They're defined
+ // to avoid copy & paste of the raw strings where they're needed.
+ //
+ // The fields are not static so that they're exposed to Scala code that uses this class. See
+ // SparkSubmitArguments.scala. That is also why this class is not abstract - to allow code to
+ // easily use these constants without having to create dummy implementations of this class.
+ protected final String CLASS = "--class";
+ protected final String CONF = "--conf";
+ protected final String DEPLOY_MODE = "--deploy-mode";
+ protected final String DRIVER_CLASS_PATH = "--driver-class-path";
+ protected final String DRIVER_CORES = "--driver-cores";
+ protected final String DRIVER_JAVA_OPTIONS = "--driver-java-options";
+ protected final String DRIVER_LIBRARY_PATH = "--driver-library-path";
+ protected final String DRIVER_MEMORY = "--driver-memory";
+ protected final String EXECUTOR_MEMORY = "--executor-memory";
+ protected final String FILES = "--files";
+ protected final String JARS = "--jars";
+ protected final String KILL_SUBMISSION = "--kill";
+ protected final String MASTER = "--master";
+ protected final String NAME = "--name";
+ protected final String PACKAGES = "--packages";
+ protected final String PROPERTIES_FILE = "--properties-file";
+ protected final String PROXY_USER = "--proxy-user";
+ protected final String PY_FILES = "--py-files";
+ protected final String REPOSITORIES = "--repositories";
+ protected final String STATUS = "--status";
+ protected final String TOTAL_EXECUTOR_CORES = "--total-executor-cores";
+
+ // Options that do not take arguments.
+ protected final String HELP = "--help";
+ protected final String SUPERVISE = "--supervise";
+ protected final String VERBOSE = "--verbose";
+ protected final String VERSION = "--version";
+
+ // Standalone-only options.
+
+ // YARN-only options.
+ protected final String ARCHIVES = "--archives";
+ protected final String EXECUTOR_CORES = "--executor-cores";
+ protected final String QUEUE = "--queue";
+ protected final String NUM_EXECUTORS = "--num-executors";
+
+ /**
+ * This is the canonical list of spark-submit options. Each entry in the array contains the
+ * different aliases for the same option; the first element of each entry is the "official"
+ * name of the option, passed to {@link #handle(String, String)}.
+ *
+ * Options not listed here nor in the "switch" list below will result in a call to
+ * {@link $#handleUnknown(String)}.
+ *
+ * These two arrays are visible for tests.
+ */
+ final String[][] opts = {
+ { ARCHIVES },
+ { CLASS },
+ { CONF, "-c" },
+ { DEPLOY_MODE },
+ { DRIVER_CLASS_PATH },
+ { DRIVER_CORES },
+ { DRIVER_JAVA_OPTIONS },
+ { DRIVER_LIBRARY_PATH },
+ { DRIVER_MEMORY },
+ { EXECUTOR_CORES },
+ { EXECUTOR_MEMORY },
+ { FILES },
+ { JARS },
+ { KILL_SUBMISSION },
+ { MASTER },
+ { NAME },
+ { NUM_EXECUTORS },
+ { PACKAGES },
+ { PROPERTIES_FILE },
+ { PROXY_USER },
+ { PY_FILES },
+ { QUEUE },
+ { REPOSITORIES },
+ { STATUS },
+ { TOTAL_EXECUTOR_CORES },
+ };
+
+ /**
+ * List of switches (command line options that do not take parameters) recognized by spark-submit.
+ */
+ final String[][] switches = {
+ { HELP, "-h" },
+ { SUPERVISE },
+ { VERBOSE, "-v" },
+ { VERSION },
+ };
+
+ /**
+ * Parse a list of spark-submit command line options.
+ *
+ * See SparkSubmitArguments.scala for a more formal description of available options.
+ *
+ * @throws IllegalArgumentException If an error is found during parsing.
+ */
+ protected final void parse(List args) {
+ Pattern eqSeparatedOpt = Pattern.compile("(--[^=]+)=(.+)");
+
+ int idx = 0;
+ for (idx = 0; idx < args.size(); idx++) {
+ String arg = args.get(idx);
+ String value = null;
+
+ Matcher m = eqSeparatedOpt.matcher(arg);
+ if (m.matches()) {
+ arg = m.group(1);
+ value = m.group(2);
+ }
+
+ // Look for options with a value.
+ String name = findCliOption(arg, opts);
+ if (name != null) {
+ if (value == null) {
+ if (idx == args.size() - 1) {
+ throw new IllegalArgumentException(
+ String.format("Missing argument for option '%s'.", arg));
+ }
+ idx++;
+ value = args.get(idx);
+ }
+ if (!handle(name, value)) {
+ break;
+ }
+ continue;
+ }
+
+ // Look for a switch.
+ name = findCliOption(arg, switches);
+ if (name != null) {
+ if (!handle(name, null)) {
+ break;
+ }
+ continue;
+ }
+
+ if (!handleUnknown(arg)) {
+ break;
+ }
+ }
+
+ if (idx < args.size()) {
+ idx++;
+ }
+ handleExtraArgs(args.subList(idx, args.size()));
+ }
+
+ /**
+ * Callback for when an option with an argument is parsed.
+ *
+ * @param opt The long name of the cli option (might differ from actual command line).
+ * @param value The value. This will be null if the option does not take a value.
+ * @return Whether to continue parsing the argument list.
+ */
+ protected boolean handle(String opt, String value) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Callback for when an unrecognized option is parsed.
+ *
+ * @param opt Unrecognized option from the command line.
+ * @return Whether to continue parsing the argument list.
+ */
+ protected boolean handleUnknown(String opt) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Callback for remaining command line arguments after either {@link #handle(String, String)} or
+ * {@link #handleUnknown(String)} return "false". This will be called at the end of parsing even
+ * when there are no remaining arguments.
+ *
+ * @param extra List of remaining arguments.
+ */
+ protected void handleExtraArgs(List extra) {
+ throw new UnsupportedOperationException();
+ }
+
+ private String findCliOption(String name, String[][] available) {
+ for (String[] candidates : available) {
+ for (String candidate : candidates) {
+ if (candidate.equals(name)) {
+ return candidates[0];
+ }
+ }
+ }
+ return null;
+ }
+
+}
diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
new file mode 100644
index 0000000000000..7ed756f4b8591
--- /dev/null
+++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+/**
+ * Library for launching Spark applications.
+ *
+ * This library allows applications to launch Spark programmatically. There's only one entry
+ * point to the library - the {@link org.apache.spark.launcher.SparkLauncher} class.
+ *
+ * To launch a Spark application, just instantiate a {@link org.apache.spark.launcher.SparkLauncher}
+ * and configure the application to run. For example:
+ *
+ *
+ * {@code
+ * import org.apache.spark.launcher.SparkLauncher;
+ *
+ * public class MyLauncher {
+ * public static void main(String[] args) throws Exception {
+ * Process spark = new SparkLauncher()
+ * .setAppResource("/my/app.jar")
+ * .setMainClass("my.spark.app.Main")
+ * .setMaster("local")
+ * .setConf(SparkLauncher.DRIVER_MEMORY, "2g")
+ * .launch();
+ * spark.waitFor();
+ * }
+ * }
+ * }
+ *
+ */
+package org.apache.spark.launcher;
diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java
new file mode 100644
index 0000000000000..dba0203867372
--- /dev/null
+++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import static org.apache.spark.launcher.CommandBuilderUtils.*;
+
+public class CommandBuilderUtilsSuite {
+
+ @Test
+ public void testValidOptionStrings() {
+ testOpt("a b c d e", Arrays.asList("a", "b", "c", "d", "e"));
+ testOpt("a 'b c' \"d\" e", Arrays.asList("a", "b c", "d", "e"));
+ testOpt("a 'b\\\"c' \"'d'\" e", Arrays.asList("a", "b\\\"c", "'d'", "e"));
+ testOpt("a 'b\"c' \"\\\"d\\\"\" e", Arrays.asList("a", "b\"c", "\"d\"", "e"));
+ testOpt(" a b c \\\\ ", Arrays.asList("a", "b", "c", "\\"));
+
+ // Following tests ported from UtilsSuite.scala.
+ testOpt("", new ArrayList());
+ testOpt("a", Arrays.asList("a"));
+ testOpt("aaa", Arrays.asList("aaa"));
+ testOpt("a b c", Arrays.asList("a", "b", "c"));
+ testOpt(" a b\t c ", Arrays.asList("a", "b", "c"));
+ testOpt("a 'b c'", Arrays.asList("a", "b c"));
+ testOpt("a 'b c' d", Arrays.asList("a", "b c", "d"));
+ testOpt("'b c'", Arrays.asList("b c"));
+ testOpt("a \"b c\"", Arrays.asList("a", "b c"));
+ testOpt("a \"b c\" d", Arrays.asList("a", "b c", "d"));
+ testOpt("\"b c\"", Arrays.asList("b c"));
+ testOpt("a 'b\" c' \"d' e\"", Arrays.asList("a", "b\" c", "d' e"));
+ testOpt("a\t'b\nc'\nd", Arrays.asList("a", "b\nc", "d"));
+ testOpt("a \"b\\\\c\"", Arrays.asList("a", "b\\c"));
+ testOpt("a \"b\\\"c\"", Arrays.asList("a", "b\"c"));
+ testOpt("a 'b\\\"c'", Arrays.asList("a", "b\\\"c"));
+ testOpt("'a'b", Arrays.asList("ab"));
+ testOpt("'a''b'", Arrays.asList("ab"));
+ testOpt("\"a\"b", Arrays.asList("ab"));
+ testOpt("\"a\"\"b\"", Arrays.asList("ab"));
+ testOpt("''", Arrays.asList(""));
+ testOpt("\"\"", Arrays.asList(""));
+ }
+
+ @Test
+ public void testInvalidOptionStrings() {
+ testInvalidOpt("\\");
+ testInvalidOpt("\"abcde");
+ testInvalidOpt("'abcde");
+ }
+
+ @Test
+ public void testWindowsBatchQuoting() {
+ assertEquals("abc", quoteForBatchScript("abc"));
+ assertEquals("\"a b c\"", quoteForBatchScript("a b c"));
+ assertEquals("\"a \"\"b\"\" c\"", quoteForBatchScript("a \"b\" c"));
+ assertEquals("\"a\"\"b\"\"c\"", quoteForBatchScript("a\"b\"c"));
+ assertEquals("\"ab^=\"\"cd\"\"\"", quoteForBatchScript("ab=\"cd\""));
+ }
+
+ @Test
+ public void testPythonArgQuoting() {
+ assertEquals("\"abc\"", quoteForPython("abc"));
+ assertEquals("\"a b c\"", quoteForPython("a b c"));
+ assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c"));
+ }
+
+ private void testOpt(String opts, List expected) {
+ assertEquals(String.format("test string failed to parse: [[ %s ]]", opts),
+ expected, parseOptionString(opts));
+ }
+
+ private void testInvalidOpt(String opts) {
+ try {
+ parseOptionString(opts);
+ fail("Expected exception for invalid option string.");
+ } catch (IllegalArgumentException e) {
+ // pass.
+ }
+ }
+
+}
diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
new file mode 100644
index 0000000000000..252d5abae1ca3
--- /dev/null
+++ b/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.BufferedReader;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.junit.Assert.*;
+
+/**
+ * These tests require the Spark assembly to be built before they can be run.
+ */
+public class SparkLauncherSuite {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SparkLauncherSuite.class);
+
+ @Test
+ public void testChildProcLauncher() throws Exception {
+ Map env = new HashMap();
+ env.put("SPARK_PRINT_LAUNCH_COMMAND", "1");
+
+ SparkLauncher launcher = new SparkLauncher(env)
+ .setSparkHome(System.getProperty("spark.test.home"))
+ .setMaster("local")
+ .setAppResource("spark-internal")
+ .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS,
+ "-Dfoo=bar -Dtest.name=-testChildProcLauncher")
+ .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path"))
+ .setMainClass(SparkLauncherTestApp.class.getName())
+ .addAppArgs("proc");
+ final Process app = launcher.launch();
+ new Redirector("stdout", app.getInputStream()).start();
+ new Redirector("stderr", app.getErrorStream()).start();
+ assertEquals(0, app.waitFor());
+ }
+
+ public static class SparkLauncherTestApp {
+
+ public static void main(String[] args) throws Exception {
+ assertEquals(1, args.length);
+ assertEquals("proc", args[0]);
+ assertEquals("bar", System.getProperty("foo"));
+ assertEquals("local", System.getProperty(SparkLauncher.SPARK_MASTER));
+ }
+
+ }
+
+ private static class Redirector extends Thread {
+
+ private final InputStream in;
+
+ Redirector(String name, InputStream in) {
+ this.in = in;
+ setName(name);
+ setDaemon(true);
+ }
+
+ @Override
+ public void run() {
+ try {
+ BufferedReader reader = new BufferedReader(new InputStreamReader(in, "UTF-8"));
+ String line;
+ while ((line = reader.readLine()) != null) {
+ LOG.warn(line);
+ }
+ } catch (Exception e) {
+ LOG.error("Error reading process output.", e);
+ }
+ }
+
+ }
+
+}
diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
new file mode 100644
index 0000000000000..815edc4e4971f
--- /dev/null
+++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class SparkSubmitCommandBuilderSuite {
+
+ private static File dummyPropsFile;
+ private static SparkSubmitOptionParser parser;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ dummyPropsFile = File.createTempFile("spark", "properties");
+ parser = new SparkSubmitOptionParser();
+ }
+
+ @AfterClass
+ public static void cleanUp() throws Exception {
+ dummyPropsFile.delete();
+ }
+
+ @Test
+ public void testDriverCmdBuilder() throws Exception {
+ testCmdBuilder(true);
+ }
+
+ @Test
+ public void testClusterCmdBuilder() throws Exception {
+ testCmdBuilder(false);
+ }
+
+ @Test
+ public void testCliParser() throws Exception {
+ List sparkSubmitArgs = Arrays.asList(
+ parser.MASTER,
+ "local",
+ parser.DRIVER_MEMORY,
+ "42g",
+ parser.DRIVER_CLASS_PATH,
+ "/driverCp",
+ parser.DRIVER_JAVA_OPTIONS,
+ "extraJavaOpt",
+ parser.CONF,
+ SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath");
+ Map env = new HashMap();
+ List cmd = buildCommand(sparkSubmitArgs, env);
+
+ assertTrue(findInStringList(env.get(CommandBuilderUtils.getLibPathEnvName()),
+ File.pathSeparator, "/driverLibPath"));
+ assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp"));
+ assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g"));
+ assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g"));
+ }
+
+ @Test
+ public void testShellCliParser() throws Exception {
+ List sparkSubmitArgs = Arrays.asList(
+ parser.CLASS,
+ "org.apache.spark.repl.Main",
+ parser.MASTER,
+ "foo",
+ "--app-arg",
+ "bar",
+ "--app-switch",
+ parser.FILES,
+ "baz",
+ parser.NAME,
+ "appName");
+
+ List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs();
+ List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch");
+ assertEquals(expected, args.subList(args.size() - expected.size(), args.size()));
+ }
+
+ @Test
+ public void testAlternateSyntaxParsing() throws Exception {
+ List sparkSubmitArgs = Arrays.asList(
+ parser.CLASS + "=org.my.Class",
+ parser.MASTER + "=foo",
+ parser.DEPLOY_MODE + "=bar");
+
+ List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs();
+ assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS));
+ assertEquals("foo", findArgValue(cmd, parser.MASTER));
+ assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE));
+ }
+
+ @Test
+ public void testPySparkLauncher() throws Exception {
+ List sparkSubmitArgs = Arrays.asList(
+ SparkSubmitCommandBuilder.PYSPARK_SHELL,
+ "--master=foo",
+ "--deploy-mode=bar");
+
+ Map env = new HashMap();
+ List cmd = buildCommand(sparkSubmitArgs, env);
+ assertEquals("python", cmd.get(cmd.size() - 1));
+ assertEquals(
+ String.format("\"%s\" \"foo\" \"%s\" \"bar\" \"%s\"",
+ parser.MASTER, parser.DEPLOY_MODE, SparkSubmitCommandBuilder.PYSPARK_SHELL_RESOURCE),
+ env.get("PYSPARK_SUBMIT_ARGS"));
+ }
+
+ @Test
+ public void testPySparkFallback() throws Exception {
+ List sparkSubmitArgs = Arrays.asList(
+ "--master=foo",
+ "--deploy-mode=bar",
+ "script.py",
+ "arg1");
+
+ Map env = new HashMap();
+ List cmd = buildCommand(sparkSubmitArgs, env);
+
+ assertEquals("foo", findArgValue(cmd, "--master"));
+ assertEquals("bar", findArgValue(cmd, "--deploy-mode"));
+ assertEquals("script.py", cmd.get(cmd.size() - 2));
+ assertEquals("arg1", cmd.get(cmd.size() - 1));
+ }
+
+ private void testCmdBuilder(boolean isDriver) throws Exception {
+ String deployMode = isDriver ? "client" : "cluster";
+
+ SparkSubmitCommandBuilder launcher =
+ new SparkSubmitCommandBuilder(Collections.emptyList());
+ launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME,
+ System.getProperty("spark.test.home"));
+ launcher.master = "yarn";
+ launcher.deployMode = deployMode;
+ launcher.appResource = "/foo";
+ launcher.appName = "MyApp";
+ launcher.mainClass = "my.Class";
+ launcher.propertiesFile = dummyPropsFile.getAbsolutePath();
+ launcher.appArgs.add("foo");
+ launcher.appArgs.add("bar");
+ launcher.conf.put(SparkLauncher.DRIVER_MEMORY, "1g");
+ launcher.conf.put(SparkLauncher.DRIVER_EXTRA_CLASSPATH, "/driver");
+ launcher.conf.put(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, "-Ddriver -XX:MaxPermSize=256m");
+ launcher.conf.put(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, "/native");
+ launcher.conf.put("spark.foo", "foo");
+
+ Map env = new HashMap();
+ List cmd = launcher.buildCommand(env);
+
+ // Checks below are different for driver and non-driver mode.
+
+ if (isDriver) {
+ assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms1g"));
+ assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx1g"));
+ } else {
+ boolean found = false;
+ for (String arg : cmd) {
+ if (arg.startsWith("-Xms") || arg.startsWith("-Xmx")) {
+ found = true;
+ break;
+ }
+ }
+ assertFalse("Memory arguments should not be set.", found);
+ }
+
+ for (String arg : cmd) {
+ if (arg.startsWith("-XX:MaxPermSize=")) {
+ if (isDriver) {
+ assertEquals("-XX:MaxPermSize=256m", arg);
+ } else {
+ assertEquals("-XX:MaxPermSize=128m", arg);
+ }
+ }
+ }
+
+ String[] cp = findArgValue(cmd, "-cp").split(Pattern.quote(File.pathSeparator));
+ if (isDriver) {
+ assertTrue("Driver classpath should contain provided entry.", contains("/driver", cp));
+ } else {
+ assertFalse("Driver classpath should not be in command.", contains("/driver", cp));
+ }
+
+ String libPath = env.get(CommandBuilderUtils.getLibPathEnvName());
+ if (isDriver) {
+ assertNotNull("Native library path should be set.", libPath);
+ assertTrue("Native library path should contain provided entry.",
+ contains("/native", libPath.split(Pattern.quote(File.pathSeparator))));
+ } else {
+ assertNull("Native library should not be set.", libPath);
+ }
+
+ // Checks below are the same for both driver and non-driver mode.
+ assertEquals(dummyPropsFile.getAbsolutePath(), findArgValue(cmd, parser.PROPERTIES_FILE));
+ assertEquals("yarn", findArgValue(cmd, parser.MASTER));
+ assertEquals(deployMode, findArgValue(cmd, parser.DEPLOY_MODE));
+ assertEquals("my.Class", findArgValue(cmd, parser.CLASS));
+ assertEquals("MyApp", findArgValue(cmd, parser.NAME));
+
+ boolean appArgsOk = false;
+ for (int i = 0; i < cmd.size(); i++) {
+ if (cmd.get(i).equals("/foo")) {
+ assertEquals("foo", cmd.get(i + 1));
+ assertEquals("bar", cmd.get(i + 2));
+ assertEquals(cmd.size(), i + 3);
+ appArgsOk = true;
+ break;
+ }
+ }
+ assertTrue("App resource and args should be added to command.", appArgsOk);
+
+ Map conf = parseConf(cmd, parser);
+ assertEquals("foo", conf.get("spark.foo"));
+ }
+
+ private boolean contains(String needle, String[] haystack) {
+ for (String entry : haystack) {
+ if (entry.equals(needle)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private Map parseConf(List cmd, SparkSubmitOptionParser parser) {
+ Map conf = new HashMap();
+ for (int i = 0; i < cmd.size(); i++) {
+ if (cmd.get(i).equals(parser.CONF)) {
+ String[] val = cmd.get(i + 1).split("=", 2);
+ conf.put(val[0], val[1]);
+ i += 1;
+ }
+ }
+ return conf;
+ }
+
+ private String findArgValue(List cmd, String name) {
+ for (int i = 0; i < cmd.size(); i++) {
+ if (cmd.get(i).equals(name)) {
+ return cmd.get(i + 1);
+ }
+ }
+ fail(String.format("arg '%s' not found", name));
+ return null;
+ }
+
+ private boolean findInStringList(String list, String sep, String needle) {
+ return contains(needle, list.split(sep));
+ }
+
+ private List buildCommand(List args, Map env) throws Exception {
+ SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args);
+ builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home"));
+ return builder.buildCommand(env);
+ }
+
+}
diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java
new file mode 100644
index 0000000000000..f3d2109917056
--- /dev/null
+++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.launcher;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import static org.apache.spark.launcher.SparkSubmitOptionParser.*;
+
+public class SparkSubmitOptionParserSuite {
+
+ private SparkSubmitOptionParser parser;
+
+ @Before
+ public void setUp() {
+ parser = spy(new DummyParser());
+ }
+
+ @Test
+ public void testAllOptions() {
+ int count = 0;
+ for (String[] optNames : parser.opts) {
+ for (String optName : optNames) {
+ String value = optName + "-value";
+ parser.parse(Arrays.asList(optName, value));
+ count++;
+ verify(parser).handle(eq(optNames[0]), eq(value));
+ verify(parser, times(count)).handle(anyString(), anyString());
+ verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList()));
+ }
+ }
+
+ for (String[] switchNames : parser.switches) {
+ int switchCount = 0;
+ for (String name : switchNames) {
+ parser.parse(Arrays.asList(name));
+ count++;
+ switchCount++;
+ verify(parser, times(switchCount)).handle(eq(switchNames[0]), same((String) null));
+ verify(parser, times(count)).handle(anyString(), any(String.class));
+ verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList()));
+ }
+ }
+ }
+
+ @Test
+ public void testExtraOptions() {
+ List args = Arrays.asList(parser.MASTER, parser.MASTER, "foo", "bar");
+ parser.parse(args);
+ verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER));
+ verify(parser).handleUnknown(eq("foo"));
+ verify(parser).handleExtraArgs(eq(Arrays.asList("bar")));
+ }
+
+ @Test(expected=IllegalArgumentException.class)
+ public void testMissingArg() {
+ parser.parse(Arrays.asList(parser.MASTER));
+ }
+
+ @Test
+ public void testEqualSeparatedOption() {
+ List args = Arrays.asList(parser.MASTER + "=" + parser.MASTER);
+ parser.parse(args);
+ verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER));
+ verify(parser).handleExtraArgs(eq(Collections.emptyList()));
+ }
+
+ private static class DummyParser extends SparkSubmitOptionParser {
+
+ @Override
+ protected boolean handle(String opt, String value) {
+ return true;
+ }
+
+ @Override
+ protected boolean handleUnknown(String opt) {
+ return false;
+ }
+
+ @Override
+ protected void handleExtraArgs(List extra) {
+
+ }
+
+ }
+
+}
diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000..00c20ad69cd4d
--- /dev/null
+++ b/launcher/src/test/resources/log4j.properties
@@ -0,0 +1,31 @@
+#
+# 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 core/target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+
+# Some tests will set "test.name" to avoid overwriting the main log file.
+log4j.appender.file.file=target/unit-tests${test.name}.log
+
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+org.eclipse.jetty.LEVEL=WARN
diff --git a/make-distribution.sh b/make-distribution.sh
index dd990d4b96e46..82d33408cd5e5 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -127,6 +127,7 @@ if [ ! $(command -v "$MVN") ] ; then
fi
VERSION=$("$MVN" help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1)
+SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/null | grep -v "INFO" | tail -n 1)
SPARK_HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\
| grep -v "INFO"\
| tail -n 1)
@@ -196,6 +197,7 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE"
# Copy jars
cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
+cp "$SPARK_HOME"/launcher/target/spark-launcher_$SCALA_VERSION-$VERSION.jar "$DISTDIR/lib/"
# This will fail if the -Pyarn profile is not provided
# In this case, silence the error and ignore the return code of this command
cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
diff --git a/mllib/pom.xml b/mllib/pom.xml
index 4c8f34417ca65..b5c949e155cfd 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
index 3b95a9e6936e8..707da537d238f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
@@ -17,15 +17,22 @@
package org.apache.spark.mllib.clustering
+import org.json4s._
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods._
+
import org.apache.spark.api.java.JavaRDD
-import org.apache.spark.rdd.RDD
-import org.apache.spark.SparkContext._
import org.apache.spark.mllib.linalg.Vector
+import org.apache.spark.mllib.util.{Loader, Saveable}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.SparkContext
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.Row
/**
* A clustering model for K-means. Each point belongs to the cluster with the closest center.
*/
-class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable {
+class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable {
/** Total number of clusters. */
def k: Int = clusterCenters.length
@@ -58,4 +65,59 @@ class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable {
private def clusterCentersWithNorm: Iterable[VectorWithNorm] =
clusterCenters.map(new VectorWithNorm(_))
+
+ override def save(sc: SparkContext, path: String): Unit = {
+ KMeansModel.SaveLoadV1_0.save(sc, this, path)
+ }
+
+ override protected def formatVersion: String = "1.0"
+}
+
+object KMeansModel extends Loader[KMeansModel] {
+ override def load(sc: SparkContext, path: String): KMeansModel = {
+ KMeansModel.SaveLoadV1_0.load(sc, path)
+ }
+
+ private case class Cluster(id: Int, point: Vector)
+
+ private object Cluster {
+ def apply(r: Row): Cluster = {
+ Cluster(r.getInt(0), r.getAs[Vector](1))
+ }
+ }
+
+ private[clustering]
+ object SaveLoadV1_0 {
+
+ private val thisFormatVersion = "1.0"
+
+ private[clustering]
+ val thisClassName = "org.apache.spark.mllib.clustering.KMeansModel"
+
+ def save(sc: SparkContext, model: KMeansModel, path: String): Unit = {
+ val sqlContext = new SQLContext(sc)
+ import sqlContext.implicits._
+ val metadata = compact(render(
+ ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k)))
+ sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path))
+ val dataRDD = sc.parallelize(model.clusterCenters.zipWithIndex).map { case (point, id) =>
+ Cluster(id, point)
+ }.toDF()
+ dataRDD.saveAsParquetFile(Loader.dataPath(path))
+ }
+
+ def load(sc: SparkContext, path: String): KMeansModel = {
+ implicit val formats = DefaultFormats
+ val sqlContext = new SQLContext(sc)
+ val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path)
+ assert(className == thisClassName)
+ assert(formatVersion == thisFormatVersion)
+ val k = (metadata \ "k").extract[Int]
+ val centriods = sqlContext.parquetFile(Loader.dataPath(path))
+ Loader.checkSchema[Cluster](centriods.schema)
+ val localCentriods = centriods.map(Cluster.apply).collect()
+ assert(k == localCentriods.size)
+ new KMeansModel(localCentriods.sortBy(_.id).map(_.point))
+ }
+ }
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
index ced042e2f96ca..c1d1a224817e8 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
@@ -22,6 +22,7 @@ import org.apache.spark.Logging
import org.apache.spark.SparkContext._
import org.apache.spark.mllib.evaluation.binary._
import org.apache.spark.rdd.{RDD, UnionRDD}
+import org.apache.spark.sql.DataFrame
/**
* :: Experimental ::
@@ -53,6 +54,13 @@ class BinaryClassificationMetrics(
*/
def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0)
+ /**
+ * An auxiliary constructor taking a DataFrame.
+ * @param scoreAndLabels a DataFrame with two double columns: score and label
+ */
+ private[mllib] def this(scoreAndLabels: DataFrame) =
+ this(scoreAndLabels.map(r => (r.getDouble(0), r.getDouble(1))))
+
/** Unpersist intermediate RDDs used in the computation. */
def unpersist() {
cumulativeCounts.unpersist()
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
index caee5917000aa..7bf250eb5a383 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
@@ -21,9 +21,10 @@ import scala.util.Random
import org.scalatest.FunSuite
-import org.apache.spark.mllib.linalg.{Vector, Vectors}
+import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors}
import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext}
import org.apache.spark.mllib.util.TestingUtils._
+import org.apache.spark.util.Utils
class KMeansSuite extends FunSuite with MLlibTestSparkContext {
@@ -257,6 +258,47 @@ class KMeansSuite extends FunSuite with MLlibTestSparkContext {
assert(predicts(0) != predicts(3))
}
}
+
+ test("model save/load") {
+ val tempDir = Utils.createTempDir()
+ val path = tempDir.toURI.toString
+
+ Array(true, false).foreach { case selector =>
+ val model = KMeansSuite.createModel(10, 3, selector)
+ // Save model, load it back, and compare.
+ try {
+ model.save(sc, path)
+ val sameModel = KMeansModel.load(sc, path)
+ KMeansSuite.checkEqual(model, sameModel)
+ } finally {
+ Utils.deleteRecursively(tempDir)
+ }
+ }
+ }
+}
+
+object KMeansSuite extends FunSuite {
+ def createModel(dim: Int, k: Int, isSparse: Boolean): KMeansModel = {
+ val singlePoint = isSparse match {
+ case true =>
+ Vectors.sparse(dim, Array.empty[Int], Array.empty[Double])
+ case _ =>
+ Vectors.dense(Array.fill[Double](dim)(0.0))
+ }
+ new KMeansModel(Array.fill[Vector](k)(singlePoint))
+ }
+
+ def checkEqual(a: KMeansModel, b: KMeansModel): Unit = {
+ assert(a.k === b.k)
+ a.clusterCenters.zip(b.clusterCenters).foreach {
+ case (ca: SparseVector, cb: SparseVector) =>
+ assert(ca === cb)
+ case (ca: DenseVector, cb: DenseVector) =>
+ assert(ca === cb)
+ case _ =>
+ throw new AssertionError("checkEqual failed since the two clusters were not identical.\n")
+ }
+ }
}
class KMeansClusterSuite extends FunSuite with LocalClusterSparkContext {
diff --git a/network/common/pom.xml b/network/common/pom.xml
index 8f7c924d6b3a3..74437f37c47e4 100644
--- a/network/common/pom.xml
+++ b/network/common/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java
index 5bc6e5a2418a9..f0a89c9d9116c 100644
--- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java
+++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java
@@ -35,7 +35,6 @@
import org.apache.spark.network.server.TransportChannelHandler;
import org.apache.spark.network.server.TransportRequestHandler;
import org.apache.spark.network.server.TransportServer;
-import org.apache.spark.network.server.StreamManager;
import org.apache.spark.network.util.NettyUtils;
import org.apache.spark.network.util.TransportConf;
diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java
index 986957c1509fd..f76bb49e874fc 100644
--- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java
+++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java
@@ -17,7 +17,6 @@
package org.apache.spark.network.protocol;
-import com.google.common.base.Charsets;
import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java
index 873c694250942..9162d0b977f83 100644
--- a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java
+++ b/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java
@@ -20,7 +20,6 @@
import com.google.common.base.Charsets;
import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
/** Provides a canonical set of Encoders for simple types. */
public class Encoders {
diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java
index ebd764eb5eb5f..6b991375fc486 100644
--- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java
+++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java
@@ -17,7 +17,6 @@
package org.apache.spark.network.protocol;
-import com.google.common.base.Charsets;
import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java
index ef209991804b4..b7ce8541e565e 100644
--- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java
+++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java
@@ -28,7 +28,6 @@
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
-import io.netty.util.internal.PlatformDependent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
index bf8a1fc42fc6d..73da9b7346f4d 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java
@@ -17,19 +17,13 @@
package org.apache.spark.network.util;
-import java.nio.ByteBuffer;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
+import java.nio.ByteBuffer;
-import com.google.common.base.Preconditions;
-import com.google.common.io.Closeables;
import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
import io.netty.buffer.Unpooled;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java
index 2a4b88b64cdc9..dabd6261d2aa0 100644
--- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java
+++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java
@@ -25,7 +25,6 @@
import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.ServerChannel;
-import io.netty.channel.epoll.Epoll;
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.epoll.EpollServerSocketChannel;
import io.netty.channel.epoll.EpollSocketChannel;
diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml
index c2d0300ecd904..a2bcca26d8344 100644
--- a/network/shuffle/pom.xml
+++ b/network/shuffle/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
index 3777a18e33f78..026cbd260d16c 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java
@@ -19,16 +19,13 @@
import java.util.concurrent.ConcurrentMap;
-import com.google.common.base.Charsets;
import com.google.common.collect.Maps;
-import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.spark.network.client.RpcResponseCallback;
import org.apache.spark.network.client.TransportClient;
-import org.apache.spark.network.protocol.Encodable;
import org.apache.spark.network.server.RpcHandler;
import org.apache.spark.network.server.StreamManager;
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java
index 8ed2e0b39ad23..e653f5cb147ee 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java
@@ -29,7 +29,6 @@
import org.apache.spark.network.shuffle.protocol.BlockTransferMessage;
import org.apache.spark.network.shuffle.protocol.OpenBlocks;
import org.apache.spark.network.shuffle.protocol.StreamHandle;
-import org.apache.spark.network.util.JavaUtils;
/**
* Simple wrapper on top of a TransportClient which interprets each chunk as a whole block, and
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java
index 62fce9b0d16cd..60485bace643c 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java
@@ -23,7 +23,6 @@
import io.netty.buffer.ByteBuf;
import org.apache.spark.network.protocol.Encoders;
-import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
/** Request to read a set of blocks. Returns {@link StreamHandle}. */
public class OpenBlocks extends BlockTransferMessage {
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java
index 7eb4385044077..38acae3b31d64 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java
@@ -21,7 +21,6 @@
import io.netty.buffer.ByteBuf;
import org.apache.spark.network.protocol.Encoders;
-import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
/**
* Initial registration message between an executor and its local shuffle server.
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java
index bc9daa6158ba3..9a9220211a50c 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java
@@ -20,8 +20,6 @@
import com.google.common.base.Objects;
import io.netty.buffer.ByteBuf;
-import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
-
/**
* Identifier for a fixed number of chunks to read from a stream created by an "open blocks"
* message. This is used by {@link org.apache.spark.network.shuffle.OneForOneBlockFetcher}.
diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java
index 0b23e112bd512..2ff9aaa650f92 100644
--- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java
+++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java
@@ -23,7 +23,6 @@
import io.netty.buffer.ByteBuf;
import org.apache.spark.network.protocol.Encoders;
-import org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
/** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */
diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
index 67a07f38eb5a0..23b4e06f064e1 100644
--- a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
+++ b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java
@@ -17,12 +17,12 @@
package org.apache.spark.network.sasl;
-import java.util.Map;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
-import com.google.common.collect.ImmutableMap;
import org.junit.Test;
-import static org.junit.Assert.*;
/**
* Jointly tests SparkSaslClient and SparkSaslServer, as both are black boxes.
diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java
index 842741e3d354f..b35a6d685dd02 100644
--- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java
+++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java
@@ -28,11 +28,16 @@
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
-import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.buffer.NettyManagedBuffer;
diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml
index 39b99f54f6dbc..cea7a20c223e2 100644
--- a/network/yarn/pom.xml
+++ b/network/yarn/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/pom.xml b/pom.xml
index 77d08d6ee826f..a19da73cf45b3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,7 +25,7 @@
14
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
pom
Spark Project Parent POM
@@ -105,6 +105,7 @@
external/zeromq
examples
repl
+ launcher
@@ -422,6 +423,13 @@
2.42.2
test
+
+
+ xml-apis
+ xml-apis
+ 1.4.01
+ test
+
org.slf4j
slf4j-api
@@ -583,10 +591,18 @@
jackson-databind
${fasterxml.jackson.version}
+
com.fasterxml.jackson.module
jackson-module-scala_2.10
${fasterxml.jackson.version}
+
+
+ com.google.guava
+ guava
+
+
org.scala-lang
@@ -1180,7 +1196,7 @@
true
- ${session.executionRootDirectory}
+ ${spark.test.home}
1
false
false
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 4f17df59f4c1f..35e748f26bbaa 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -34,11 +34,11 @@ object BuildCommons {
val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl,
sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka,
- streamingMqtt, streamingTwitter, streamingZeromq) =
+ streamingMqtt, streamingTwitter, streamingZeromq, launcher) =
Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl",
"sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink",
"streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter",
- "streaming-zeromq").map(ProjectRef(buildLocation, _))
+ "streaming-zeromq", "launcher").map(ProjectRef(buildLocation, _))
val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl,
sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl",
@@ -155,8 +155,9 @@ object SparkBuild extends PomBuild {
(allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings))
// TODO: Add Sql to mima checks
+ // TODO: remove launcher from this list after 1.3.
allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl,
- networkCommon, networkShuffle, networkYarn).contains(x)).foreach {
+ networkCommon, networkShuffle, networkYarn, launcher).contains(x)).foreach {
x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)
}
diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst
index b706c5e376ef4..15101470afc07 100644
--- a/python/docs/pyspark.mllib.rst
+++ b/python/docs/pyspark.mllib.rst
@@ -16,6 +16,13 @@ pyspark.mllib.clustering module
:members:
:undoc-members:
+pyspark.mllib.evaluation module
+-------------------------------
+
+.. automodule:: pyspark.mllib.evaluation
+ :members:
+ :undoc-members:
+
pyspark.mllib.feature module
-------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 6011caf9f1c5a..78dccc40470e3 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -21,6 +21,8 @@
from threading import Lock
from tempfile import NamedTemporaryFile
+from py4j.java_collections import ListConverter
+
from pyspark import accumulators
from pyspark.accumulators import Accumulator
from pyspark.broadcast import Broadcast
@@ -30,13 +32,11 @@
from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \
PairDeserializer, AutoBatchedSerializer, NoOpSerializer
from pyspark.storagelevel import StorageLevel
-from pyspark.rdd import RDD
+from pyspark.rdd import RDD, _load_from_socket
from pyspark.traceback_utils import CallSite, first_spark_call
from pyspark.status import StatusTracker
from pyspark.profiler import ProfilerCollector, BasicProfiler
-from py4j.java_collections import ListConverter
-
__all__ = ['SparkContext']
@@ -59,7 +59,6 @@ class SparkContext(object):
_gateway = None
_jvm = None
- _writeToFile = None
_next_accum_id = 0
_active_spark_context = None
_lock = Lock()
@@ -221,7 +220,6 @@ def _ensure_initialized(cls, instance=None, gateway=None):
if not SparkContext._gateway:
SparkContext._gateway = gateway or launch_gateway()
SparkContext._jvm = SparkContext._gateway.jvm
- SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile
if instance:
if (SparkContext._active_spark_context and
@@ -840,8 +838,9 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False):
# by runJob() in order to avoid having to pass a Python lambda into
# SparkContext#runJob.
mappedRDD = rdd.mapPartitions(partitionFunc)
- it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal)
- return list(mappedRDD._collect_iterator_through_file(it))
+ port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions,
+ allowLocal)
+ return list(_load_from_socket(port, mappedRDD._jrdd_deserializer))
def show_profiles(self):
""" Print the profile stats to stdout """
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 936857e75c7e9..43d2cf5171880 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -41,7 +41,7 @@ def launch_gateway():
submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS")
submit_args = submit_args if submit_args is not None else ""
submit_args = shlex.split(submit_args)
- command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"]
+ command = [os.path.join(SPARK_HOME, script)] + submit_args
# Start a socket that will be used by PythonGatewayServer to communicate its port to us
callback_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
@@ -58,7 +58,6 @@ def launch_gateway():
# Don't send ctrl-c / SIGINT to the Java gateway:
def preexec_func():
signal.signal(signal.SIGINT, signal.SIG_IGN)
- env["IS_SUBPROCESS"] = "1" # tell JVM to exit after python exits
proc = Popen(command, stdin=PIPE, preexec_fn=preexec_func, env=env)
else:
# preexec_fn not supported on Windows
diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py
index 5233c5801e2e6..83880a5afcd1d 100644
--- a/python/pyspark/ml/pipeline.py
+++ b/python/pyspark/ml/pipeline.py
@@ -39,7 +39,7 @@ def fit(self, dataset, params={}):
Fits a model to the input dataset with optional parameters.
:param dataset: input dataset, which is an instance of
- :py:class:`pyspark.sql.SchemaRDD`
+ :py:class:`pyspark.sql.DataFrame`
:param params: an optional param map that overwrites embedded
params
:returns: fitted model
@@ -62,7 +62,7 @@ def transform(self, dataset, params={}):
Transforms the input dataset with optional parameters.
:param dataset: input dataset, which is an instance of
- :py:class:`pyspark.sql.SchemaRDD`
+ :py:class:`pyspark.sql.DataFrame`
:param params: an optional param map that overwrites embedded
params
:returns: transformed dataset
diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py
index 4bae96f678388..31a66b3d2f730 100644
--- a/python/pyspark/ml/wrapper.py
+++ b/python/pyspark/ml/wrapper.py
@@ -102,7 +102,7 @@ def _fit_java(self, dataset, params={}):
"""
Fits a Java model to the input dataset.
:param dataset: input dataset, which is an instance of
- :py:class:`pyspark.sql.SchemaRDD`
+ :py:class:`pyspark.sql.DataFrame`
:param params: additional params (overwriting embedded values)
:return: fitted Java model
"""
diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py
new file mode 100644
index 0000000000000..16cb49cc0cfff
--- /dev/null
+++ b/python/pyspark/mllib/evaluation.py
@@ -0,0 +1,83 @@
+#
+# 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.
+#
+
+from pyspark.mllib.common import JavaModelWrapper
+from pyspark.sql import SQLContext
+from pyspark.sql.types import StructField, StructType, DoubleType
+
+
+class BinaryClassificationMetrics(JavaModelWrapper):
+ """
+ Evaluator for binary classification.
+
+ >>> scoreAndLabels = sc.parallelize([
+ ... (0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)], 2)
+ >>> metrics = BinaryClassificationMetrics(scoreAndLabels)
+ >>> metrics.areaUnderROC()
+ 0.70...
+ >>> metrics.areaUnderPR()
+ 0.83...
+ >>> metrics.unpersist()
+ """
+
+ def __init__(self, scoreAndLabels):
+ """
+ :param scoreAndLabels: an RDD of (score, label) pairs
+ """
+ sc = scoreAndLabels.ctx
+ sql_ctx = SQLContext(sc)
+ df = sql_ctx.createDataFrame(scoreAndLabels, schema=StructType([
+ StructField("score", DoubleType(), nullable=False),
+ StructField("label", DoubleType(), nullable=False)]))
+ java_class = sc._jvm.org.apache.spark.mllib.evaluation.BinaryClassificationMetrics
+ java_model = java_class(df._jdf)
+ super(BinaryClassificationMetrics, self).__init__(java_model)
+
+ def areaUnderROC(self):
+ """
+ Computes the area under the receiver operating characteristic
+ (ROC) curve.
+ """
+ return self.call("areaUnderROC")
+
+ def areaUnderPR(self):
+ """
+ Computes the area under the precision-recall curve.
+ """
+ return self.call("areaUnderPR")
+
+ def unpersist(self):
+ """
+ Unpersists intermediate RDDs used in the computation.
+ """
+ self.call("unpersist")
+
+
+def _test():
+ import doctest
+ from pyspark import SparkContext
+ import pyspark.mllib.evaluation
+ globs = pyspark.mllib.evaluation.__dict__.copy()
+ globs['sc'] = SparkContext('local[4]', 'PythonTest')
+ (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
+ globs['sc'].stop()
+ if failure_count:
+ exit(-1)
+
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index cb12fed98c53d..bf17f513c0bc3 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -19,7 +19,6 @@
from collections import defaultdict
from itertools import chain, ifilter, imap
import operator
-import os
import sys
import shlex
from subprocess import Popen, PIPE
@@ -29,6 +28,7 @@
import heapq
import bisect
import random
+import socket
from math import sqrt, log, isinf, isnan, pow, ceil
from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
@@ -111,6 +111,17 @@ def _parse_memory(s):
return int(float(s[:-1]) * units[s[-1].lower()])
+def _load_from_socket(port, serializer):
+ sock = socket.socket()
+ try:
+ sock.connect(("localhost", port))
+ rf = sock.makefile("rb", 65536)
+ for item in serializer.load_stream(rf):
+ yield item
+ finally:
+ sock.close()
+
+
class Partitioner(object):
def __init__(self, numPartitions, partitionFunc):
self.numPartitions = numPartitions
@@ -698,21 +709,8 @@ def collect(self):
Return a list that contains all of the elements in this RDD.
"""
with SCCallSiteSync(self.context) as css:
- bytesInJava = self._jrdd.collect().iterator()
- return list(self._collect_iterator_through_file(bytesInJava))
-
- def _collect_iterator_through_file(self, iterator):
- # Transferring lots of data through Py4J can be slow because
- # socket.readline() is inefficient. Instead, we'll dump the data to a
- # file and read it back.
- tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir)
- tempFile.close()
- self.ctx._writeToFile(iterator, tempFile.name)
- # Read the data into Python and deserialize it:
- with open(tempFile.name, 'rb') as tempFile:
- for item in self._jrdd_deserializer.load_stream(tempFile):
- yield item
- os.unlink(tempFile.name)
+ port = self.ctx._jvm.PythonRDD.collectAndServe(self._jrdd.rdd())
+ return list(_load_from_socket(port, self._jrdd_deserializer))
def reduce(self, f):
"""
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 5c3b7377c33b5..e8ce4547455a5 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -19,13 +19,11 @@
import itertools
import warnings
import random
-import os
-from tempfile import NamedTemporaryFile
from py4j.java_collections import ListConverter, MapConverter
from pyspark.context import SparkContext
-from pyspark.rdd import RDD
+from pyspark.rdd import RDD, _load_from_socket
from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer
from pyspark.storagelevel import StorageLevel
from pyspark.traceback_utils import SCCallSiteSync
@@ -310,14 +308,8 @@ def collect(self):
[Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')]
"""
with SCCallSiteSync(self._sc) as css:
- bytesInJava = self._jdf.javaToPython().collect().iterator()
- tempFile = NamedTemporaryFile(delete=False, dir=self._sc._temp_dir)
- tempFile.close()
- self._sc._writeToFile(bytesInJava, tempFile.name)
- # Read the data into Python and deserialize it:
- with open(tempFile.name, 'rb') as tempFile:
- rs = list(BatchedSerializer(PickleSerializer()).load_stream(tempFile))
- os.unlink(tempFile.name)
+ port = self._sc._jvm.PythonRDD.collectAndServe(self._jdf.javaToPython().rdd())
+ rs = list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
cls = _create_cls(self.schema)
return [cls(r) for r in rs]
diff --git a/python/run-tests b/python/run-tests
index a2c2f37a54eda..b7630c356cfae 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -75,6 +75,7 @@ function run_mllib_tests() {
echo "Run mllib tests ..."
run_test "pyspark/mllib/classification.py"
run_test "pyspark/mllib/clustering.py"
+ run_test "pyspark/mllib/evaluation.py"
run_test "pyspark/mllib/feature.py"
run_test "pyspark/mllib/linalg.py"
run_test "pyspark/mllib/rand.py"
diff --git a/repl/pom.xml b/repl/pom.xml
index b883344bf0ceb..295f88ea3ecf9 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index f966f25c5a14c..ed9b207a86a0b 100644
--- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -263,14 +263,14 @@ class ReplSuite extends FunSuite {
assertDoesNotContain("Exception", output)
}
- test("SPARK-2576 importing SQLContext.createSchemaRDD.") {
+ test("SPARK-2576 importing SQLContext.createDataFrame.") {
// We need to use local-cluster to test this case.
val output = runInterpreter("local-cluster[1,1,512]",
"""
|val sqlContext = new org.apache.spark.sql.SQLContext(sc)
- |import sqlContext.createSchemaRDD
+ |import sqlContext.implicits._
|case class TestCaseClass(value: Int)
- |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toSchemaRDD.collect
+ |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDF.collect
""".stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh
index 5e812a1d91c6b..92e76a3fe6ca2 100755
--- a/sbin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@ -121,45 +121,63 @@ if [ "$SPARK_NICENESS" = "" ]; then
export SPARK_NICENESS=0
fi
+run_command() {
+ mode="$1"
+ shift
-case $option in
+ mkdir -p "$SPARK_PID_DIR"
- (start|spark-submit)
+ if [ -f "$pid" ]; then
+ TARGET_ID="$(cat "$pid")"
+ if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then
+ echo "$command running as process $TARGET_ID. Stop it first."
+ exit 1
+ fi
+ fi
- mkdir -p "$SPARK_PID_DIR"
+ if [ "$SPARK_MASTER" != "" ]; then
+ echo rsync from "$SPARK_MASTER"
+ rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' "$SPARK_MASTER/" "$SPARK_HOME"
+ fi
- if [ -f $pid ]; then
- TARGET_ID="$(cat "$pid")"
- if [[ $(ps -p "$TARGET_ID" -o args=) =~ $command ]]; then
- echo "$command running as process $TARGET_ID. Stop it first."
- exit 1
- fi
- fi
+ spark_rotate_log "$log"
+ echo "starting $command, logging to $log"
+
+ case "$mode" in
+ (class)
+ nohup nice -n "$SPARK_NICENESS" "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+ newpid="$!"
+ ;;
+
+ (submit)
+ nohup nice -n "$SPARK_NICENESS" "$SPARK_PREFIX"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null &
+ newpid="$!"
+ ;;
+
+ (*)
+ echo "unknown mode: $mode"
+ exit 1
+ ;;
+ esac
+
+ echo "$newpid" > "$pid"
+ sleep 2
+ # Check if the process has died; in that case we'll tail the log so the user can see
+ if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then
+ echo "failed to launch $command:"
+ tail -2 "$log" | sed 's/^/ /'
+ echo "full log in $log"
+ fi
+}
- if [ "$SPARK_MASTER" != "" ]; then
- echo rsync from "$SPARK_MASTER"
- rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME"
- fi
+case $option in
- spark_rotate_log "$log"
- echo "starting $command, logging to $log"
- if [ $option == spark-submit ]; then
- source "$SPARK_HOME"/bin/utils.sh
- gatherSparkSubmitOpts "$@"
- nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command \
- "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" >> "$log" 2>&1 < /dev/null &
- else
- nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
- fi
- newpid=$!
- echo $newpid > $pid
- sleep 2
- # Check if the process has died; in that case we'll tail the log so the user can see
- if [[ ! $(ps -p "$newpid" -o args=) =~ $command ]]; then
- echo "failed to launch $command:"
- tail -2 "$log" | sed 's/^/ /'
- echo "full log in $log"
- fi
+ (submit)
+ run_command submit "$@"
+ ;;
+
+ (start)
+ run_command class "$@"
;;
(stop)
diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh
index ba1a84abc1fef..76316a3067c93 100755
--- a/sbin/start-slaves.sh
+++ b/sbin/start-slaves.sh
@@ -64,6 +64,6 @@ else
SPARK_WORKER_WEBUI_PORT=8081
fi
for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
- "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
+ "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT"
done
fi
diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh
index 070cc7a87e6f2..5b0aeb177fff3 100755
--- a/sbin/start-thriftserver.sh
+++ b/sbin/start-thriftserver.sh
@@ -52,4 +52,4 @@ fi
export SUBMIT_USAGE_FUNCTION=usage
-exec "$FWDIR"/sbin/spark-daemon.sh spark-submit $CLASS 1 "$@"
+exec "$FWDIR"/sbin/spark-daemon.sh submit $CLASS 1 "$@"
diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh
index 971d5d49da664..1a9abe07db844 100755
--- a/sbin/stop-all.sh
+++ b/sbin/stop-all.sh
@@ -17,8 +17,8 @@
# limitations under the License.
#
-# Start all spark daemons.
-# Run this on the master nde
+# Stop all spark daemons.
+# Run this on the master node.
sbin="`dirname "$0"`"
diff --git a/sbin/stop-master.sh b/sbin/stop-master.sh
index b6bdaa4db373c..729702d92191e 100755
--- a/sbin/stop-master.sh
+++ b/sbin/stop-master.sh
@@ -17,7 +17,7 @@
# limitations under the License.
#
-# Starts the master on the machine this script is executed on.
+# Stops the master on the machine this script is executed on.
sbin=`dirname "$0"`
sbin=`cd "$sbin"; pwd`
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index a1947fb022e54..8ad026dbdf8ff 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index c363a5efacde8..54ab13ca352d2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -385,7 +385,7 @@ class SqlParser extends AbstractSparkSQLParser {
protected lazy val dotExpressionHeader: Parser[Expression] =
(ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ {
- case i1 ~ i2 ~ rest => UnresolvedAttribute(i1 + "." + i2 + rest.mkString(".", ".", ""))
+ case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString("."))
}
protected lazy val dataType: Parser[DataType] =
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e4e542562f22d..7753331748d7b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -310,7 +310,7 @@ class Analyzer(catalog: Catalog,
}
/**
- * In many dialects of SQL is it valid to sort by attributes that are not present in the SELECT
+ * In many dialects of SQL it is valid to sort by attributes that are not present in the SELECT
* clause. This rule detects such queries and adds the required attributes to the original
* projection, so that they will be available during sorting. Another projection is added to
* remove these attributes after sorting.
@@ -321,7 +321,8 @@ class Analyzer(catalog: Catalog,
if !s.resolved && p.resolved =>
val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name })
val resolved = unresolved.flatMap(child.resolve(_, resolver))
- val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a })
+ val requiredAttributes =
+ AttributeSet(resolved.flatMap(_.collect { case a: Attribute => a }))
val missingInProject = requiredAttributes -- p.output
if (missingInProject.nonEmpty) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
index 92d322845f5c5..bf39603d13bd5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala
@@ -240,7 +240,6 @@ object DataType {
/**
* :: DeveloperApi ::
- *
* The base type of all Spark SQL data types.
*
* @group dataType
@@ -282,7 +281,6 @@ abstract class DataType {
/**
* :: DeveloperApi ::
- *
* The data type representing `NULL` values. Please use the singleton [[DataTypes.NullType]].
*
* @group dataType
@@ -342,7 +340,6 @@ protected[sql] abstract class NativeType extends DataType {
/**
* :: DeveloperApi ::
- *
* The data type representing `String` values. Please use the singleton [[DataTypes.StringType]].
*
* @group dataType
@@ -369,7 +366,6 @@ case object StringType extends StringType
/**
* :: DeveloperApi ::
- *
* The data type representing `Array[Byte]` values.
* Please use the singleton [[DataTypes.BinaryType]].
*
@@ -405,7 +401,6 @@ case object BinaryType extends BinaryType
/**
* :: DeveloperApi ::
- *
* The data type representing `Boolean` values. Please use the singleton [[DataTypes.BooleanType]].
*
*@group dataType
@@ -432,7 +427,6 @@ case object BooleanType extends BooleanType
/**
* :: DeveloperApi ::
- *
* The data type representing `java.sql.Timestamp` values.
* Please use the singleton [[DataTypes.TimestampType]].
*
@@ -464,7 +458,6 @@ case object TimestampType extends TimestampType
/**
* :: DeveloperApi ::
- *
* The data type representing `java.sql.Date` values.
* Please use the singleton [[DataTypes.DateType]].
*
@@ -492,6 +485,12 @@ class DateType private() extends NativeType {
case object DateType extends DateType
+/**
+ * :: DeveloperApi ::
+ * Numeric data types.
+ *
+ * @group dataType
+ */
abstract class NumericType extends NativeType with PrimitiveType {
// Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for
// implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a
@@ -523,7 +522,6 @@ protected[sql] sealed abstract class IntegralType extends NumericType {
/**
* :: DeveloperApi ::
- *
* The data type representing `Long` values. Please use the singleton [[DataTypes.LongType]].
*
* @group dataType
@@ -554,7 +552,6 @@ case object LongType extends LongType
/**
* :: DeveloperApi ::
- *
* The data type representing `Int` values. Please use the singleton [[DataTypes.IntegerType]].
*
* @group dataType
@@ -585,7 +582,6 @@ case object IntegerType extends IntegerType
/**
* :: DeveloperApi ::
- *
* The data type representing `Short` values. Please use the singleton [[DataTypes.ShortType]].
*
* @group dataType
@@ -616,7 +612,6 @@ case object ShortType extends ShortType
/**
* :: DeveloperApi ::
- *
* The data type representing `Byte` values. Please use the singleton [[DataTypes.ByteType]].
*
* @group dataType
@@ -666,7 +661,6 @@ case class PrecisionInfo(precision: Int, scale: Int)
/**
* :: DeveloperApi ::
- *
* The data type representing `java.math.BigDecimal` values.
* A Decimal that might have fixed precision and scale, or unlimited values for these.
*
@@ -745,7 +739,6 @@ object DecimalType {
/**
* :: DeveloperApi ::
- *
* The data type representing `Double` values. Please use the singleton [[DataTypes.DoubleType]].
*
* @group dataType
@@ -775,7 +768,6 @@ case object DoubleType extends DoubleType
/**
* :: DeveloperApi ::
- *
* The data type representing `Float` values. Please use the singleton [[DataTypes.FloatType]].
*
* @group dataType
@@ -811,7 +803,6 @@ object ArrayType {
/**
* :: DeveloperApi ::
- *
* The data type for collections of multiple values.
* Internally these are represented as columns that contain a ``scala.collection.Seq``.
*
@@ -854,7 +845,6 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT
/**
* A field inside a StructType.
- *
* @param name The name of this field.
* @param dataType The data type of this field.
* @param nullable Indicates if values of this field can be `null` values.
@@ -949,7 +939,6 @@ object StructType {
/**
* :: DeveloperApi ::
- *
* A [[StructType]] object can be constructed by
* {{{
* StructType(fields: Seq[StructField])
@@ -1118,7 +1107,6 @@ object MapType {
/**
* :: DeveloperApi ::
- *
* The data type for Maps. Keys in a map are not allowed to have `null` values.
*
* Please use [[DataTypes.createMapType()]] to create a specific instance.
diff --git a/sql/core/pom.xml b/sql/core/pom.xml
index d4c8c687b67bd..3640104e497d4 100644
--- a/sql/core/pom.xml
+++ b/sql/core/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index ce800e0754559..9c49e84bf9680 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -542,20 +542,16 @@ class SQLContext(@transient val sparkContext: SparkContext)
* @group specificdata
*/
@Experimental
- def jsonFile(path: String, schema: StructType): DataFrame = {
- val json = sparkContext.textFile(path)
- jsonRDD(json, schema)
- }
+ def jsonFile(path: String, schema: StructType): DataFrame =
+ load("json", schema, Map("path" -> path))
/**
* :: Experimental ::
* @group specificdata
*/
@Experimental
- def jsonFile(path: String, samplingRatio: Double): DataFrame = {
- val json = sparkContext.textFile(path)
- jsonRDD(json, samplingRatio)
- }
+ def jsonFile(path: String, samplingRatio: Double): DataFrame =
+ load("json", Map("path" -> path, "samplingRatio" -> samplingRatio.toString))
/**
* Loads an RDD[String] storing JSON objects (one object per record), returning the result as a
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
index 02e5b015e8ec2..3f97a11ceb97d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala
@@ -34,10 +34,13 @@ import org.apache.spark.sql.execution.SparkPlan
package object sql {
/**
- * Converts a logical plan into zero or more SparkPlans.
+ * Converts a logical plan into zero or more SparkPlans. This API is exposed for experimenting
+ * with the query planner and is not designed to be stable across spark releases. Developers
+ * writing libraries should instead consider using the stable APIs provided in
+ * [[org.apache.spark.sql.sources]]
*/
@DeveloperApi
- protected[sql] type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan]
+ type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan]
/**
* Type alias for [[DataFrame]]. Kept here for backward source compatibility for Scala.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 097bf0dd23c89..4dedcd365f6cc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -1049,4 +1049,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll {
rdd.toDF().registerTempTable("distinctData")
checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), Row(2))
}
+
+ test("SPARK-6145: ORDER BY test for nested fields") {
+ jsonRDD(sparkContext.makeRDD(
+ """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)).registerTempTable("nestedOrder")
+ // These should be successfully analyzed
+ sql("SELECT 1 FROM nestedOrder ORDER BY a.b").queryExecution.analyzed
+ sql("SELECT a.b FROM nestedOrder ORDER BY a.b").queryExecution.analyzed
+ sql("SELECT 1 FROM nestedOrder ORDER BY a.a.a").queryExecution.analyzed
+ sql("SELECT 1 FROM nestedOrder ORDER BY c[0].d").queryExecution.analyzed
+ }
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
index 9d94d3406acfb..0c21f725f0b49 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala
@@ -19,6 +19,8 @@ package org.apache.spark.sql.json
import java.sql.{Date, Timestamp}
+import org.scalactic.Tolerance._
+
import org.apache.spark.sql.TestData._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.functions._
@@ -551,6 +553,32 @@ class JsonSuite extends QueryTest {
jsonDF.registerTempTable("jsonTable")
}
+ test("jsonFile should be based on JSONRelation") {
+ val file = getTempFilePath("json")
+ val path = file.toString
+ sparkContext.parallelize(1 to 100).map(i => s"""{"a": 1, "b": "str$i"}""").saveAsTextFile(path)
+ val jsonDF = jsonFile(path, 0.49)
+
+ val analyzed = jsonDF.queryExecution.analyzed
+ assert(
+ analyzed.isInstanceOf[LogicalRelation],
+ "The DataFrame returned by jsonFile should be based on JSONRelation.")
+ val relation = analyzed.asInstanceOf[LogicalRelation].relation
+ assert(
+ relation.isInstanceOf[JSONRelation],
+ "The DataFrame returned by jsonFile should be based on JSONRelation.")
+ assert(relation.asInstanceOf[JSONRelation].path === path)
+ assert(relation.asInstanceOf[JSONRelation].samplingRatio === (0.49 +- 0.001))
+
+ val schema = StructType(StructField("a", LongType, true) :: Nil)
+ val logicalRelation =
+ jsonFile(path, schema).queryExecution.analyzed.asInstanceOf[LogicalRelation]
+ val relationWithSchema = logicalRelation.relation.asInstanceOf[JSONRelation]
+ assert(relationWithSchema.path === path)
+ assert(relationWithSchema.schema === schema)
+ assert(relationWithSchema.samplingRatio > 0.99)
+ }
+
test("Loading a JSON dataset from a text file") {
val file = getTempFilePath("json")
val path = file.toString
diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml
index 123a1f629ab1c..f466a3c0b5dc2 100644
--- a/sql/hive-thriftserver/pom.xml
+++ b/sql/hive-thriftserver/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
@@ -41,6 +41,11 @@
spark-hive_${scala.binary.version}
${project.version}
+
+ com.google.guava
+ guava
+ runtime
+
${hive.group}
hive-cli
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 72c474d66055c..0e3f4eb98cbf7 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -21,7 +21,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../../pom.xml
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 1e92ba686a57d..0370b0e9e1aa3 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -20,7 +20,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala
index 985ded9111f74..6bdfe45dc7f83 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala
@@ -20,6 +20,7 @@ import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.{Await, ExecutionContext, Future}
+import scala.language.postfixOps
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
diff --git a/tools/pom.xml b/tools/pom.xml
index e7419ed2c607a..181236d1bcbf6 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 65344aa8738e0..c13534f0410a1 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -19,7 +19,7 @@
4.0.0
org.apache.spark
- spark-parent
+ spark-parent_2.10
1.3.0-SNAPSHOT
../pom.xml
|