spark git commit: [SPARK-12349][ML] Make spark.ml PCAModel load backwards compatible

2015-12-21 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master ce1798b3a -> d0f695089


[SPARK-12349][ML] Make spark.ml PCAModel load backwards compatible

Only load explainedVariance in PCAModel if it was written with Spark > 1.6.x
jkbradley is this kind of what you had in mind?

Author: Sean Owen <so...@cloudera.com>

Closes #10327 from srowen/SPARK-12349.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d0f69508
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d0f69508
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d0f69508

Branch: refs/heads/master
Commit: d0f695089e4627273133c5f49ef7a83c1840c8f5
Parents: ce1798b
Author: Sean Owen <so...@cloudera.com>
Authored: Mon Dec 21 10:21:22 2015 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Mon Dec 21 10:21:22 2015 +

--
 .../scala/org/apache/spark/ml/feature/PCA.scala | 33 +---
 1 file changed, 28 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d0f69508/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala 
b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
index 53d33ea..759be81 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala
@@ -167,14 +167,37 @@ object PCAModel extends MLReadable[PCAModel] {
 
 private val className = classOf[PCAModel].getName
 
+/**
+ * Loads a [[PCAModel]] from data located at the input path. Note that the 
model includes an
+ * `explainedVariance` member that is not recorded by Spark 1.6 and 
earlier. A model
+ * can be loaded from such older data but will have an empty vector for
+ * `explainedVariance`.
+ *
+ * @param path path to serialized model data
+ * @return a [[PCAModel]]
+ */
 override def load(path: String): PCAModel = {
   val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
+
+  // explainedVariance field is not present in Spark <= 1.6
+  val versionRegex = "([0-9]+)\\.([0-9])+.*".r
+  val hasExplainedVariance = metadata.sparkVersion match {
+case versionRegex(major, minor) =>
+  (major.toInt >= 2 || (major.toInt == 1 && minor.toInt > 6))
+case _ => false
+  }
+
   val dataPath = new Path(path, "data").toString
-  val Row(pc: DenseMatrix, explainedVariance: DenseVector) =
-sqlContext.read.parquet(dataPath)
-.select("pc", "explainedVariance")
-.head()
-  val model = new PCAModel(metadata.uid, pc, explainedVariance)
+  val model = if (hasExplainedVariance) {
+val Row(pc: DenseMatrix, explainedVariance: DenseVector) =
+  sqlContext.read.parquet(dataPath)
+.select("pc", "explainedVariance")
+.head()
+new PCAModel(metadata.uid, pc, explainedVariance)
+  } else {
+val Row(pc: DenseMatrix) = 
sqlContext.read.parquet(dataPath).select("pc").head()
+new PCAModel(metadata.uid, pc, 
Vectors.dense(Array.empty[Double]).asInstanceOf[DenseVector])
+  }
   DefaultParamsReader.getAndSetParams(model, metadata)
   model
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12311][CORE] Restore previous value of "os.arch" property in test suites after forcing to set specific value to "os.arch" property

2015-12-24 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9e85bb71a -> 392046611


[SPARK-12311][CORE] Restore previous value of "os.arch" property in test suites 
after forcing to set specific value to "os.arch" property

Restore the original value of os.arch property after each test

Since some of tests forced to set the specific value to os.arch property, we 
need to set the original value.

Author: Kazuaki Ishizaki 

Closes #10289 from kiszk/SPARK-12311.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/39204661
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/39204661
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/39204661

Branch: refs/heads/master
Commit: 392046611837a3a740ff97fa8177ca7c12316fb7
Parents: 9e85bb7
Author: Kazuaki Ishizaki 
Authored: Thu Dec 24 13:37:28 2015 +
Committer: Sean Owen 
Committed: Thu Dec 24 13:37:28 2015 +

--
 .../sort/IndexShuffleBlockResolverSuite.scala   |  7 +++-
 .../org/apache/spark/CheckpointSuite.scala  |  7 ++--
 .../spark/ExternalShuffleServiceSuite.scala |  7 +++-
 .../org/apache/spark/FileServerSuite.scala  |  7 ++--
 .../test/scala/org/apache/spark/FileSuite.scala |  7 ++--
 .../org/apache/spark/HashShuffleSuite.scala |  1 +
 .../apache/spark/HeartbeatReceiverSuite.scala   |  1 +
 .../org/apache/spark/JobCancellationSuite.scala |  7 ++--
 .../org/apache/spark/LocalSparkContext.scala|  9 +++--
 .../org/apache/spark/SecurityManagerSuite.scala |  4 +--
 .../org/apache/spark/SharedSparkContext.scala   | 11 +++---
 .../org/apache/spark/ShuffleNettySuite.scala|  1 +
 .../org/apache/spark/SortShuffleSuite.scala |  2 ++
 .../StandaloneDynamicAllocationSuite.scala  | 21 +++-
 .../spark/deploy/client/AppClientSuite.scala| 21 +++-
 .../deploy/history/HistoryServerSuite.scala |  3 +-
 .../deploy/rest/StandaloneRestSubmitSuite.scala |  8 +++--
 .../input/WholeTextFileRecordReaderSuite.scala  |  7 +++-
 .../netty/NettyBlockTransferServiceSuite.scala  | 18 ++
 .../apache/spark/rdd/AsyncRDDActionsSuite.scala |  9 +++--
 .../apache/spark/rdd/LocalCheckpointSuite.scala |  1 +
 .../org/apache/spark/rpc/RpcEnvSuite.scala  | 11 --
 .../serializer/SerializationDebuggerSuite.scala |  1 +
 .../BypassMergeSortShuffleWriterSuite.scala | 11 --
 .../spark/storage/BlockManagerSuite.scala   | 35 +++-
 .../spark/storage/DiskBlockManagerSuite.scala   | 16 ++---
 .../storage/DiskBlockObjectWriterSuite.scala|  7 +++-
 .../org/apache/spark/ui/UISeleniumSuite.scala   |  9 +++--
 .../spark/util/ClosureCleanerSuite2.scala   | 11 --
 .../apache/spark/util/SizeEstimatorSuite.scala  |  4 +++
 .../ml/source/libsvm/LibSVMRelationSuite.scala  |  7 ++--
 .../apache/spark/ml/util/TempDirectory.scala|  7 ++--
 .../mllib/util/LocalClusterSparkContext.scala   | 11 +++---
 .../mllib/util/MLlibTestSparkContext.scala  | 15 +
 .../spark/repl/ExecutorClassLoaderSuite.scala   | 15 +
 .../spark/streaming/CheckpointSuite.scala   | 15 ++---
 .../spark/streaming/DStreamClosureSuite.scala   |  9 +++--
 .../spark/streaming/DStreamScopeSuite.scala |  7 +++-
 .../spark/streaming/MapWithStateSuite.scala |  9 +++--
 .../streaming/ReceiverInputDStreamSuite.scala   |  6 +++-
 .../spark/streaming/UISeleniumSuite.scala   |  9 +++--
 .../streaming/rdd/MapWithStateRDDSuite.scala| 11 --
 .../rdd/WriteAheadLogBackedBlockRDDSuite.scala  | 16 +++--
 .../streaming/util/WriteAheadLogSuite.scala |  9 +++--
 .../deploy/yarn/BaseYarnClusterSuite.scala  | 13 ++--
 .../apache/spark/deploy/yarn/ClientSuite.scala  | 18 --
 .../spark/deploy/yarn/YarnAllocatorSuite.scala  |  7 +++-
 .../deploy/yarn/YarnSparkHadoopUtilSuite.scala  |  5 +--
 .../network/yarn/YarnShuffleServiceSuite.scala  | 27 +--
 49 files changed, 338 insertions(+), 142 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/39204661/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
--
diff --git 
a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
 
b/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
index 0b19861..f200ff3 100644
--- 
a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
+++ 
b/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
@@ -42,6 +42,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite 
with BeforeAndAfterEa
   private val conf: SparkConf = new SparkConf(loadDefaults = false)
 
   override def 

spark git commit: [SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT syntax

2015-12-24 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 392046611 -> 502476e45


[SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT 
syntax

In the past Spark JDBC write only worked with technologies which support the 
following INSERT statement syntax (JdbcUtils.scala: insertStatement()):

INSERT INTO $table VALUES ( ?, ?, ..., ? )

But some technologies require a list of column names:

INSERT INTO $table ( $colNameList ) VALUES ( ?, ?, ..., ? )

This was blocking the use of e.g. the Progress JDBC Driver for Cassandra.

Another limitation is that syntax 1 relies no the dataframe field ordering 
match that of the target table. This works fine, as long as the target table 
has been created by writer.jdbc().

If the target table contains more columns (not created by writer.jdbc()), then 
the insert fails due mismatch of number of columns or their data types.

This PR switches to the recommended second INSERT syntax. Column names are 
taken from datafram field names.

Author: CK50 

Closes #10380 from CK50/master-SPARK-12010-2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/502476e4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/502476e4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/502476e4

Branch: refs/heads/master
Commit: 502476e45c314a1229b3bce1c61f5cb94a9fc04b
Parents: 3920466
Author: CK50 
Authored: Thu Dec 24 13:39:11 2015 +
Committer: Sean Owen 
Committed: Thu Dec 24 13:39:11 2015 +

--
 .../sql/execution/datasources/jdbc/JdbcUtils.scala  | 12 
 1 file changed, 4 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/502476e4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index 252f1cf..28cd688 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -63,14 +63,10 @@ object JdbcUtils extends Logging {
* Returns a PreparedStatement that inserts a row into table via conn.
*/
   def insertStatement(conn: Connection, table: String, rddSchema: StructType): 
PreparedStatement = {
-val sql = new StringBuilder(s"INSERT INTO $table VALUES (")
-var fieldsLeft = rddSchema.fields.length
-while (fieldsLeft > 0) {
-  sql.append("?")
-  if (fieldsLeft > 1) sql.append(", ") else sql.append(")")
-  fieldsLeft = fieldsLeft - 1
-}
-conn.prepareStatement(sql.toString())
+val columns = rddSchema.fields.map(_.name).mkString(",")
+val placeholders = rddSchema.fields.map(_ => "?").mkString(",")
+val sql = s"INSERT INTO $table ($columns) VALUES ($placeholders)"
+conn.prepareStatement(sql)
   }
 
   /**


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT syntax

2015-12-24 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 4dd8712c1 -> 865dd8bcc


[SPARK-12010][SQL] Spark JDBC requires support for column-name-free INSERT 
syntax

In the past Spark JDBC write only worked with technologies which support the 
following INSERT statement syntax (JdbcUtils.scala: insertStatement()):

INSERT INTO $table VALUES ( ?, ?, ..., ? )

But some technologies require a list of column names:

INSERT INTO $table ( $colNameList ) VALUES ( ?, ?, ..., ? )

This was blocking the use of e.g. the Progress JDBC Driver for Cassandra.

Another limitation is that syntax 1 relies no the dataframe field ordering 
match that of the target table. This works fine, as long as the target table 
has been created by writer.jdbc().

If the target table contains more columns (not created by writer.jdbc()), then 
the insert fails due mismatch of number of columns or their data types.

This PR switches to the recommended second INSERT syntax. Column names are 
taken from datafram field names.

Author: CK50 

Closes #10380 from CK50/master-SPARK-12010-2.

(cherry picked from commit 502476e45c314a1229b3bce1c61f5cb94a9fc04b)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/865dd8bc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/865dd8bc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/865dd8bc

Branch: refs/heads/branch-1.6
Commit: 865dd8bccfc994310ad6664151d469043706ef3b
Parents: 4dd8712
Author: CK50 
Authored: Thu Dec 24 13:39:11 2015 +
Committer: Sean Owen 
Committed: Thu Dec 24 13:41:35 2015 +

--
 .../sql/execution/datasources/jdbc/JdbcUtils.scala  | 12 
 1 file changed, 4 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/865dd8bc/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index 252f1cf..28cd688 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -63,14 +63,10 @@ object JdbcUtils extends Logging {
* Returns a PreparedStatement that inserts a row into table via conn.
*/
   def insertStatement(conn: Connection, table: String, rddSchema: StructType): 
PreparedStatement = {
-val sql = new StringBuilder(s"INSERT INTO $table VALUES (")
-var fieldsLeft = rddSchema.fields.length
-while (fieldsLeft > 0) {
-  sql.append("?")
-  if (fieldsLeft > 1) sql.append(", ") else sql.append(")")
-  fieldsLeft = fieldsLeft - 1
-}
-conn.prepareStatement(sql.toString())
+val columns = rddSchema.fields.map(_.name).mkString(",")
+val placeholders = rddSchema.fields.map(_ => "?").mkString(",")
+val sql = s"INSERT INTO $table ($columns) VALUES ($placeholders)"
+conn.prepareStatement(sql)
   }
 
   /**


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12440][CORE] Avoid setCheckpoint warning when directory is not local

2015-12-24 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 502476e45 -> ea4aab7e8


[SPARK-12440][CORE] Avoid setCheckpoint warning when directory is not local

In SparkContext method `setCheckpointDir`, a warning is issued when spark 
master is not local and the passed directory for the checkpoint dir appears to 
be local.

In practice, when relying on HDFS configuration file and using a relative path 
for the checkpoint directory (using an incomplete URI without HDFS scheme, 
...), this warning should not be issued and might be confusing.
In fact, in this case, the checkpoint directory is successfully created, and 
the checkpointing mechanism works as expected.

This PR uses the `FileSystem` instance created with the given directory, and 
checks whether it is local or not.
(The rationale is that since this same `FileSystem` instance is used to create 
the checkpoint dir anyway and can therefore be reliably used to determine if it 
is local or not).

The warning is only issued if the directory is not local, on top of the 
existing conditions.

Author: pierre-borckmans 

Closes #10392 from pierre-borckmans/SPARK-12440_CheckpointDir_Warning_NonLocal.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ea4aab7e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ea4aab7e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ea4aab7e

Branch: refs/heads/master
Commit: ea4aab7e87fbcf9ac90f93af79cc892b56508aa0
Parents: 502476e
Author: pierre-borckmans 
Authored: Thu Dec 24 13:48:21 2015 +
Committer: Sean Owen 
Committed: Thu Dec 24 13:48:21 2015 +

--
 core/src/main/scala/org/apache/spark/SparkContext.scala | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ea4aab7e/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 67230f4..d506782 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -2073,8 +2073,9 @@ class SparkContext(config: SparkConf) extends Logging 
with ExecutorAllocationCli
 // its own local file system, which is incorrect because the checkpoint 
files
 // are actually on the executor machines.
 if (!isLocal && Utils.nonLocalPaths(directory).isEmpty) {
-  logWarning("Checkpoint directory must be non-local " +
-"if Spark is running on a cluster: " + directory)
+  logWarning("Spark is not running in local mode, therefore the checkpoint 
directory " +
+s"must not be on the local filesystem. Directory '$directory' " +
+"appears to be on the local filesystem.")
 }
 
 checkpointDir = Option(directory).map { dir =>


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12263][DOCS] IllegalStateException: Memory can't be 0 for SPARK_WORKER_MEMORY without unit

2015-12-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 27af6157f -> 932cf4424


[SPARK-12263][DOCS] IllegalStateException: Memory can't be 0 for 
SPARK_WORKER_MEMORY without unit

Updated the Worker Unit IllegalStateException message to indicate no values 
less than 1MB instead of 0 to help solve this.
Requesting review

Author: Neelesh Srinivas Salian 

Closes #10483 from nssalian/SPARK-12263.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/932cf442
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/932cf442
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/932cf442

Branch: refs/heads/master
Commit: 932cf44248e067ee7cae6fef79ddf2ab9b1c36d8
Parents: 27af615
Author: Neelesh Srinivas Salian 
Authored: Wed Dec 30 11:14:13 2015 +
Committer: Sean Owen 
Committed: Wed Dec 30 11:14:13 2015 +

--
 .../scala/org/apache/spark/deploy/worker/WorkerArguments.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/932cf442/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala 
b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 5181142..de3c7cd 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -175,7 +175,7 @@ private[worker] class WorkerArguments(args: Array[String], 
conf: SparkConf) {
 
   def checkWorkerMemory(): Unit = {
 if (memory <= 0) {
-  val message = "Memory can't be 0, missing a M or G on the end of the 
memory specification?"
+  val message = "Memory is below 1MB, or missing a M/G at the end of the 
memory specification?"
   throw new IllegalStateException(message)
 }
   }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11860][PYSAPRK][DOCUMENTATION] Invalid argument specification …

2015-11-25 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 638500265 -> b9b6fbe89


[SPARK-11860][PYSAPRK][DOCUMENTATION] Invalid argument specification …

…for registerFunction [Python]

Straightforward change on the python doc

Author: Jeff Zhang 

Closes #9901 from zjffdu/SPARK-11860.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b9b6fbe8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b9b6fbe8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b9b6fbe8

Branch: refs/heads/master
Commit: b9b6fbe89b6d1a890faa02c1a53bb670a6255362
Parents: 6385002
Author: Jeff Zhang 
Authored: Wed Nov 25 13:49:58 2015 +
Committer: Sean Owen 
Committed: Wed Nov 25 13:49:58 2015 +

--
 python/pyspark/sql/context.py | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b9b6fbe8/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 5a85ac3..a49c1b5 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -195,14 +195,15 @@ class SQLContext(object):
 @ignore_unicode_prefix
 @since(1.2)
 def registerFunction(self, name, f, returnType=StringType()):
-"""Registers a lambda function as a UDF so it can be used in SQL 
statements.
+"""Registers a python function (including lambda function) as a UDF
+so it can be used in SQL statements.
 
 In addition to a name and the function itself, the return type can be 
optionally specified.
 When the return type is not given it default to a string and 
conversion will automatically
 be done.  For any other return type, the produced object must match 
the specified type.
 
 :param name: name of the UDF
-:param samplingRatio: lambda function
+:param f: python function
 :param returnType: a :class:`DataType` object
 
 >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x))


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11860][PYSAPRK][DOCUMENTATION] Invalid argument specification …

2015-11-25 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 a986a3bde -> 4971eaaa5


[SPARK-11860][PYSAPRK][DOCUMENTATION] Invalid argument specification …

…for registerFunction [Python]

Straightforward change on the python doc

Author: Jeff Zhang 

Closes #9901 from zjffdu/SPARK-11860.

(cherry picked from commit b9b6fbe89b6d1a890faa02c1a53bb670a6255362)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4971eaaa
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4971eaaa
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4971eaaa

Branch: refs/heads/branch-1.6
Commit: 4971eaaa5768ad20c5a77278e435c97409a9ca8f
Parents: a986a3b
Author: Jeff Zhang 
Authored: Wed Nov 25 13:49:58 2015 +
Committer: Sean Owen 
Committed: Wed Nov 25 13:50:10 2015 +

--
 python/pyspark/sql/context.py | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4971eaaa/python/pyspark/sql/context.py
--
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 5a85ac3..a49c1b5 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -195,14 +195,15 @@ class SQLContext(object):
 @ignore_unicode_prefix
 @since(1.2)
 def registerFunction(self, name, f, returnType=StringType()):
-"""Registers a lambda function as a UDF so it can be used in SQL 
statements.
+"""Registers a python function (including lambda function) as a UDF
+so it can be used in SQL statements.
 
 In addition to a name and the function itself, the return type can be 
optionally specified.
 When the return type is not given it default to a string and 
conversion will automatically
 be done.  For any other return type, the produced object must match 
the specified type.
 
 :param name: name of the UDF
-:param samplingRatio: lambda function
+:param f: python function
 :param returnType: a :class:`DataType` object
 
 >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x))


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11686][CORE] Issue WARN when dynamic allocation is disabled due to spark.dynamicAllocation.enabled and spark.executor.instances both set

2015-11-25 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 997896643 -> a986a3bde


[SPARK-11686][CORE] Issue WARN when dynamic allocation is disabled due to 
spark.dynamicAllocation.enabled and spark.executor.instances both set

Changed the log type to a 'warning' instead of 'info' as required.

Author: Ashwin Swaroop 

Closes #9926 from ashwinswaroop/master.

(cherry picked from commit 63850026576b3ea7783f9d4b975171dc3cff6e4c)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a986a3bd
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a986a3bd
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a986a3bd

Branch: refs/heads/branch-1.6
Commit: a986a3bde7426fbbc6b95848dcc55f1d6f22a1b1
Parents: 9978966
Author: Ashwin Swaroop 
Authored: Wed Nov 25 13:41:14 2015 +
Committer: Sean Owen 
Committed: Wed Nov 25 13:41:26 2015 +

--
 core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a986a3bd/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index e19ba11..2c10779 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -556,7 +556,7 @@ class SparkContext(config: SparkConf) extends Logging with 
ExecutorAllocationCli
 // Optionally scale number of executors dynamically based on workload. 
Exposed for testing.
 val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(_conf)
 if (!dynamicAllocationEnabled && 
_conf.getBoolean("spark.dynamicAllocation.enabled", false)) {
-  logInfo("Dynamic Allocation and num executors both set, thus dynamic 
allocation disabled.")
+  logWarning("Dynamic Allocation and num executors both set, thus dynamic 
allocation disabled.")
 }
 
 _executorAllocationManager =


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11686][CORE] Issue WARN when dynamic allocation is disabled due to spark.dynamicAllocation.enabled and spark.executor.instances both set

2015-11-25 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master a0f1a1183 -> 638500265


[SPARK-11686][CORE] Issue WARN when dynamic allocation is disabled due to 
spark.dynamicAllocation.enabled and spark.executor.instances both set

Changed the log type to a 'warning' instead of 'info' as required.

Author: Ashwin Swaroop 

Closes #9926 from ashwinswaroop/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/63850026
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/63850026
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/63850026

Branch: refs/heads/master
Commit: 63850026576b3ea7783f9d4b975171dc3cff6e4c
Parents: a0f1a11
Author: Ashwin Swaroop 
Authored: Wed Nov 25 13:41:14 2015 +
Committer: Sean Owen 
Committed: Wed Nov 25 13:41:14 2015 +

--
 core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/63850026/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index e19ba11..2c10779 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -556,7 +556,7 @@ class SparkContext(config: SparkConf) extends Logging with 
ExecutorAllocationCli
 // Optionally scale number of executors dynamically based on workload. 
Exposed for testing.
 val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(_conf)
 if (!dynamicAllocationEnabled && 
_conf.getBoolean("spark.dynamicAllocation.enabled", false)) {
-  logInfo("Dynamic Allocation and num executors both set, thus dynamic 
allocation disabled.")
+  logWarning("Dynamic Allocation and num executors both set, thus dynamic 
allocation disabled.")
 }
 
 _executorAllocationManager =


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11859][MESOS] SparkContext accepts invalid Master URLs in the form zk://host:port for a multi-master Mesos cluster using ZooKeeper

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 0ddfe7868 -> e07494420


[SPARK-11859][MESOS] SparkContext accepts invalid Master URLs in the form 
zk://host:port for a multi-master Mesos cluster using ZooKeeper

* According to below doc and validation logic in 
[SparkSubmit.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L231),
 master URL for a mesos cluster should always start with `mesos://`

http://spark.apache.org/docs/latest/running-on-mesos.html
`The Master URLs for Mesos are in the form mesos://host:5050 for a 
single-master Mesos cluster, or mesos://zk://host:2181 for a multi-master Mesos 
cluster using ZooKeeper.`

* However, 
[SparkContext.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L2749)
 fails the validation and can receive master URL in the form `zk://host:port`

* For the master URLs in the form `zk:host:port`, the valid form should be 
`mesos://zk://host:port`

* This PR restrict the validation in `SparkContext.scala`, and now only mesos 
master URLs prefixed with `mesos://` can be accepted.

* This PR also updated corresponding unit test.

Author: toddwan 

Closes #9886 from toddwan/S11859.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e0749442
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e0749442
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e0749442

Branch: refs/heads/master
Commit: e0749442051d6e29dae4f4cdcb2937c0b015f98f
Parents: 0ddfe78
Author: toddwan 
Authored: Mon Nov 30 09:26:29 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 09:26:29 2015 +

--
 .../main/scala/org/apache/spark/SparkContext.scala  | 16 ++--
 .../spark/SparkContextSchedulerCreationSuite.scala  |  5 +
 2 files changed, 15 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e0749442/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index b030d3c..8a62b71 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -2708,15 +2708,14 @@ object SparkContext extends Logging {
 scheduler.initialize(backend)
 (backend, scheduler)
 
-  case mesosUrl @ MESOS_REGEX(_) =>
+  case MESOS_REGEX(mesosUrl) =>
 MesosNativeLibrary.load()
 val scheduler = new TaskSchedulerImpl(sc)
 val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", 
defaultValue = true)
-val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw 
Mesos URLs
 val backend = if (coarseGrained) {
-  new CoarseMesosSchedulerBackend(scheduler, sc, url, 
sc.env.securityManager)
+  new CoarseMesosSchedulerBackend(scheduler, sc, mesosUrl, 
sc.env.securityManager)
 } else {
-  new MesosSchedulerBackend(scheduler, sc, url)
+  new MesosSchedulerBackend(scheduler, sc, mesosUrl)
 }
 scheduler.initialize(backend)
 (backend, scheduler)
@@ -2727,6 +2726,11 @@ object SparkContext extends Logging {
 scheduler.initialize(backend)
 (backend, scheduler)
 
+  case zkUrl if zkUrl.startsWith("zk://") =>
+logWarning("Master URL for a multi-master Mesos cluster managed by 
ZooKeeper should be " +
+  "in the form mesos://zk://host:port. Current Master URL will stop 
working in Spark 2.0.")
+createTaskScheduler(sc, "mesos://" + zkUrl)
+
   case _ =>
 throw new SparkException("Could not parse Master URL: '" + master + 
"'")
 }
@@ -2745,8 +2749,8 @@ private object SparkMasterRegex {
   val LOCAL_CLUSTER_REGEX = 
"""local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
   // Regular expression for connecting to Spark deploy clusters
   val SPARK_REGEX = """spark://(.*)""".r
-  // Regular expression for connection to Mesos cluster by mesos:// or zk:// 
url
-  val MESOS_REGEX = """(mesos|zk)://.*""".r
+  // Regular expression for connection to Mesos cluster by mesos:// or 
mesos://zk:// url
+  val MESOS_REGEX = """mesos://(.*)""".r
   // Regular expression for connection to Simr cluster
   val SIMR_REGEX = """simr://(.*)""".r
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/e0749442/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala 

spark git commit: [SPARK-11859][MESOS] SparkContext accepts invalid Master URLs in the form zk://host:port for a multi-master Mesos cluster using ZooKeeper

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 a4a2a7deb -> 12d97b0c5


[SPARK-11859][MESOS] SparkContext accepts invalid Master URLs in the form 
zk://host:port for a multi-master Mesos cluster using ZooKeeper

* According to below doc and validation logic in 
[SparkSubmit.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L231),
 master URL for a mesos cluster should always start with `mesos://`

http://spark.apache.org/docs/latest/running-on-mesos.html
`The Master URLs for Mesos are in the form mesos://host:5050 for a 
single-master Mesos cluster, or mesos://zk://host:2181 for a multi-master Mesos 
cluster using ZooKeeper.`

* However, 
[SparkContext.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L2749)
 fails the validation and can receive master URL in the form `zk://host:port`

* For the master URLs in the form `zk:host:port`, the valid form should be 
`mesos://zk://host:port`

* This PR restrict the validation in `SparkContext.scala`, and now only mesos 
master URLs prefixed with `mesos://` can be accepted.

* This PR also updated corresponding unit test.

Author: toddwan 

Closes #9886 from toddwan/S11859.

(cherry picked from commit e0749442051d6e29dae4f4cdcb2937c0b015f98f)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/12d97b0c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/12d97b0c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/12d97b0c

Branch: refs/heads/branch-1.6
Commit: 12d97b0c5213e04453f81156f04ed95d877f199c
Parents: a4a2a7d
Author: toddwan 
Authored: Mon Nov 30 09:26:29 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 09:26:37 2015 +

--
 .../main/scala/org/apache/spark/SparkContext.scala  | 16 ++--
 .../spark/SparkContextSchedulerCreationSuite.scala  |  5 +
 2 files changed, 15 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/12d97b0c/core/src/main/scala/org/apache/spark/SparkContext.scala
--
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index b030d3c..8a62b71 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -2708,15 +2708,14 @@ object SparkContext extends Logging {
 scheduler.initialize(backend)
 (backend, scheduler)
 
-  case mesosUrl @ MESOS_REGEX(_) =>
+  case MESOS_REGEX(mesosUrl) =>
 MesosNativeLibrary.load()
 val scheduler = new TaskSchedulerImpl(sc)
 val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", 
defaultValue = true)
-val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw 
Mesos URLs
 val backend = if (coarseGrained) {
-  new CoarseMesosSchedulerBackend(scheduler, sc, url, 
sc.env.securityManager)
+  new CoarseMesosSchedulerBackend(scheduler, sc, mesosUrl, 
sc.env.securityManager)
 } else {
-  new MesosSchedulerBackend(scheduler, sc, url)
+  new MesosSchedulerBackend(scheduler, sc, mesosUrl)
 }
 scheduler.initialize(backend)
 (backend, scheduler)
@@ -2727,6 +2726,11 @@ object SparkContext extends Logging {
 scheduler.initialize(backend)
 (backend, scheduler)
 
+  case zkUrl if zkUrl.startsWith("zk://") =>
+logWarning("Master URL for a multi-master Mesos cluster managed by 
ZooKeeper should be " +
+  "in the form mesos://zk://host:port. Current Master URL will stop 
working in Spark 2.0.")
+createTaskScheduler(sc, "mesos://" + zkUrl)
+
   case _ =>
 throw new SparkException("Could not parse Master URL: '" + master + 
"'")
 }
@@ -2745,8 +2749,8 @@ private object SparkMasterRegex {
   val LOCAL_CLUSTER_REGEX = 
"""local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
   // Regular expression for connecting to Spark deploy clusters
   val SPARK_REGEX = """spark://(.*)""".r
-  // Regular expression for connection to Mesos cluster by mesos:// or zk:// 
url
-  val MESOS_REGEX = """(mesos|zk)://.*""".r
+  // Regular expression for connection to Mesos cluster by mesos:// or 
mesos://zk:// url
+  val MESOS_REGEX = """mesos://(.*)""".r
   // Regular expression for connection to Simr cluster
   val SIMR_REGEX = """simr://(.*)""".r
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/12d97b0c/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala

spark git commit: [SPARK-12023][BUILD] Fix warnings while packaging spark with maven.

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 aaf835f1d -> 33cd171b2


[SPARK-12023][BUILD] Fix warnings while packaging spark with maven.

this is a trivial fix, discussed 
[here](http://stackoverflow.com/questions/28500401/maven-assembly-plugin-warning-the-assembly-descriptor-contains-a-filesystem-roo/).

Author: Prashant Sharma 

Closes #10014 from ScrapCodes/assembly-warning.

(cherry picked from commit bf0e85a70a54a2d7fd6804b6bd00c63c20e2bb00)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/33cd171b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/33cd171b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/33cd171b

Branch: refs/heads/branch-1.6
Commit: 33cd171b24d45081c8dd1b22a8bd0152a392d115
Parents: aaf835f
Author: Prashant Sharma 
Authored: Mon Nov 30 10:11:27 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 10:11:39 2015 +

--
 assembly/src/main/assembly/assembly.xml  | 8 
 external/mqtt/src/main/assembly/assembly.xml | 2 +-
 2 files changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/33cd171b/assembly/src/main/assembly/assembly.xml
--
diff --git a/assembly/src/main/assembly/assembly.xml 
b/assembly/src/main/assembly/assembly.xml
index 7111563..009d4b9 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -32,7 +32,7 @@
   
 
${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/
   
-  
/ui-resources/org/apache/spark/ui/static
+  
ui-resources/org/apache/spark/ui/static
   
 **/*
   
@@ -41,7 +41,7 @@
   
 ${project.parent.basedir}/sbin/
   
-  /sbin
+  sbin
   
 **/*
   
@@ -50,7 +50,7 @@
   
 ${project.parent.basedir}/bin/
   
-  /bin
+  bin
   
 **/*
   
@@ -59,7 +59,7 @@
   
 ${project.parent.basedir}/assembly/target/${spark.jar.dir}
   
-  /
+  
   
 ${spark.jar.basename}
   

http://git-wip-us.apache.org/repos/asf/spark/blob/33cd171b/external/mqtt/src/main/assembly/assembly.xml
--
diff --git a/external/mqtt/src/main/assembly/assembly.xml 
b/external/mqtt/src/main/assembly/assembly.xml
index ecab5b3..c110b01 100644
--- a/external/mqtt/src/main/assembly/assembly.xml
+++ b/external/mqtt/src/main/assembly/assembly.xml
@@ -24,7 +24,7 @@
   
 
   
${project.build.directory}/scala-${scala.binary.version}/test-classes
-  /
+  
 
   
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12023][BUILD] Fix warnings while packaging spark with maven.

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 26c3581f1 -> bf0e85a70


[SPARK-12023][BUILD] Fix warnings while packaging spark with maven.

this is a trivial fix, discussed 
[here](http://stackoverflow.com/questions/28500401/maven-assembly-plugin-warning-the-assembly-descriptor-contains-a-filesystem-roo/).

Author: Prashant Sharma 

Closes #10014 from ScrapCodes/assembly-warning.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bf0e85a7
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bf0e85a7
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bf0e85a7

Branch: refs/heads/master
Commit: bf0e85a70a54a2d7fd6804b6bd00c63c20e2bb00
Parents: 26c3581
Author: Prashant Sharma 
Authored: Mon Nov 30 10:11:27 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 10:11:27 2015 +

--
 assembly/src/main/assembly/assembly.xml  | 8 
 external/mqtt/src/main/assembly/assembly.xml | 2 +-
 2 files changed, 5 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bf0e85a7/assembly/src/main/assembly/assembly.xml
--
diff --git a/assembly/src/main/assembly/assembly.xml 
b/assembly/src/main/assembly/assembly.xml
index 7111563..009d4b9 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -32,7 +32,7 @@
   
 
${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/
   
-  
/ui-resources/org/apache/spark/ui/static
+  
ui-resources/org/apache/spark/ui/static
   
 **/*
   
@@ -41,7 +41,7 @@
   
 ${project.parent.basedir}/sbin/
   
-  /sbin
+  sbin
   
 **/*
   
@@ -50,7 +50,7 @@
   
 ${project.parent.basedir}/bin/
   
-  /bin
+  bin
   
 **/*
   
@@ -59,7 +59,7 @@
   
 ${project.parent.basedir}/assembly/target/${spark.jar.dir}
   
-  /
+  
   
 ${spark.jar.basename}
   

http://git-wip-us.apache.org/repos/asf/spark/blob/bf0e85a7/external/mqtt/src/main/assembly/assembly.xml
--
diff --git a/external/mqtt/src/main/assembly/assembly.xml 
b/external/mqtt/src/main/assembly/assembly.xml
index ecab5b3..c110b01 100644
--- a/external/mqtt/src/main/assembly/assembly.xml
+++ b/external/mqtt/src/main/assembly/assembly.xml
@@ -24,7 +24,7 @@
   
 
   
${project.build.directory}/scala-${scala.binary.version}/test-classes
-  /
+  
 
   
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11898][MLLIB] Use broadcast for the global tables in Word2Vec

2015-12-01 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9693b0d5a -> a0af0e351


[SPARK-11898][MLLIB] Use broadcast for the global tables in Word2Vec

jira: https://issues.apache.org/jira/browse/SPARK-11898
syn0Global and sync1Global in word2vec are quite large objects with size (vocab 
* vectorSize * 8), yet they are passed to worker using basic task serialization.

Use broadcast can greatly improve the performance. My benchmark shows that, for 
1M vocabulary and default vectorSize 100, changing to broadcast can help,

1. decrease the worker memory consumption by 45%.
2. decrease running time by 40%.

This will also help extend the upper limit for Word2Vec.

Author: Yuhao Yang 

Closes #9878 from hhbyyh/w2vBC.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a0af0e35
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a0af0e35
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a0af0e35

Branch: refs/heads/master
Commit: a0af0e351e45a8be47a6f65efd132eaa4a00c9e4
Parents: 9693b0d
Author: Yuhao Yang 
Authored: Tue Dec 1 09:26:58 2015 +
Committer: Sean Owen 
Committed: Tue Dec 1 09:26:58 2015 +

--
 .../main/scala/org/apache/spark/mllib/feature/Word2Vec.scala  | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a0af0e35/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
index a47f27b..655ac0b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
@@ -316,12 +316,15 @@ class Word2Vec extends Serializable with Logging {
   Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 
0.5f) / vectorSize)
 val syn1Global = new Array[Float](vocabSize * vectorSize)
 var alpha = learningRate
+
 for (k <- 1 to numIterations) {
+  val bcSyn0Global = sc.broadcast(syn0Global)
+  val bcSyn1Global = sc.broadcast(syn1Global)
   val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) =>
 val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) 
<< 8))
 val syn0Modify = new Array[Int](vocabSize)
 val syn1Modify = new Array[Int](vocabSize)
-val model = iter.foldLeft((syn0Global, syn1Global, 0, 0)) {
+val model = iter.foldLeft((bcSyn0Global.value, bcSyn1Global.value, 0, 
0)) {
   case ((syn0, syn1, lastWordCount, wordCount), sentence) =>
 var lwc = lastWordCount
 var wc = wordCount
@@ -405,6 +408,8 @@ class Word2Vec extends Serializable with Logging {
 }
 i += 1
   }
+  bcSyn0Global.unpersist(false)
+  bcSyn1Global.unpersist(false)
 }
 newSentences.unpersist()
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [DOC] Explicitly state that top maintains the order of elements

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 e9653921e -> aaf835f1d


[DOC] Explicitly state that top maintains the order of elements

Top is implemented in terms of takeOrdered, which already maintains the
order, so top should, too.

Author: Wieland Hoffmann 

Closes #10013 from mineo/top-order.

(cherry picked from commit 26c3581f17f475fab2f3b5301b8f253ff2fa6438)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/aaf835f1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/aaf835f1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/aaf835f1

Branch: refs/heads/branch-1.6
Commit: aaf835f1d67470d859fae3c0cc3143e4ccaaf3d0
Parents: e965392
Author: Wieland Hoffmann 
Authored: Mon Nov 30 09:32:48 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 09:32:58 2015 +

--
 core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 4 ++--
 core/src/main/scala/org/apache/spark/rdd/RDD.scala  | 3 ++-
 2 files changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/aaf835f1/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala 
b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 871be0b..1e9d4f1 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -556,7 +556,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends 
Serializable {
 
   /**
* Returns the top k (largest) elements from this RDD as defined by
-   * the specified Comparator[T].
+   * the specified Comparator[T] and maintains the order.
* @param num k, the number of top elements to return
* @param comp the comparator that defines the order
* @return an array of top elements
@@ -567,7 +567,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends 
Serializable {
 
   /**
* Returns the top k (largest) elements from this RDD using the
-   * natural ordering for T.
+   * natural ordering for T and maintains the order.
* @param num k, the number of top elements to return
* @return an array of top elements
*/

http://git-wip-us.apache.org/repos/asf/spark/blob/aaf835f1/core/src/main/scala/org/apache/spark/rdd/RDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 2aeb5ee..8b3731d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -1327,7 +1327,8 @@ abstract class RDD[T: ClassTag](
 
   /**
* Returns the top k (largest) elements from this RDD as defined by the 
specified
-   * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For 
example:
+   * implicit Ordering[T] and maintains the ordering. This does the opposite of
+   * [[takeOrdered]]. For example:
* {{{
*   sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1)
*   // returns Array(12)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [DOC] Explicitly state that top maintains the order of elements

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 953e8e6dc -> 26c3581f1


[DOC] Explicitly state that top maintains the order of elements

Top is implemented in terms of takeOrdered, which already maintains the
order, so top should, too.

Author: Wieland Hoffmann 

Closes #10013 from mineo/top-order.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/26c3581f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/26c3581f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/26c3581f

Branch: refs/heads/master
Commit: 26c3581f17f475fab2f3b5301b8f253ff2fa6438
Parents: 953e8e6
Author: Wieland Hoffmann 
Authored: Mon Nov 30 09:32:48 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 09:32:48 2015 +

--
 core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 4 ++--
 core/src/main/scala/org/apache/spark/rdd/RDD.scala  | 3 ++-
 2 files changed, 4 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/26c3581f/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
--
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala 
b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 871be0b..1e9d4f1 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -556,7 +556,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends 
Serializable {
 
   /**
* Returns the top k (largest) elements from this RDD as defined by
-   * the specified Comparator[T].
+   * the specified Comparator[T] and maintains the order.
* @param num k, the number of top elements to return
* @param comp the comparator that defines the order
* @return an array of top elements
@@ -567,7 +567,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends 
Serializable {
 
   /**
* Returns the top k (largest) elements from this RDD using the
-   * natural ordering for T.
+   * natural ordering for T and maintains the order.
* @param num k, the number of top elements to return
* @return an array of top elements
*/

http://git-wip-us.apache.org/repos/asf/spark/blob/26c3581f/core/src/main/scala/org/apache/spark/rdd/RDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 2aeb5ee..8b3731d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -1327,7 +1327,8 @@ abstract class RDD[T: ClassTag](
 
   /**
* Returns the top k (largest) elements from this RDD as defined by the 
specified
-   * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For 
example:
+   * implicit Ordering[T] and maintains the ordering. This does the opposite of
+   * [[takeOrdered]]. For example:
* {{{
*   sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1)
*   // returns Array(12)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR][BUILD] Changed the comment to reflect the plugin project is there to support SBT pom reader only.

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master e07494420 -> 953e8e6dc


[MINOR][BUILD] Changed the comment to reflect the plugin project is there to 
support SBT pom reader only.

Author: Prashant Sharma 

Closes #10012 from ScrapCodes/minor-build-comment.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/953e8e6d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/953e8e6d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/953e8e6d

Branch: refs/heads/master
Commit: 953e8e6dcb32cd88005834e9c3720740e201826c
Parents: e074944
Author: Prashant Sharma 
Authored: Mon Nov 30 09:30:58 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 09:30:58 2015 +

--
 project/project/SparkPluginBuild.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/953e8e6d/project/project/SparkPluginBuild.scala
--
diff --git a/project/project/SparkPluginBuild.scala 
b/project/project/SparkPluginBuild.scala
index 471d00b..cbb88dc 100644
--- a/project/project/SparkPluginBuild.scala
+++ b/project/project/SparkPluginBuild.scala
@@ -19,9 +19,8 @@ import sbt._
 import sbt.Keys._
 
 /**
- * This plugin project is there to define new scala style rules for spark. 
This is
- * a plugin project so that this gets compiled first and is put on the 
classpath and
- * becomes available for scalastyle sbt plugin.
+ * This plugin project is there because we use our custom fork of 
sbt-pom-reader plugin. This is
+ * a plugin project so that this gets compiled first and is available on the 
classpath for SBT build.
  */
 object SparkPluginDef extends Build {
   lazy val root = Project("plugins", file(".")) dependsOn(sbtPomReader)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR][BUILD] Changed the comment to reflect the plugin project is there to support SBT pom reader only.

2015-11-30 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 12d97b0c5 -> e9653921e


[MINOR][BUILD] Changed the comment to reflect the plugin project is there to 
support SBT pom reader only.

Author: Prashant Sharma 

Closes #10012 from ScrapCodes/minor-build-comment.

(cherry picked from commit 953e8e6dcb32cd88005834e9c3720740e201826c)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e9653921
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e9653921
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e9653921

Branch: refs/heads/branch-1.6
Commit: e9653921e84092f36988e3d73618c58b0c10e938
Parents: 12d97b0
Author: Prashant Sharma 
Authored: Mon Nov 30 09:30:58 2015 +
Committer: Sean Owen 
Committed: Mon Nov 30 09:31:09 2015 +

--
 project/project/SparkPluginBuild.scala | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e9653921/project/project/SparkPluginBuild.scala
--
diff --git a/project/project/SparkPluginBuild.scala 
b/project/project/SparkPluginBuild.scala
index 471d00b..cbb88dc 100644
--- a/project/project/SparkPluginBuild.scala
+++ b/project/project/SparkPluginBuild.scala
@@ -19,9 +19,8 @@ import sbt._
 import sbt.Keys._
 
 /**
- * This plugin project is there to define new scala style rules for spark. 
This is
- * a plugin project so that this gets compiled first and is put on the 
classpath and
- * becomes available for scalastyle sbt plugin.
+ * This plugin project is there because we use our custom fork of 
sbt-pom-reader plugin. This is
+ * a plugin project so that this gets compiled first and is available on the 
classpath for SBT build.
  */
 object SparkPluginDef extends Build {
   lazy val root = Project("plugins", file(".")) dependsOn(sbtPomReader)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: rmse was wrongly calculated

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 34ded83ed -> 48bfe3f89


rmse was wrongly calculated

It was multiplying with U instaed of dividing by U

Author: Viveka Kulharia 

Closes #9771 from vivkul/patch-1.

(cherry picked from commit 1429e0a2b562469146b6fa06051c85a00092e5b8)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/48bfe3f8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/48bfe3f8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/48bfe3f8

Branch: refs/heads/branch-1.6
Commit: 48bfe3f894c321bee7f1868b1d07939b59f8ba77
Parents: 34ded83
Author: Viveka Kulharia 
Authored: Wed Nov 18 09:10:15 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 09:10:28 2015 +

--
 examples/src/main/python/als.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/48bfe3f8/examples/src/main/python/als.py
--
diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py
index 1c3a787..205ca02 100755
--- a/examples/src/main/python/als.py
+++ b/examples/src/main/python/als.py
@@ -36,7 +36,7 @@ np.random.seed(42)
 
 def rmse(R, ms, us):
 diff = R - ms * us.T
-return np.sqrt(np.sum(np.power(diff, 2)) / M * U)
+return np.sqrt(np.sum(np.power(diff, 2)) / (M * U))
 
 
 def update(i, vec, mat, ratings):


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: rmse was wrongly calculated

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9631ca352 -> 1429e0a2b


rmse was wrongly calculated

It was multiplying with U instaed of dividing by U

Author: Viveka Kulharia 

Closes #9771 from vivkul/patch-1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1429e0a2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1429e0a2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1429e0a2

Branch: refs/heads/master
Commit: 1429e0a2b562469146b6fa06051c85a00092e5b8
Parents: 9631ca3
Author: Viveka Kulharia 
Authored: Wed Nov 18 09:10:15 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 09:10:15 2015 +

--
 examples/src/main/python/als.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1429e0a2/examples/src/main/python/als.py
--
diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py
index 1c3a787..205ca02 100755
--- a/examples/src/main/python/als.py
+++ b/examples/src/main/python/als.py
@@ -36,7 +36,7 @@ np.random.seed(42)
 
 def rmse(R, ms, us):
 diff = R - ms * us.T
-return np.sqrt(np.sum(np.power(diff, 2)) / M * U)
+return np.sqrt(np.sum(np.power(diff, 2)) / (M * U))
 
 
 def update(i, vec, mat, ratings):


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: rmse was wrongly calculated

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 0ed6d9cf3 -> 4b8dc2556


rmse was wrongly calculated

It was multiplying with U instaed of dividing by U

Author: Viveka Kulharia 

Closes #9771 from vivkul/patch-1.

(cherry picked from commit 1429e0a2b562469146b6fa06051c85a00092e5b8)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4b8dc255
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4b8dc255
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4b8dc255

Branch: refs/heads/branch-1.5
Commit: 4b8dc25562e150e5ebc46b03a105c55de3098a1a
Parents: 0ed6d9c
Author: Viveka Kulharia 
Authored: Wed Nov 18 09:10:15 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 09:10:43 2015 +

--
 examples/src/main/python/als.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4b8dc255/examples/src/main/python/als.py
--
diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py
index 1c3a787..205ca02 100755
--- a/examples/src/main/python/als.py
+++ b/examples/src/main/python/als.py
@@ -36,7 +36,7 @@ np.random.seed(42)
 
 def rmse(R, ms, us):
 diff = R - ms * us.T
-return np.sqrt(np.sum(np.power(diff, 2)) / M * U)
+return np.sqrt(np.sum(np.power(diff, 2)) / (M * U))
 
 
 def update(i, vec, mat, ratings):


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: rmse was wrongly calculated

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.4 073c89f06 -> e12fbd80c


rmse was wrongly calculated

It was multiplying with U instaed of dividing by U

Author: Viveka Kulharia 

Closes #9771 from vivkul/patch-1.

(cherry picked from commit 1429e0a2b562469146b6fa06051c85a00092e5b8)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e12fbd80
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e12fbd80
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e12fbd80

Branch: refs/heads/branch-1.4
Commit: e12fbd80c33dc119b0b336b43495a0afd952dea9
Parents: 073c89f
Author: Viveka Kulharia 
Authored: Wed Nov 18 09:10:15 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 09:10:54 2015 +

--
 examples/src/main/python/als.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e12fbd80/examples/src/main/python/als.py
--
diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py
index 1c3a787..205ca02 100755
--- a/examples/src/main/python/als.py
+++ b/examples/src/main/python/als.py
@@ -36,7 +36,7 @@ np.random.seed(42)
 
 def rmse(R, ms, us):
 diff = R - ms * us.T
-return np.sqrt(np.sum(np.power(diff, 2)) / M * U)
+return np.sqrt(np.sum(np.power(diff, 2)) / (M * U))
 
 
 def update(i, vec, mat, ratings):


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-10946][SQL] JDBC - Use Statement.executeUpdate instead of PreparedStatement.executeUpdate for DDLs

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 0eb82133f -> 5da7d4130


[SPARK-10946][SQL] JDBC - Use Statement.executeUpdate instead of 
PreparedStatement.executeUpdate for DDLs

New changes with JDBCRDD

Author: somideshmukh 

Closes #9733 from somideshmukh/SomilBranch-1.1.

(cherry picked from commit b8f4379ba1c5c1a8f3b4c88bd97031dc8ad2dfea)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5da7d413
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5da7d413
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5da7d413

Branch: refs/heads/branch-1.6
Commit: 5da7d41307c02e0204a50a5c3d9a35e839d29910
Parents: 0eb8213
Author: somideshmukh 
Authored: Wed Nov 18 08:51:01 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 08:54:09 2015 +

--
 sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala | 2 +-
 .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5da7d413/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index e63a4d5..03867be 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -297,7 +297,7 @@ final class DataFrameWriter private[sql](df: DataFrame) {
   if (!tableExists) {
 val schema = JdbcUtils.schemaString(df, url)
 val sql = s"CREATE TABLE $table ($schema)"
-conn.prepareStatement(sql).executeUpdate()
+conn.createStatement.executeUpdate(sql)
   }
 } finally {
   conn.close()

http://git-wip-us.apache.org/repos/asf/spark/blob/5da7d413/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index 32d28e5..7375a5c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -55,7 +55,7 @@ object JdbcUtils extends Logging {
* Drops a table from the JDBC database.
*/
   def dropTable(conn: Connection, table: String): Unit = {
-conn.prepareStatement(s"DROP TABLE $table").executeUpdate()
+conn.createStatement.executeUpdate(s"DROP TABLE $table")
   }
 
   /**


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-10946][SQL] JDBC - Use Statement.executeUpdate instead of PreparedStatement.executeUpdate for DDLs

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 1714350bd -> b8f4379ba


[SPARK-10946][SQL] JDBC - Use Statement.executeUpdate instead of 
PreparedStatement.executeUpdate for DDLs

New changes with JDBCRDD

Author: somideshmukh 

Closes #9733 from somideshmukh/SomilBranch-1.1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b8f4379b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b8f4379b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b8f4379b

Branch: refs/heads/master
Commit: b8f4379ba1c5c1a8f3b4c88bd97031dc8ad2dfea
Parents: 1714350
Author: somideshmukh 
Authored: Wed Nov 18 08:51:01 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 08:51:01 2015 +

--
 sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala | 2 +-
 .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b8f4379b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
--
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
index e63a4d5..03867be 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala
@@ -297,7 +297,7 @@ final class DataFrameWriter private[sql](df: DataFrame) {
   if (!tableExists) {
 val schema = JdbcUtils.schemaString(df, url)
 val sql = s"CREATE TABLE $table ($schema)"
-conn.prepareStatement(sql).executeUpdate()
+conn.createStatement.executeUpdate(sql)
   }
 } finally {
   conn.close()

http://git-wip-us.apache.org/repos/asf/spark/blob/b8f4379b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index 32d28e5..7375a5c 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -55,7 +55,7 @@ object JdbcUtils extends Logging {
* Drops a table from the JDBC database.
*/
   def dropTable(conn: Connection, table: String): Unit = {
-conn.prepareStatement(s"DROP TABLE $table").executeUpdate()
+conn.createStatement.executeUpdate(s"DROP TABLE $table")
   }
 
   /**


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-6541] Sort executors by ID (numeric)

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 5da7d4130 -> 04938d929


[SPARK-6541] Sort executors by ID (numeric)

"Force" the executor ID sort with Int.

Author: Jean-Baptiste Onofré 

Closes #9165 from jbonofre/SPARK-6541.

(cherry picked from commit e62820c85fe02c70f9ed51b2e68d41ff8cfecd40)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/04938d92
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/04938d92
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/04938d92

Branch: refs/heads/branch-1.6
Commit: 04938d9295a2b546b66ac980a54ff8b3a52f4cf4
Parents: 5da7d41
Author: Jean-Baptiste Onofré 
Authored: Wed Nov 18 08:57:58 2015 +
Committer: Sean Owen 
Committed: Wed Nov 18 08:58:06 2015 +

--
 .../resources/org/apache/spark/ui/static/sorttable.js  |  2 +-
 .../scala/org/apache/spark/ui/jobs/ExecutorTable.scala | 13 +++--
 2 files changed, 12 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/04938d92/core/src/main/resources/org/apache/spark/ui/static/sorttable.js
--
diff --git a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js 
b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js
index dde6069..a73d9a5 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js
@@ -89,7 +89,7 @@ sorttable = {
 // make it clickable to sort
 headrow[i].sorttable_columnindex = i;
 headrow[i].sorttable_tbody = table.tBodies[0];
-dean_addEvent(headrow[i],"click", function(e) {
+dean_addEvent(headrow[i],"click", sorttable.innerSortFunction = 
function(e) {
 
   if (this.className.search(/\bsorttable_sorted\b/) != -1) {
 // if we're already sorted by this column, just 

http://git-wip-us.apache.org/repos/asf/spark/blob/04938d92/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
index be144f6..1268f44 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.ui.jobs
 
 import scala.collection.mutable
-import scala.xml.Node
+import scala.xml.{Unparsed, Node}
 
 import org.apache.spark.ui.{ToolTips, UIUtils}
 import org.apache.spark.ui.jobs.UIData.StageUIData
@@ -52,7 +52,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: 
Int, parent: Stage
 
 
   
-Executor ID
+Executor ID
 Address
 Task Time
 Total Tasks
@@ -89,6 +89,15 @@ private[ui] class ExecutorTable(stageId: Int, 
stageAttemptId: Int, parent: Stage
 {createExecutorTable()}
   
 
+
+  {Unparsed {
+"""
+  |  window.onload = function() {
+  |
sorttable.innerSortFunction.apply(document.getElementById('executorid'), [])
+  |  };
+""".stripMargin
+  }}
+
   }
 
   private def createExecutorTable() : Seq[Node] = {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11652][CORE] Remote code execution with InvokerTransformer

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 04938d929 -> 34ded83ed


[SPARK-11652][CORE] Remote code execution with InvokerTransformer

Update to Commons Collections 3.2.2 to avoid any potential remote code 
execution vulnerability

Author: Sean Owen <so...@cloudera.com>

Closes #9731 from srowen/SPARK-11652.

(cherry picked from commit 9631ca35275b0ce8a5219f975907ac36ed11f528)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/34ded83e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/34ded83e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/34ded83e

Branch: refs/heads/branch-1.6
Commit: 34ded83ed1c496a2e4458a1e1bd73372efa63c80
Parents: 04938d9
Author: Sean Owen <so...@cloudera.com>
Authored: Wed Nov 18 08:59:20 2015 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Wed Nov 18 08:59:28 2015 +

--
 pom.xml | 7 +++
 1 file changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/34ded83e/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 940e2d8..ad84911 100644
--- a/pom.xml
+++ b/pom.xml
@@ -162,6 +162,8 @@
 
 3.1
 3.4.1
+
+3.2.2
 2.10.5
 2.10
 ${scala.version}
@@ -476,6 +478,11 @@
 ${commons.math3.version}
   
   
+org.apache.commons
+commons-collections
+${commons.collections.version}
+  
+  
 org.apache.ivy
 ivy
 ${ivy.version}


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11652][CORE] Remote code execution with InvokerTransformer

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master e62820c85 -> 9631ca352


[SPARK-11652][CORE] Remote code execution with InvokerTransformer

Update to Commons Collections 3.2.2 to avoid any potential remote code 
execution vulnerability

Author: Sean Owen <so...@cloudera.com>

Closes #9731 from srowen/SPARK-11652.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9631ca35
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9631ca35
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9631ca35

Branch: refs/heads/master
Commit: 9631ca35275b0ce8a5219f975907ac36ed11f528
Parents: e62820c
Author: Sean Owen <so...@cloudera.com>
Authored: Wed Nov 18 08:59:20 2015 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Wed Nov 18 08:59:20 2015 +

--
 pom.xml | 7 +++
 1 file changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9631ca35/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 940e2d8..ad84911 100644
--- a/pom.xml
+++ b/pom.xml
@@ -162,6 +162,8 @@
 
 3.1
 3.4.1
+
+3.2.2
 2.10.5
 2.10
 ${scala.version}
@@ -476,6 +478,11 @@
 ${commons.math3.version}
   
   
+org.apache.commons
+commons-collections
+${commons.collections.version}
+  
+  
 org.apache.ivy
 ivy
 ${ivy.version}


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11652][CORE] Remote code execution with InvokerTransformer

2015-11-18 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 f7a7230f3 -> 0ed6d9cf3


[SPARK-11652][CORE] Remote code execution with InvokerTransformer

Update to Commons Collections 3.2.2 to avoid any potential remote code 
execution vulnerability

Author: Sean Owen <so...@cloudera.com>

Closes #9731 from srowen/SPARK-11652.

(cherry picked from commit 9631ca35275b0ce8a5219f975907ac36ed11f528)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0ed6d9cf
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0ed6d9cf
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0ed6d9cf

Branch: refs/heads/branch-1.5
Commit: 0ed6d9cf3aa0c2b4c09ac297408e91a52de3961c
Parents: f7a7230
Author: Sean Owen <so...@cloudera.com>
Authored: Wed Nov 18 08:59:20 2015 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Wed Nov 18 09:00:49 2015 +

--
 pom.xml | 7 +++
 1 file changed, 7 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0ed6d9cf/pom.xml
--
diff --git a/pom.xml b/pom.xml
index 24cdd05..df18ce1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -158,6 +158,8 @@
 
 3.1
 3.4.1
+
+3.2.2
 2.10.4
 2.10
 ${scala.version}
@@ -447,6 +449,11 @@
 ${commons.math3.version}
   
   
+org.apache.commons
+commons-collections
+${commons.collections.version}
+  
+  
 org.apache.ivy
 ivy
 ${ivy.version}


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11906][WEB UI] Speculation Tasks Cause ProgressBar UI Overflow

2015-11-24 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 12eea834d -> 800bd799a


[SPARK-11906][WEB UI] Speculation Tasks Cause ProgressBar UI Overflow

When there are speculative tasks in the stage, running progress bar could 
overflow and goes hidden on a new line:
![image](https://cloud.githubusercontent.com/assets/4317392/11326841/5fd3482e-9142-11e5-8ca5-cb2f0c0c8964.png)
3 completed / 2 running (including 1 speculative) out of 4 total tasks

This is a simple fix by capping the started tasks at `total - completed` tasks
![image](https://cloud.githubusercontent.com/assets/4317392/11326842/6bb67260-9142-11e5-90f0-37f9174878ec.png)

I should note my preferred way to fix it is via css style
```css
.progress { display: flex; }
```
which shifts the correction burden from driver to web browser. However I 
couldn't get selenium test to measure the position/dimension of the progress 
bar correctly to get this unit tested.

It also has the side effect that the width will be calibrated so the running 
occupies 2 / 5 instead of 1 / 4.
![image](https://cloud.githubusercontent.com/assets/4317392/11326848/7b03e9f0-9142-11e5-89ad-bd99cb0647cf.png)

All in all, since this cosmetic bug is minor enough, I suppose the original 
simple fix should be good enough.

Author: Forest Fang 

Closes #9896 from saurfang/progressbar.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/800bd799
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/800bd799
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/800bd799

Branch: refs/heads/master
Commit: 800bd799acf7f10a469d8d6537279953129eb2c6
Parents: 12eea83
Author: Forest Fang 
Authored: Tue Nov 24 09:03:32 2015 +
Committer: Sean Owen 
Committed: Tue Nov 24 09:03:32 2015 +

--
 core/src/main/scala/org/apache/spark/ui/UIUtils.scala |  4 +++-
 .../src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala | 10 ++
 2 files changed, 13 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/800bd799/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index 25dcb60..84a1116 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -319,7 +319,9 @@ private[spark] object UIUtils extends Logging {
   skipped: Int,
   total: Int): Seq[Node] = {
 val completeWidth = "width: %s%%".format((completed.toDouble/total)*100)
-val startWidth = "width: %s%%".format((started.toDouble/total)*100)
+// started + completed can be > total when there are speculative tasks
+val boundedStarted = math.min(started, total - completed)
+val startWidth = "width: %s%%".format((boundedStarted.toDouble/total)*100)
 
 
   

http://git-wip-us.apache.org/repos/asf/spark/blob/800bd799/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
index 2b693c1..dd8d5ec 100644
--- a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
@@ -57,6 +57,16 @@ class UIUtilsSuite extends SparkFunSuite {
 )
   }
 
+  test("SPARK-11906: Progress bar should not overflow because of speculative 
tasks") {
+val generated = makeProgressBar(2, 3, 0, 0, 4).head.child.filter(_.label 
== "div")
+val expected = Seq(
+  ,
+  
+)
+assert(generated.sameElements(expected),
+  s"\nRunning progress bar should round 
down\n\nExpected:\n$expected\nGenerated:\n$generated")
+  }
+
   private def verify(
   desc: String, expected: Elem, errorMsg: String = "", baseUrl: String = 
""): Unit = {
 val generated = makeDescription(desc, baseUrl)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-11906][WEB UI] Speculation Tasks Cause ProgressBar UI Overflow

2015-11-24 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 e1b0a2376 -> 17ea95133


[SPARK-11906][WEB UI] Speculation Tasks Cause ProgressBar UI Overflow

When there are speculative tasks in the stage, running progress bar could 
overflow and goes hidden on a new line:
![image](https://cloud.githubusercontent.com/assets/4317392/11326841/5fd3482e-9142-11e5-8ca5-cb2f0c0c8964.png)
3 completed / 2 running (including 1 speculative) out of 4 total tasks

This is a simple fix by capping the started tasks at `total - completed` tasks
![image](https://cloud.githubusercontent.com/assets/4317392/11326842/6bb67260-9142-11e5-90f0-37f9174878ec.png)

I should note my preferred way to fix it is via css style
```css
.progress { display: flex; }
```
which shifts the correction burden from driver to web browser. However I 
couldn't get selenium test to measure the position/dimension of the progress 
bar correctly to get this unit tested.

It also has the side effect that the width will be calibrated so the running 
occupies 2 / 5 instead of 1 / 4.
![image](https://cloud.githubusercontent.com/assets/4317392/11326848/7b03e9f0-9142-11e5-89ad-bd99cb0647cf.png)

All in all, since this cosmetic bug is minor enough, I suppose the original 
simple fix should be good enough.

Author: Forest Fang 

Closes #9896 from saurfang/progressbar.

(cherry picked from commit 800bd799acf7f10a469d8d6537279953129eb2c6)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/17ea9513
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/17ea9513
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/17ea9513

Branch: refs/heads/branch-1.6
Commit: 17ea9513313187cfb6e7c7e66e6e469c5cef63cf
Parents: e1b0a23
Author: Forest Fang 
Authored: Tue Nov 24 09:03:32 2015 +
Committer: Sean Owen 
Committed: Tue Nov 24 09:03:44 2015 +

--
 core/src/main/scala/org/apache/spark/ui/UIUtils.scala |  4 +++-
 .../src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala | 10 ++
 2 files changed, 13 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/17ea9513/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
--
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index 25dcb60..84a1116 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -319,7 +319,9 @@ private[spark] object UIUtils extends Logging {
   skipped: Int,
   total: Int): Seq[Node] = {
 val completeWidth = "width: %s%%".format((completed.toDouble/total)*100)
-val startWidth = "width: %s%%".format((started.toDouble/total)*100)
+// started + completed can be > total when there are speculative tasks
+val boundedStarted = math.min(started, total - completed)
+val startWidth = "width: %s%%".format((boundedStarted.toDouble/total)*100)
 
 
   

http://git-wip-us.apache.org/repos/asf/spark/blob/17ea9513/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
index 2b693c1..dd8d5ec 100644
--- a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala
@@ -57,6 +57,16 @@ class UIUtilsSuite extends SparkFunSuite {
 )
   }
 
+  test("SPARK-11906: Progress bar should not overflow because of speculative 
tasks") {
+val generated = makeProgressBar(2, 3, 0, 0, 4).head.child.filter(_.label 
== "div")
+val expected = Seq(
+  ,
+  
+)
+assert(generated.sameElements(expected),
+  s"\nRunning progress bar should round 
down\n\nExpected:\n$expected\nGenerated:\n$generated")
+  }
+
   private def verify(
   desc: String, expected: Elem, errorMsg: String = "", baseUrl: String = 
""): Unit = {
 val generated = makeDescription(desc, baseUrl)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12269][STREAMING][KINESIS] Update aws-java-sdk version

2016-01-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master bd723bd53 -> 8fe928b4f


[SPARK-12269][STREAMING][KINESIS] Update aws-java-sdk version

The current Spark Streaming kinesis connector references a quite old version 
1.9.40 of the AWS Java SDK (1.10.40 is current). Numerous AWS features 
including Kinesis Firehose are unavailable in 1.9. Those two versions of the 
AWS SDK in turn require conflicting versions of Jackson (2.4.4 and 2.5.3 
respectively) such that one cannot include the current AWS SDK in a project 
that also uses the Spark Streaming Kinesis ASL.

Author: BrianLondon 

Closes #10256 from BrianLondon/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8fe928b4
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8fe928b4
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8fe928b4

Branch: refs/heads/master
Commit: 8fe928b4fe380ba527164bd413402abfed13c0e1
Parents: bd723bd
Author: BrianLondon 
Authored: Mon Jan 11 09:32:06 2016 +
Committer: Sean Owen 
Committed: Mon Jan 11 09:32:06 2016 +

--
 dev/deps/spark-deps-hadoop-2.2 | 8 
 dev/deps/spark-deps-hadoop-2.3 | 8 
 dev/deps/spark-deps-hadoop-2.4 | 8 
 dev/deps/spark-deps-hadoop-2.6 | 8 
 pom.xml| 6 +++---
 5 files changed, 19 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8fe928b4/dev/deps/spark-deps-hadoop-2.2
--
diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2
index e4373f7..cd3ff29 100644
--- a/dev/deps/spark-deps-hadoop-2.2
+++ b/dev/deps/spark-deps-hadoop-2.2
@@ -84,13 +84,13 @@ hadoop-yarn-server-web-proxy-2.2.0.jar
 httpclient-4.3.2.jar
 httpcore-4.3.2.jar
 ivy-2.4.0.jar
-jackson-annotations-2.4.4.jar
-jackson-core-2.4.4.jar
+jackson-annotations-2.5.3.jar
+jackson-core-2.5.3.jar
 jackson-core-asl-1.9.13.jar
-jackson-databind-2.4.4.jar
+jackson-databind-2.5.3.jar
 jackson-jaxrs-1.9.13.jar
 jackson-mapper-asl-1.9.13.jar
-jackson-module-scala_2.10-2.4.4.jar
+jackson-module-scala_2.10-2.5.3.jar
 jackson-xc-1.9.13.jar
 janino-2.7.8.jar
 jansi-1.4.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8fe928b4/dev/deps/spark-deps-hadoop-2.3
--
diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3
index 7478181..0985089 100644
--- a/dev/deps/spark-deps-hadoop-2.3
+++ b/dev/deps/spark-deps-hadoop-2.3
@@ -79,13 +79,13 @@ hadoop-yarn-server-web-proxy-2.3.0.jar
 httpclient-4.3.2.jar
 httpcore-4.3.2.jar
 ivy-2.4.0.jar
-jackson-annotations-2.4.4.jar
-jackson-core-2.4.4.jar
+jackson-annotations-2.5.3.jar
+jackson-core-2.5.3.jar
 jackson-core-asl-1.9.13.jar
-jackson-databind-2.4.4.jar
+jackson-databind-2.5.3.jar
 jackson-jaxrs-1.9.13.jar
 jackson-mapper-asl-1.9.13.jar
-jackson-module-scala_2.10-2.4.4.jar
+jackson-module-scala_2.10-2.5.3.jar
 jackson-xc-1.9.13.jar
 janino-2.7.8.jar
 jansi-1.4.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8fe928b4/dev/deps/spark-deps-hadoop-2.4
--
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index faffb8b..50f0626 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -79,13 +79,13 @@ hadoop-yarn-server-web-proxy-2.4.0.jar
 httpclient-4.3.2.jar
 httpcore-4.3.2.jar
 ivy-2.4.0.jar
-jackson-annotations-2.4.4.jar
-jackson-core-2.4.4.jar
+jackson-annotations-2.5.3.jar
+jackson-core-2.5.3.jar
 jackson-core-asl-1.9.13.jar
-jackson-databind-2.4.4.jar
+jackson-databind-2.5.3.jar
 jackson-jaxrs-1.9.13.jar
 jackson-mapper-asl-1.9.13.jar
-jackson-module-scala_2.10-2.4.4.jar
+jackson-module-scala_2.10-2.5.3.jar
 jackson-xc-1.9.13.jar
 janino-2.7.8.jar
 jansi-1.4.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/8fe928b4/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index e703c7a..2b6ca98 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -85,13 +85,13 @@ htrace-core-3.0.4.jar
 httpclient-4.3.2.jar
 httpcore-4.3.2.jar
 ivy-2.4.0.jar
-jackson-annotations-2.4.4.jar
-jackson-core-2.4.4.jar
+jackson-annotations-2.5.3.jar
+jackson-core-2.5.3.jar
 jackson-core-asl-1.9.13.jar
-jackson-databind-2.4.4.jar
+jackson-databind-2.5.3.jar
 jackson-jaxrs-1.9.13.jar
 jackson-mapper-asl-1.9.13.jar
-jackson-module-scala_2.10-2.4.4.jar
+jackson-module-scala_2.10-2.5.3.jar
 jackson-xc-1.9.13.jar
 janino-2.7.8.jar
 jansi-1.4.jar


spark git commit: removed lambda from sortByKey()

2016-01-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 43b72d83e -> d4cfd2acd


removed lambda from sortByKey()

According to the documentation the sortByKey method does not take a lambda as 
an argument, thus the example is flawed. Removed the argument completely as 
this will default to ascending sort.

Author: Udo Klein 

Closes #10640 from udoklein/patch-1.

(cherry picked from commit bd723bd53d9a28239b60939a248a4ea13340aad8)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d4cfd2ac
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d4cfd2ac
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d4cfd2ac

Branch: refs/heads/branch-1.6
Commit: d4cfd2acd62f2b0638a1248a38263c04eaf8
Parents: 43b72d8
Author: Udo Klein 
Authored: Mon Jan 11 09:30:08 2016 +
Committer: Sean Owen 
Committed: Mon Jan 11 09:30:27 2016 +

--
 examples/src/main/python/sort.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d4cfd2ac/examples/src/main/python/sort.py
--
diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py
index f6b0ecb..b6c2916 100755
--- a/examples/src/main/python/sort.py
+++ b/examples/src/main/python/sort.py
@@ -30,7 +30,7 @@ if __name__ == "__main__":
 lines = sc.textFile(sys.argv[1], 1)
 sortedCount = lines.flatMap(lambda x: x.split(' ')) \
 .map(lambda x: (int(x), 1)) \
-.sortByKey(lambda x: x)
+.sortByKey()
 # This is just a demo on how to bring all the sorted data back to a single 
node.
 # In reality, we wouldn't want to collect all the data to the driver node.
 output = sortedCount.collect()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: removed lambda from sortByKey()

2016-01-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.5 665aa47f8 -> 0e2aa4198


removed lambda from sortByKey()

According to the documentation the sortByKey method does not take a lambda as 
an argument, thus the example is flawed. Removed the argument completely as 
this will default to ascending sort.

Author: Udo Klein 

Closes #10640 from udoklein/patch-1.

(cherry picked from commit bd723bd53d9a28239b60939a248a4ea13340aad8)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0e2aa419
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0e2aa419
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0e2aa419

Branch: refs/heads/branch-1.5
Commit: 0e2aa41988c4ae8391b48b0902badf0cda188dcc
Parents: 665aa47
Author: Udo Klein 
Authored: Mon Jan 11 09:30:08 2016 +
Committer: Sean Owen 
Committed: Mon Jan 11 09:30:40 2016 +

--
 examples/src/main/python/sort.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0e2aa419/examples/src/main/python/sort.py
--
diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py
index f6b0ecb..b6c2916 100755
--- a/examples/src/main/python/sort.py
+++ b/examples/src/main/python/sort.py
@@ -30,7 +30,7 @@ if __name__ == "__main__":
 lines = sc.textFile(sys.argv[1], 1)
 sortedCount = lines.flatMap(lambda x: x.split(' ')) \
 .map(lambda x: (int(x), 1)) \
-.sortByKey(lambda x: x)
+.sortByKey()
 # This is just a demo on how to bring all the sorted data back to a single 
node.
 # In reality, we wouldn't want to collect all the data to the driver node.
 output = sortedCount.collect()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: removed lambda from sortByKey()

2016-01-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master f253feff6 -> bd723bd53


removed lambda from sortByKey()

According to the documentation the sortByKey method does not take a lambda as 
an argument, thus the example is flawed. Removed the argument completely as 
this will default to ascending sort.

Author: Udo Klein 

Closes #10640 from udoklein/patch-1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bd723bd5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bd723bd5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bd723bd5

Branch: refs/heads/master
Commit: bd723bd53d9a28239b60939a248a4ea13340aad8
Parents: f253fef
Author: Udo Klein 
Authored: Mon Jan 11 09:30:08 2016 +
Committer: Sean Owen 
Committed: Mon Jan 11 09:30:08 2016 +

--
 examples/src/main/python/sort.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/bd723bd5/examples/src/main/python/sort.py
--
diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py
index f6b0ecb..b6c2916 100755
--- a/examples/src/main/python/sort.py
+++ b/examples/src/main/python/sort.py
@@ -30,7 +30,7 @@ if __name__ == "__main__":
 lines = sc.textFile(sys.argv[1], 1)
 sortedCount = lines.flatMap(lambda x: x.split(' ')) \
 .map(lambda x: (int(x), 1)) \
-.sortByKey(lambda x: x)
+.sortByKey()
 # This is just a demo on how to bring all the sorted data back to a single 
node.
 # In reality, we wouldn't want to collect all the data to the driver node.
 output = sortedCount.collect()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-7615][MLLIB] MLLIB Word2Vec wordVectors divided by Euclidean Norm equals to zero

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 8cfa218f4 -> c48f2a3a5


[SPARK-7615][MLLIB] MLLIB Word2Vec wordVectors divided by Euclidean Norm equals 
to zero

Cosine similarity with 0 vector should be 0

Related to https://github.com/apache/spark/pull/10152

Author: Sean Owen <so...@cloudera.com>

Closes #10696 from srowen/SPARK-7615.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c48f2a3a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c48f2a3a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c48f2a3a

Branch: refs/heads/master
Commit: c48f2a3a5fd714ad2ff19b29337e55583988431e
Parents: 8cfa218
Author: Sean Owen <so...@cloudera.com>
Authored: Tue Jan 12 11:50:33 2016 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Jan 12 11:50:33 2016 +

--
 .../main/scala/org/apache/spark/mllib/feature/Word2Vec.scala  | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c48f2a3a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
index dc5d070..dee8988 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
@@ -543,7 +543,12 @@ class Word2VecModel private[spark] (
 val cosVec = cosineVec.map(_.toDouble)
 var ind = 0
 while (ind < numWords) {
-  cosVec(ind) /= wordVecNorms(ind)
+  val norm = wordVecNorms(ind)
+  if (norm == 0.0) {
+cosVec(ind) = 0.0
+  } else {
+cosVec(ind) /= norm
+  }
   ind += 1
 }
 wordList.zip(cosVec)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-5273][MLLIB][DOCS] Improve documentation examples for LinearRegression

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 3221a7d91 -> 4c67d55c0


[SPARK-5273][MLLIB][DOCS] Improve documentation examples for LinearRegression

Use a much smaller step size in LinearRegressionWithSGD MLlib examples to 
achieve a reasonable RMSE.

Our training folks hit this exact same issue when concocting an example and had 
the same solution.

Author: Sean Owen <so...@cloudera.com>

Closes #10675 from srowen/SPARK-5273.

(cherry picked from commit 9c7f34af37ef328149c1d66b4689d80a1589e1cc)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4c67d55c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4c67d55c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4c67d55c

Branch: refs/heads/branch-1.6
Commit: 4c67d55c0ccf086e91d1755b62c8526f2ff51f21
Parents: 3221a7d
Author: Sean Owen <so...@cloudera.com>
Authored: Tue Jan 12 12:13:32 2016 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Jan 12 13:26:37 2016 +

--
 docs/mllib-linear-methods.md | 8 +---
 1 file changed, 5 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4c67d55c/docs/mllib-linear-methods.md
--
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md
index 20b3561..aac8f75 100644
--- a/docs/mllib-linear-methods.md
+++ b/docs/mllib-linear-methods.md
@@ -590,7 +590,8 @@ val parsedData = data.map { line =>
 
 // Building the model
 val numIterations = 100
-val model = LinearRegressionWithSGD.train(parsedData, numIterations)
+val stepSize = 0.0001
+val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize)
 
 // Evaluate model on training examples and compute training error
 val valuesAndPreds = parsedData.map { point =>
@@ -655,8 +656,9 @@ public class LinearRegression {
 
 // Building the model
 int numIterations = 100;
+double stepSize = 0.0001;
 final LinearRegressionModel model =
-  LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations);
+  LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, 
stepSize);
 
 // Evaluate model on training examples and compute training error
 JavaRDD<Tuple2<Double, Double>> valuesAndPreds = parsedData.map(
@@ -706,7 +708,7 @@ data = sc.textFile("data/mllib/ridge-data/lpsa.data")
 parsedData = data.map(parsePoint)
 
 # Build the model
-model = LinearRegressionWithSGD.train(parsedData)
+model = LinearRegressionWithSGD.train(parsedData, iterations=100, 
step=0.0001)
 
 # Evaluate the model on training data
 valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features)))


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12638][API DOC] Parameter explanation not very accurate for rdd function "aggregate"

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 a6c9c68d8 -> 46fc7a12a


[SPARK-12638][API DOC] Parameter explanation not very accurate for rdd function 
"aggregate"

Currently, RDD function aggregate's parameter doesn't explain well, especially 
parameter "zeroValue".
It's helpful to let junior scala user know that "zeroValue" attend both "seqOp" 
and "combOp" phase.

Author: Tommy YU 

Closes #10587 from Wenpei/rdd_aggregate_doc.

(cherry picked from commit 9f0995bb0d0bbe5d9b15a1ca9fa18e246ff90d66)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/46fc7a12
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/46fc7a12
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/46fc7a12

Branch: refs/heads/branch-1.6
Commit: 46fc7a12a30b82cf1bcaab0e987a98b4dace37fe
Parents: a6c9c68
Author: Tommy YU 
Authored: Tue Jan 12 13:20:04 2016 +
Committer: Sean Owen 
Committed: Tue Jan 12 13:20:29 2016 +

--
 core/src/main/scala/org/apache/spark/rdd/RDD.scala | 14 ++
 1 file changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/46fc7a12/core/src/main/scala/org/apache/spark/rdd/RDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 9fe9d83..2fb8047 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -1071,6 +1071,13 @@ abstract class RDD[T: ClassTag](
* apply the fold to each element sequentially in some defined ordering. For 
functions
* that are not commutative, the result may differ from that of a fold 
applied to a
* non-distributed collection.
+   *
+   * @param zeroValue the initial value for the accumulated result of each 
partition for the `op`
+   *  operator, and also the initial value for the combine 
results from different
+   *  partitions for the `op` operator - this will typically 
be the neutral
+   *  element (e.g. `Nil` for list concatenation or `0` for 
summation)
+   * @param op an operator used to both accumulate results within a partition 
and combine results
+   *  from different partitions
*/
   def fold(zeroValue: T)(op: (T, T) => T): T = withScope {
 // Clone the zero value since we will also be serializing it as part of 
tasks
@@ -1089,6 +1096,13 @@ abstract class RDD[T: ClassTag](
* and one operation for merging two U's, as in scala.TraversableOnce. Both 
of these functions are
* allowed to modify and return their first argument instead of creating a 
new U to avoid memory
* allocation.
+   *
+   * @param zeroValue the initial value for the accumulated result of each 
partition for the
+   *  `seqOp` operator, and also the initial value for the 
combine results from
+   *  different partitions for the `combOp` operator - this 
will typically be the
+   *  neutral element (e.g. `Nil` for list concatenation or 
`0` for summation)
+   * @param seqOp an operator used to accumulate results within a partition
+   * @param combOp an associative operator used to combine results from 
different partitions
*/
   def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) 
=> U): U = withScope {
 // Clone the zero value since we will also be serializing it as part of 
tasks


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12638][API DOC] Parameter explanation not very accurate for rdd function "aggregate"

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9c7f34af3 -> 9f0995bb0


[SPARK-12638][API DOC] Parameter explanation not very accurate for rdd function 
"aggregate"

Currently, RDD function aggregate's parameter doesn't explain well, especially 
parameter "zeroValue".
It's helpful to let junior scala user know that "zeroValue" attend both "seqOp" 
and "combOp" phase.

Author: Tommy YU 

Closes #10587 from Wenpei/rdd_aggregate_doc.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9f0995bb
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9f0995bb
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9f0995bb

Branch: refs/heads/master
Commit: 9f0995bb0d0bbe5d9b15a1ca9fa18e246ff90d66
Parents: 9c7f34a
Author: Tommy YU 
Authored: Tue Jan 12 13:20:04 2016 +
Committer: Sean Owen 
Committed: Tue Jan 12 13:20:04 2016 +

--
 core/src/main/scala/org/apache/spark/rdd/RDD.scala | 14 ++
 1 file changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9f0995bb/core/src/main/scala/org/apache/spark/rdd/RDD.scala
--
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala 
b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index de7102f..53e01a0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -970,6 +970,13 @@ abstract class RDD[T: ClassTag](
* apply the fold to each element sequentially in some defined ordering. For 
functions
* that are not commutative, the result may differ from that of a fold 
applied to a
* non-distributed collection.
+   *
+   * @param zeroValue the initial value for the accumulated result of each 
partition for the `op`
+   *  operator, and also the initial value for the combine 
results from different
+   *  partitions for the `op` operator - this will typically 
be the neutral
+   *  element (e.g. `Nil` for list concatenation or `0` for 
summation)
+   * @param op an operator used to both accumulate results within a partition 
and combine results
+   *  from different partitions
*/
   def fold(zeroValue: T)(op: (T, T) => T): T = withScope {
 // Clone the zero value since we will also be serializing it as part of 
tasks
@@ -988,6 +995,13 @@ abstract class RDD[T: ClassTag](
* and one operation for merging two U's, as in scala.TraversableOnce. Both 
of these functions are
* allowed to modify and return their first argument instead of creating a 
new U to avoid memory
* allocation.
+   *
+   * @param zeroValue the initial value for the accumulated result of each 
partition for the
+   *  `seqOp` operator, and also the initial value for the 
combine results from
+   *  different partitions for the `combOp` operator - this 
will typically be the
+   *  neutral element (e.g. `Nil` for list concatenation or 
`0` for summation)
+   * @param seqOp an operator used to accumulate results within a partition
+   * @param combOp an associative operator used to combine results from 
different partitions
*/
   def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) 
=> U): U = withScope {
 // Clone the zero value since we will also be serializing it as part of 
tasks


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-12582][TEST] IndexShuffleBlockResolverSuite fails in windows

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 46fc7a12a -> 3221a7d91


[SPARK-12582][TEST] IndexShuffleBlockResolverSuite fails in windows

[SPARK-12582][Test] IndexShuffleBlockResolverSuite fails in windows

* IndexShuffleBlockResolverSuite fails in windows due to file is not closed.
* mv IndexShuffleBlockResolverSuite.scala from "test/java" to "test/scala".

https://issues.apache.org/jira/browse/SPARK-12582

Author: Yucai Yu 

Closes #10526 from yucai/master.

(cherry picked from commit 7e15044d9d9f9839c8d422bae71f27e855d559b4)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3221a7d9
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3221a7d9
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3221a7d9

Branch: refs/heads/branch-1.6
Commit: 3221a7d912bdc5a1ce5992501e1a2e6a8248c668
Parents: 46fc7a1
Author: Yucai Yu 
Authored: Tue Jan 12 13:23:23 2016 +
Committer: Sean Owen 
Committed: Tue Jan 12 13:23:34 2016 +

--
 .../sort/IndexShuffleBlockResolverSuite.scala   | 114 
 .../sort/IndexShuffleBlockResolverSuite.scala   | 131 +++
 2 files changed, 131 insertions(+), 114 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3221a7d9/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
--
diff --git 
a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
 
b/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
deleted file mode 100644
index 0b19861..000
--- 
a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.shuffle.sort
-
-import java.io.{File, FileInputStream, FileOutputStream}
-
-import org.mockito.Answers.RETURNS_SMART_NULLS
-import org.mockito.Matchers._
-import org.mockito.Mockito._
-import org.mockito.invocation.InvocationOnMock
-import org.mockito.stubbing.Answer
-import org.mockito.{Mock, MockitoAnnotations}
-import org.scalatest.BeforeAndAfterEach
-
-import org.apache.spark.shuffle.IndexShuffleBlockResolver
-import org.apache.spark.storage._
-import org.apache.spark.util.Utils
-import org.apache.spark.{SparkConf, SparkFunSuite}
-
-
-class IndexShuffleBlockResolverSuite extends SparkFunSuite with 
BeforeAndAfterEach {
-
-  @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = 
_
-  @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: 
DiskBlockManager = _
-
-  private var tempDir: File = _
-  private val conf: SparkConf = new SparkConf(loadDefaults = false)
-
-  override def beforeEach(): Unit = {
-tempDir = Utils.createTempDir()
-MockitoAnnotations.initMocks(this)
-
-when(blockManager.diskBlockManager).thenReturn(diskBlockManager)
-when(diskBlockManager.getFile(any[BlockId])).thenAnswer(
-  new Answer[File] {
-override def answer(invocation: InvocationOnMock): File = {
-  new File(tempDir, invocation.getArguments.head.toString)
-}
-  })
-  }
-
-  override def afterEach(): Unit = {
-Utils.deleteRecursively(tempDir)
-  }
-
-  test("commit shuffle files multiple times") {
-val lengths = Array[Long](10, 0, 20)
-val resolver = new IndexShuffleBlockResolver(conf, blockManager)
-val dataTmp = File.createTempFile("shuffle", null, tempDir)
-val out = new FileOutputStream(dataTmp)
-out.write(new Array[Byte](30))
-out.close()
-resolver.writeIndexFileAndCommit(1, 2, lengths, dataTmp)
-
-val dataFile = resolver.getDataFile(1, 2)
-assert(dataFile.exists())
-assert(dataFile.length() === 30)
-assert(!dataTmp.exists())
-
-val dataTmp2 = File.createTempFile("shuffle", null, tempDir)
-val out2 = new FileOutputStream(dataTmp2)
-

spark git commit: [SPARK-12582][TEST] IndexShuffleBlockResolverSuite fails in windows

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9f0995bb0 -> 7e15044d9


[SPARK-12582][TEST] IndexShuffleBlockResolverSuite fails in windows

[SPARK-12582][Test] IndexShuffleBlockResolverSuite fails in windows

* IndexShuffleBlockResolverSuite fails in windows due to file is not closed.
* mv IndexShuffleBlockResolverSuite.scala from "test/java" to "test/scala".

https://issues.apache.org/jira/browse/SPARK-12582

Author: Yucai Yu 

Closes #10526 from yucai/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7e15044d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7e15044d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7e15044d

Branch: refs/heads/master
Commit: 7e15044d9d9f9839c8d422bae71f27e855d559b4
Parents: 9f0995b
Author: Yucai Yu 
Authored: Tue Jan 12 13:23:23 2016 +
Committer: Sean Owen 
Committed: Tue Jan 12 13:23:23 2016 +

--
 .../sort/IndexShuffleBlockResolverSuite.scala   | 119 
 .../sort/IndexShuffleBlockResolverSuite.scala   | 136 +++
 2 files changed, 136 insertions(+), 119 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/7e15044d/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
--
diff --git 
a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
 
b/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
deleted file mode 100644
index f200ff3..000
--- 
a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * 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.shuffle.sort
-
-import java.io.{File, FileInputStream, FileOutputStream}
-
-import org.mockito.Answers.RETURNS_SMART_NULLS
-import org.mockito.Matchers._
-import org.mockito.Mockito._
-import org.mockito.invocation.InvocationOnMock
-import org.mockito.stubbing.Answer
-import org.mockito.{Mock, MockitoAnnotations}
-import org.scalatest.BeforeAndAfterEach
-
-import org.apache.spark.shuffle.IndexShuffleBlockResolver
-import org.apache.spark.storage._
-import org.apache.spark.util.Utils
-import org.apache.spark.{SparkConf, SparkFunSuite}
-
-
-class IndexShuffleBlockResolverSuite extends SparkFunSuite with 
BeforeAndAfterEach {
-
-  @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = 
_
-  @Mock(answer = RETURNS_SMART_NULLS) private var diskBlockManager: 
DiskBlockManager = _
-
-  private var tempDir: File = _
-  private val conf: SparkConf = new SparkConf(loadDefaults = false)
-
-  override def beforeEach(): Unit = {
-super.beforeEach()
-tempDir = Utils.createTempDir()
-MockitoAnnotations.initMocks(this)
-
-when(blockManager.diskBlockManager).thenReturn(diskBlockManager)
-when(diskBlockManager.getFile(any[BlockId])).thenAnswer(
-  new Answer[File] {
-override def answer(invocation: InvocationOnMock): File = {
-  new File(tempDir, invocation.getArguments.head.toString)
-}
-  })
-  }
-
-  override def afterEach(): Unit = {
-try {
-  Utils.deleteRecursively(tempDir)
-} finally {
-  super.afterEach()
-}
-  }
-
-  test("commit shuffle files multiple times") {
-val lengths = Array[Long](10, 0, 20)
-val resolver = new IndexShuffleBlockResolver(conf, blockManager)
-val dataTmp = File.createTempFile("shuffle", null, tempDir)
-val out = new FileOutputStream(dataTmp)
-out.write(new Array[Byte](30))
-out.close()
-resolver.writeIndexFileAndCommit(1, 2, lengths, dataTmp)
-
-val dataFile = resolver.getDataFile(1, 2)
-assert(dataFile.exists())
-assert(dataFile.length() === 30)
-assert(!dataTmp.exists())
-
-val dataTmp2 = File.createTempFile("shuffle", null, tempDir)
-val out2 = new FileOutputStream(dataTmp2)
-val lengths2 = new Array[Long](3)
-

spark git commit: [SPARK-7615][MLLIB] MLLIB Word2Vec wordVectors divided by Euclidean Norm equals to zero

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 4c67d55c0 -> 94b39f777


[SPARK-7615][MLLIB] MLLIB Word2Vec wordVectors divided by Euclidean Norm equals 
to zero

Cosine similarity with 0 vector should be 0

Related to https://github.com/apache/spark/pull/10152

Author: Sean Owen <so...@cloudera.com>

Closes #10696 from srowen/SPARK-7615.

(cherry picked from commit c48f2a3a5fd714ad2ff19b29337e55583988431e)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/94b39f77
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/94b39f77
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/94b39f77

Branch: refs/heads/branch-1.6
Commit: 94b39fecff3794727c186bd681fa4c6af4fd
Parents: 4c67d55
Author: Sean Owen <so...@cloudera.com>
Authored: Tue Jan 12 11:50:33 2016 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Jan 12 13:27:44 2016 +

--
 .../main/scala/org/apache/spark/mllib/feature/Word2Vec.scala  | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/94b39f77/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
index a010775..1dbedaa 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
@@ -539,7 +539,12 @@ class Word2VecModel private[spark] (
 val cosVec = cosineVec.map(_.toDouble)
 var ind = 0
 while (ind < numWords) {
-  cosVec(ind) /= wordVecNorms(ind)
+  val norm = wordVecNorms(ind)
+  if (norm == 0.0) {
+cosVec(ind) = 0.0
+  } else {
+cosVec(ind) /= norm
+  }
   ind += 1
 }
 wordList.zip(cosVec)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-5273][MLLIB][DOCS] Improve documentation examples for LinearRegression

2016-01-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master c48f2a3a5 -> 9c7f34af3


[SPARK-5273][MLLIB][DOCS] Improve documentation examples for LinearRegression

Use a much smaller step size in LinearRegressionWithSGD MLlib examples to 
achieve a reasonable RMSE.

Our training folks hit this exact same issue when concocting an example and had 
the same solution.

Author: Sean Owen <so...@cloudera.com>

Closes #10675 from srowen/SPARK-5273.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9c7f34af
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9c7f34af
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9c7f34af

Branch: refs/heads/master
Commit: 9c7f34af37ef328149c1d66b4689d80a1589e1cc
Parents: c48f2a3
Author: Sean Owen <so...@cloudera.com>
Authored: Tue Jan 12 12:13:32 2016 +
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Jan 12 12:13:32 2016 +

--
 docs/mllib-linear-methods.md | 8 +---
 1 file changed, 5 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9c7f34af/docs/mllib-linear-methods.md
--
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md
index 20b3561..aac8f75 100644
--- a/docs/mllib-linear-methods.md
+++ b/docs/mllib-linear-methods.md
@@ -590,7 +590,8 @@ val parsedData = data.map { line =>
 
 // Building the model
 val numIterations = 100
-val model = LinearRegressionWithSGD.train(parsedData, numIterations)
+val stepSize = 0.0001
+val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize)
 
 // Evaluate model on training examples and compute training error
 val valuesAndPreds = parsedData.map { point =>
@@ -655,8 +656,9 @@ public class LinearRegression {
 
 // Building the model
 int numIterations = 100;
+double stepSize = 0.0001;
 final LinearRegressionModel model =
-  LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations);
+  LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, 
stepSize);
 
 // Evaluate model on training examples and compute training error
 JavaRDD<Tuple2<Double, Double>> valuesAndPreds = parsedData.map(
@@ -706,7 +708,7 @@ data = sc.textFile("data/mllib/ridge-data/lpsa.data")
 parsedData = data.map(parsePoint)
 
 # Build the model
-model = LinearRegressionWithSGD.train(parsedData)
+model = LinearRegressionWithSGD.train(parsedData, iterations=100, 
step=0.0001)
 
 # Evaluate the model on training data
 valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features)))


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15707][SQL] Make Code Neat - Use map instead of if check.

2016-06-04 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 091f81e1f -> 0f307db5e


[SPARK-15707][SQL] Make Code Neat - Use map instead of if check.

## What changes were proposed in this pull request?
In forType function of object RandomDataGenerator, the code following:
if (maybeSqlTypeGenerator.isDefined){
  
  Some(generator)
} else{
 None
}
will be changed. Instead, maybeSqlTypeGenerator.map will be used.

## How was this patch tested?
All of the current unit tests passed.

Author: Weiqing Yang 

Closes #13448 from Sherry302/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0f307db5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0f307db5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0f307db5

Branch: refs/heads/master
Commit: 0f307db5e17e1e8a655cfa751218ac4ed88717a7
Parents: 091f81e
Author: Weiqing Yang 
Authored: Sat Jun 4 22:44:03 2016 +0100
Committer: Sean Owen 
Committed: Sat Jun 4 22:44:03 2016 +0100

--
 .../scala/org/apache/spark/sql/RandomDataGenerator.scala | 8 ++--
 1 file changed, 2 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0f307db5/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
index 711e870..8508697 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
@@ -236,9 +236,8 @@ object RandomDataGenerator {
 // convert it to catalyst value to call udt's deserialize.
 val toCatalystType = 
CatalystTypeConverters.createToCatalystConverter(udt.sqlType)
 
-if (maybeSqlTypeGenerator.isDefined) {
-  val sqlTypeGenerator = maybeSqlTypeGenerator.get
-  val generator = () => {
+maybeSqlTypeGenerator.map { sqlTypeGenerator =>
+  () => {
 val generatedScalaValue = sqlTypeGenerator.apply()
 if (generatedScalaValue == null) {
   null
@@ -246,9 +245,6 @@ object RandomDataGenerator {
   udt.deserialize(toCatalystType(generatedScalaValue))
 }
   }
-  Some(generator)
-} else {
-  None
 }
   case unsupportedType => None
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15707][SQL] Make Code Neat - Use map instead of if check.

2016-06-04 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 7e4c9dd55 -> 32a64d8fc


[SPARK-15707][SQL] Make Code Neat - Use map instead of if check.

## What changes were proposed in this pull request?
In forType function of object RandomDataGenerator, the code following:
if (maybeSqlTypeGenerator.isDefined){
  
  Some(generator)
} else{
 None
}
will be changed. Instead, maybeSqlTypeGenerator.map will be used.

## How was this patch tested?
All of the current unit tests passed.

Author: Weiqing Yang 

Closes #13448 from Sherry302/master.

(cherry picked from commit 0f307db5e17e1e8a655cfa751218ac4ed88717a7)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/32a64d8f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/32a64d8f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/32a64d8f

Branch: refs/heads/branch-2.0
Commit: 32a64d8fc9e7ddaf993bdd7e679113dc605a69a7
Parents: 7e4c9dd
Author: Weiqing Yang 
Authored: Sat Jun 4 22:44:03 2016 +0100
Committer: Sean Owen 
Committed: Sat Jun 4 22:44:12 2016 +0100

--
 .../scala/org/apache/spark/sql/RandomDataGenerator.scala | 8 ++--
 1 file changed, 2 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/32a64d8f/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
--
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
index 711e870..8508697 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala
@@ -236,9 +236,8 @@ object RandomDataGenerator {
 // convert it to catalyst value to call udt's deserialize.
 val toCatalystType = 
CatalystTypeConverters.createToCatalystConverter(udt.sqlType)
 
-if (maybeSqlTypeGenerator.isDefined) {
-  val sqlTypeGenerator = maybeSqlTypeGenerator.get
-  val generator = () => {
+maybeSqlTypeGenerator.map { sqlTypeGenerator =>
+  () => {
 val generatedScalaValue = sqlTypeGenerator.apply()
 if (generatedScalaValue == null) {
   null
@@ -246,9 +245,6 @@ object RandomDataGenerator {
   udt.deserialize(toCatalystType(generatedScalaValue))
 }
   }
-  Some(generator)
-} else {
-  None
 }
   case unsupportedType => None
 }


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR][BUILD] Add modernizr MIT license; specify "2014 and onwards" in license copyright

2016-06-04 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 2099e05f9 -> 681387b2d


[MINOR][BUILD] Add modernizr MIT license; specify "2014 and onwards" in license 
copyright

## What changes were proposed in this pull request?

Per conversation on dev list, add missing modernizr license.
Specify "2014 and onwards" in copyright statement.

## How was this patch tested?

(none required)

Author: Sean Owen <so...@cloudera.com>

Closes #13510 from srowen/ModernizrLicense.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/681387b2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/681387b2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/681387b2

Branch: refs/heads/master
Commit: 681387b2dc9a094cfba84188a1dd1ac9192bb99c
Parents: 2099e05
Author: Sean Owen <so...@cloudera.com>
Authored: Sat Jun 4 21:41:27 2016 +0100
Committer: Sean Owen <so...@cloudera.com>
Committed: Sat Jun 4 21:41:27 2016 +0100

--
 LICENSE|  1 +
 NOTICE |  2 +-
 licenses/LICENSE-modernizr.txt | 21 +
 3 files changed, 23 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/681387b2/LICENSE
--
diff --git a/LICENSE b/LICENSE
index f403640..94fd46f 100644
--- a/LICENSE
+++ b/LICENSE
@@ -296,3 +296,4 @@ The text of each license is also included at 
licenses/LICENSE-[project].txt.
  (MIT License) blockUI (http://jquery.malsup.com/block/)
  (MIT License) RowsGroup (http://datatables.net/license/mit)
  (MIT License) jsonFormatter 
(http://www.jqueryscript.net/other/jQuery-Plugin-For-Pretty-JSON-Formatting-jsonFormatter.html)
+ (MIT License) modernizr 
(https://github.com/Modernizr/Modernizr/blob/master/LICENSE)

http://git-wip-us.apache.org/repos/asf/spark/blob/681387b2/NOTICE
--
diff --git a/NOTICE b/NOTICE
index f4b1260..69b513e 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Spark
-Copyright 2014 The Apache Software Foundation.
+Copyright 2014 and onwards The Apache Software Foundation.
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/spark/blob/681387b2/licenses/LICENSE-modernizr.txt
--
diff --git a/licenses/LICENSE-modernizr.txt b/licenses/LICENSE-modernizr.txt
new file mode 100644
index 000..2bf24b9
--- /dev/null
+++ b/licenses/LICENSE-modernizr.txt
@@ -0,0 +1,21 @@
+The MIT License (MIT)
+
+Copyright (c)  
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
\ No newline at end of file


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR][BUILD] Add modernizr MIT license; specify "2014 and onwards" in license copyright

2016-06-04 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 729730159 -> ed1e20207


[MINOR][BUILD] Add modernizr MIT license; specify "2014 and onwards" in license 
copyright

## What changes were proposed in this pull request?

Per conversation on dev list, add missing modernizr license.
Specify "2014 and onwards" in copyright statement.

## How was this patch tested?

(none required)

Author: Sean Owen <so...@cloudera.com>

Closes #13510 from srowen/ModernizrLicense.

(cherry picked from commit 681387b2dc9a094cfba84188a1dd1ac9192bb99c)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ed1e2020
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ed1e2020
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ed1e2020

Branch: refs/heads/branch-2.0
Commit: ed1e20207c1c2e503a22d5ad2cdf505ef6ecbcad
Parents: 7297301
Author: Sean Owen <so...@cloudera.com>
Authored: Sat Jun 4 21:41:27 2016 +0100
Committer: Sean Owen <so...@cloudera.com>
Committed: Sat Jun 4 21:41:35 2016 +0100

--
 LICENSE|  1 +
 NOTICE |  2 +-
 licenses/LICENSE-modernizr.txt | 21 +
 3 files changed, 23 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ed1e2020/LICENSE
--
diff --git a/LICENSE b/LICENSE
index f403640..94fd46f 100644
--- a/LICENSE
+++ b/LICENSE
@@ -296,3 +296,4 @@ The text of each license is also included at 
licenses/LICENSE-[project].txt.
  (MIT License) blockUI (http://jquery.malsup.com/block/)
  (MIT License) RowsGroup (http://datatables.net/license/mit)
  (MIT License) jsonFormatter 
(http://www.jqueryscript.net/other/jQuery-Plugin-For-Pretty-JSON-Formatting-jsonFormatter.html)
+ (MIT License) modernizr 
(https://github.com/Modernizr/Modernizr/blob/master/LICENSE)

http://git-wip-us.apache.org/repos/asf/spark/blob/ed1e2020/NOTICE
--
diff --git a/NOTICE b/NOTICE
index f4b1260..69b513e 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,5 @@
 Apache Spark
-Copyright 2014 The Apache Software Foundation.
+Copyright 2014 and onwards The Apache Software Foundation.
 
 This product includes software developed at
 The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/spark/blob/ed1e2020/licenses/LICENSE-modernizr.txt
--
diff --git a/licenses/LICENSE-modernizr.txt b/licenses/LICENSE-modernizr.txt
new file mode 100644
index 000..2bf24b9
--- /dev/null
+++ b/licenses/LICENSE-modernizr.txt
@@ -0,0 +1,21 @@
+The MIT License (MIT)
+
+Copyright (c)  
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
\ No newline at end of file


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" is …

2016-06-05 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 0f307db5e -> 4e767d0f9


[SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" 
is …

## What changes were proposed in this pull request?

Stop using the abbreviated and ambiguous timezone "EST" in a test, since it is 
machine-local default timezone dependent, and fails in different timezones.  
Fixed [SPARK-15723](https://issues.apache.org/jira/browse/SPARK-15723).

## How was this patch tested?

Note that to reproduce this problem in any locale/timezone, you can modify the 
scalatest-maven-plugin argLine to add a timezone:

-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} 
-XX:ReservedCodeCacheSize=${CodeCacheSize} 
-Duser.timezone="Australia/Sydney"

and run

$ mvn test 
-DwildcardSuites=org.apache.spark.status.api.v1.SimpleDateParamSuite 
-Dtest=none. Equally this will fix it in an effected timezone:

-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} 
-XX:ReservedCodeCacheSize=${CodeCacheSize} 
-Duser.timezone="America/New_York"

To test the fix, apply the above change to `pom.xml` to set test TZ to 
`Australia/Sydney`, and confirm the test now passes.

Author: Brett Randall 

Closes #13462 from javabrett/SPARK-15723-SimpleDateParamSuite.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4e767d0f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4e767d0f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4e767d0f

Branch: refs/heads/master
Commit: 4e767d0f9042bfea6074c2637438859699ec4dc3
Parents: 0f307db
Author: Brett Randall 
Authored: Sun Jun 5 15:31:56 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 5 15:31:56 2016 +0100

--
 .../org/apache/spark/status/api/v1/SimpleDateParamSuite.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4e767d0f/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala 
b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
index 63b0e77..18baeb1 100644
--- 
a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
@@ -26,7 +26,8 @@ class SimpleDateParamSuite extends SparkFunSuite with 
Matchers {
 
   test("date parsing") {
 new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be 
(1424474477190L)
-new SimpleDateParam("2015-02-20T17:21:17.190EST").timestamp should be 
(1424470877190L)
+// don't use EST, it is ambiguous, use -0500 instead, see SPARK-15723
+new SimpleDateParam("2015-02-20T17:21:17.190-0500").timestamp should be 
(1424470877190L)
 new SimpleDateParam("2015-02-20").timestamp should be (142439040L) // 
GMT
 intercept[WebApplicationException] {
   new SimpleDateParam("invalid date")


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" is …

2016-06-05 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 32a64d8fc -> 8c0ec85e6


[SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" 
is …

## What changes were proposed in this pull request?

Stop using the abbreviated and ambiguous timezone "EST" in a test, since it is 
machine-local default timezone dependent, and fails in different timezones.  
Fixed [SPARK-15723](https://issues.apache.org/jira/browse/SPARK-15723).

## How was this patch tested?

Note that to reproduce this problem in any locale/timezone, you can modify the 
scalatest-maven-plugin argLine to add a timezone:

-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} 
-XX:ReservedCodeCacheSize=${CodeCacheSize} 
-Duser.timezone="Australia/Sydney"

and run

$ mvn test 
-DwildcardSuites=org.apache.spark.status.api.v1.SimpleDateParamSuite 
-Dtest=none. Equally this will fix it in an effected timezone:

-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} 
-XX:ReservedCodeCacheSize=${CodeCacheSize} 
-Duser.timezone="America/New_York"

To test the fix, apply the above change to `pom.xml` to set test TZ to 
`Australia/Sydney`, and confirm the test now passes.

Author: Brett Randall 

Closes #13462 from javabrett/SPARK-15723-SimpleDateParamSuite.

(cherry picked from commit 4e767d0f9042bfea6074c2637438859699ec4dc3)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8c0ec85e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8c0ec85e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8c0ec85e

Branch: refs/heads/branch-2.0
Commit: 8c0ec85e62f762c11e0686d1c35d1dfec05df9de
Parents: 32a64d8
Author: Brett Randall 
Authored: Sun Jun 5 15:31:56 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 5 16:12:24 2016 +0100

--
 .../org/apache/spark/status/api/v1/SimpleDateParamSuite.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8c0ec85e/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala 
b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
index 63b0e77..18baeb1 100644
--- 
a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
@@ -26,7 +26,8 @@ class SimpleDateParamSuite extends SparkFunSuite with 
Matchers {
 
   test("date parsing") {
 new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be 
(1424474477190L)
-new SimpleDateParam("2015-02-20T17:21:17.190EST").timestamp should be 
(1424470877190L)
+// don't use EST, it is ambiguous, use -0500 instead, see SPARK-15723
+new SimpleDateParam("2015-02-20T17:21:17.190-0500").timestamp should be 
(1424470877190L)
 new SimpleDateParam("2015-02-20").timestamp should be (142439040L) // 
GMT
 intercept[WebApplicationException] {
   new SimpleDateParam("invalid date")


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" is …

2016-06-05 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 a0cf7d0b2 -> 6a9f19dd5


[SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" 
is …

## What changes were proposed in this pull request?

Stop using the abbreviated and ambiguous timezone "EST" in a test, since it is 
machine-local default timezone dependent, and fails in different timezones.  
Fixed [SPARK-15723](https://issues.apache.org/jira/browse/SPARK-15723).

## How was this patch tested?

Note that to reproduce this problem in any locale/timezone, you can modify the 
scalatest-maven-plugin argLine to add a timezone:

-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} 
-XX:ReservedCodeCacheSize=${CodeCacheSize} 
-Duser.timezone="Australia/Sydney"

and run

$ mvn test 
-DwildcardSuites=org.apache.spark.status.api.v1.SimpleDateParamSuite 
-Dtest=none. Equally this will fix it in an effected timezone:

-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} 
-XX:ReservedCodeCacheSize=${CodeCacheSize} 
-Duser.timezone="America/New_York"

To test the fix, apply the above change to `pom.xml` to set test TZ to 
`Australia/Sydney`, and confirm the test now passes.

Author: Brett Randall 

Closes #13462 from javabrett/SPARK-15723-SimpleDateParamSuite.

(cherry picked from commit 4e767d0f9042bfea6074c2637438859699ec4dc3)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6a9f19dd
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6a9f19dd
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6a9f19dd

Branch: refs/heads/branch-1.6
Commit: 6a9f19dd57dadb80bccc328cf1d099bed04f7f18
Parents: a0cf7d0
Author: Brett Randall 
Authored: Sun Jun 5 15:31:56 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 5 16:12:49 2016 +0100

--
 .../org/apache/spark/status/api/v1/SimpleDateParamSuite.scala | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6a9f19dd/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala 
b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
index 63b0e77..18baeb1 100644
--- 
a/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/status/api/v1/SimpleDateParamSuite.scala
@@ -26,7 +26,8 @@ class SimpleDateParamSuite extends SparkFunSuite with 
Matchers {
 
   test("date parsing") {
 new SimpleDateParam("2015-02-20T23:21:17.190GMT").timestamp should be 
(1424474477190L)
-new SimpleDateParam("2015-02-20T17:21:17.190EST").timestamp should be 
(1424470877190L)
+// don't use EST, it is ambiguous, use -0500 instead, see SPARK-15723
+new SimpleDateParam("2015-02-20T17:21:17.190-0500").timestamp should be 
(1424470877190L)
 new SimpleDateParam("2015-02-20").timestamp should be (142439040L) // 
GMT
 intercept[WebApplicationException] {
   new SimpleDateParam("invalid date")


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [DOCUMENTATION] Fixed target JAR path

2016-06-08 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master f958c1c3e -> ca70ab27c


[DOCUMENTATION] Fixed target JAR path

## What changes were proposed in this pull request?

Mentioned Scala version in the sbt configuration file is 2.11, so the path of 
the target JAR should be `/target/scala-2.11/simple-project_2.11-1.0.jar`

## How was this patch tested?

n/a

Author: prabs 
Author: Prabeesh K 

Closes #13554 from prabeesh/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ca70ab27
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ca70ab27
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ca70ab27

Branch: refs/heads/master
Commit: ca70ab27cc73f6ea7fce5d179ca8f13459c8ba95
Parents: f958c1c
Author: prabs 
Authored: Wed Jun 8 17:22:55 2016 +0100
Committer: Sean Owen 
Committed: Wed Jun 8 17:22:55 2016 +0100

--
 docs/quick-start.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ca70ab27/docs/quick-start.md
--
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 72372a6..1b961fd 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -289,13 +289,13 @@ $ find .
 # Package a jar containing your application
 $ sbt package
 ...
-[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar
+[info] Packaging 
{..}/{..}/target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar
 
 # Use spark-submit to run your application
 $ YOUR_SPARK_HOME/bin/spark-submit \
   --class "SimpleApp" \
   --master local[4] \
-  target/scala-2.10/simple-project_2.10-1.0.jar
+  
target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [DOCUMENTATION] Fixed target JAR path

2016-06-08 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 5e9a8e715 -> b2778c8bb


[DOCUMENTATION] Fixed target JAR path

## What changes were proposed in this pull request?

Mentioned Scala version in the sbt configuration file is 2.11, so the path of 
the target JAR should be `/target/scala-2.11/simple-project_2.11-1.0.jar`

## How was this patch tested?

n/a

Author: prabs 
Author: Prabeesh K 

Closes #13554 from prabeesh/master.

(cherry picked from commit ca70ab27cc73f6ea7fce5d179ca8f13459c8ba95)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b2778c8b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b2778c8b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b2778c8b

Branch: refs/heads/branch-2.0
Commit: b2778c8bbdf3b3a2e650b17346f87f2568f88295
Parents: 5e9a8e7
Author: prabs 
Authored: Wed Jun 8 17:22:55 2016 +0100
Committer: Sean Owen 
Committed: Wed Jun 8 17:23:03 2016 +0100

--
 docs/quick-start.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b2778c8b/docs/quick-start.md
--
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 72372a6..1b961fd 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -289,13 +289,13 @@ $ find .
 # Package a jar containing your application
 $ sbt package
 ...
-[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar
+[info] Packaging 
{..}/{..}/target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar
 
 # Use spark-submit to run your application
 $ YOUR_SPARK_HOME/bin/spark-submit \
   --class "SimpleApp" \
   --master local[4] \
-  target/scala-2.10/simple-project_2.10-1.0.jar
+  
target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar
 ...
 Lines with a: 46, Lines with b: 23
 {% endhighlight %}


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15818][BUILD] Upgrade to Hadoop 2.7.2

2016-06-09 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 921fa40b1 -> 147c02082


[SPARK-15818][BUILD] Upgrade to Hadoop 2.7.2

## What changes were proposed in this pull request?

Updating the Hadoop version from 2.7.0 to 2.7.2 if we use the Hadoop-2.7 build 
profile

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
Existing tests

(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

I'd like us to use Hadoop 2.7.2 owing to the Hadoop release notes stating 
Hadoop 2.7.0 is not ready for production use

https://hadoop.apache.org/docs/r2.7.0/ states

"Apache Hadoop 2.7.0 is a minor release in the 2.x.y release line, building 
upon the previous stable release 2.6.0.
This release is not yet ready for production use. Production users should use 
2.7.1 release and beyond."

Hadoop 2.7.1 release notes:
"Apache Hadoop 2.7.1 is a minor release in the 2.x.y release line, building 
upon the previous release 2.7.0. This is the next stable release after Apache 
Hadoop 2.6.x."

And then Hadoop 2.7.2 release notes:
"Apache Hadoop 2.7.2 is a minor release in the 2.x.y release line, building 
upon the previous stable release 2.7.1."

I've tested this is OK with Intel hardware and IBM Java 8 so let's test it with 
OpenJDK, ideally this will be pushed to branch-2.0 and master.

Author: Adam Roberts 

Closes #13556 from a-roberts/patch-2.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/147c0208
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/147c0208
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/147c0208

Branch: refs/heads/master
Commit: 147c020823080c60b495f7950629d8134bf895db
Parents: 921fa40
Author: Adam Roberts 
Authored: Thu Jun 9 10:34:01 2016 +0100
Committer: Sean Owen 
Committed: Thu Jun 9 10:34:01 2016 +0100

--
 dev/deps/spark-deps-hadoop-2.4 | 30 +++---
 dev/deps/spark-deps-hadoop-2.6 | 30 +++---
 dev/deps/spark-deps-hadoop-2.7 | 30 +++---
 pom.xml|  6 +++---
 4 files changed, 48 insertions(+), 48 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/147c0208/dev/deps/spark-deps-hadoop-2.4
--
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index f0491ec..501bf58 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -53,21 +53,21 @@ eigenbase-properties-1.1.5.jar
 guava-14.0.1.jar
 guice-3.0.jar
 guice-servlet-3.0.jar
-hadoop-annotations-2.4.0.jar
-hadoop-auth-2.4.0.jar
-hadoop-client-2.4.0.jar
-hadoop-common-2.4.0.jar
-hadoop-hdfs-2.4.0.jar
-hadoop-mapreduce-client-app-2.4.0.jar
-hadoop-mapreduce-client-common-2.4.0.jar
-hadoop-mapreduce-client-core-2.4.0.jar
-hadoop-mapreduce-client-jobclient-2.4.0.jar
-hadoop-mapreduce-client-shuffle-2.4.0.jar
-hadoop-yarn-api-2.4.0.jar
-hadoop-yarn-client-2.4.0.jar
-hadoop-yarn-common-2.4.0.jar
-hadoop-yarn-server-common-2.4.0.jar
-hadoop-yarn-server-web-proxy-2.4.0.jar
+hadoop-annotations-2.4.1.jar
+hadoop-auth-2.4.1.jar
+hadoop-client-2.4.1.jar
+hadoop-common-2.4.1.jar
+hadoop-hdfs-2.4.1.jar
+hadoop-mapreduce-client-app-2.4.1.jar
+hadoop-mapreduce-client-common-2.4.1.jar
+hadoop-mapreduce-client-core-2.4.1.jar
+hadoop-mapreduce-client-jobclient-2.4.1.jar
+hadoop-mapreduce-client-shuffle-2.4.1.jar
+hadoop-yarn-api-2.4.1.jar
+hadoop-yarn-client-2.4.1.jar
+hadoop-yarn-common-2.4.1.jar
+hadoop-yarn-server-common-2.4.1.jar
+hadoop-yarn-server-web-proxy-2.4.1.jar
 hk2-api-2.4.0-b34.jar
 hk2-locator-2.4.0-b34.jar
 hk2-utils-2.4.0-b34.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/147c0208/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index b3dced6..b915727 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -58,21 +58,21 @@ gson-2.2.4.jar
 guava-14.0.1.jar
 guice-3.0.jar
 guice-servlet-3.0.jar
-hadoop-annotations-2.6.0.jar
-hadoop-auth-2.6.0.jar
-hadoop-client-2.6.0.jar
-hadoop-common-2.6.0.jar
-hadoop-hdfs-2.6.0.jar
-hadoop-mapreduce-client-app-2.6.0.jar
-hadoop-mapreduce-client-common-2.6.0.jar
-hadoop-mapreduce-client-core-2.6.0.jar
-hadoop-mapreduce-client-jobclient-2.6.0.jar
-hadoop-mapreduce-client-shuffle-2.6.0.jar
-hadoop-yarn-api-2.6.0.jar
-hadoop-yarn-client-2.6.0.jar
-hadoop-yarn-common-2.6.0.jar
-hadoop-yarn-server-common-2.6.0.jar
-hadoop-yarn-server-web-proxy-2.6.0.jar
+hadoop-annotations-2.6.4.jar
+hadoop-auth-2.6.4.jar
+hadoop-client-2.6.4.jar
+hadoop-common-2.6.4.jar

spark git commit: [SPARK-15818][BUILD] Upgrade to Hadoop 2.7.2

2016-06-09 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 8ee93eed9 -> 77c08d224


[SPARK-15818][BUILD] Upgrade to Hadoop 2.7.2

## What changes were proposed in this pull request?

Updating the Hadoop version from 2.7.0 to 2.7.2 if we use the Hadoop-2.7 build 
profile

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, 
manual tests)
Existing tests

(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)

I'd like us to use Hadoop 2.7.2 owing to the Hadoop release notes stating 
Hadoop 2.7.0 is not ready for production use

https://hadoop.apache.org/docs/r2.7.0/ states

"Apache Hadoop 2.7.0 is a minor release in the 2.x.y release line, building 
upon the previous stable release 2.6.0.
This release is not yet ready for production use. Production users should use 
2.7.1 release and beyond."

Hadoop 2.7.1 release notes:
"Apache Hadoop 2.7.1 is a minor release in the 2.x.y release line, building 
upon the previous release 2.7.0. This is the next stable release after Apache 
Hadoop 2.6.x."

And then Hadoop 2.7.2 release notes:
"Apache Hadoop 2.7.2 is a minor release in the 2.x.y release line, building 
upon the previous stable release 2.7.1."

I've tested this is OK with Intel hardware and IBM Java 8 so let's test it with 
OpenJDK, ideally this will be pushed to branch-2.0 and master.

Author: Adam Roberts 

Closes #13556 from a-roberts/patch-2.

(cherry picked from commit 147c020823080c60b495f7950629d8134bf895db)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/77c08d22
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/77c08d22
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/77c08d22

Branch: refs/heads/branch-2.0
Commit: 77c08d2240bef7d814fc6e4dd0a53fbdf1e2f795
Parents: 8ee93ee
Author: Adam Roberts 
Authored: Thu Jun 9 10:34:01 2016 +0100
Committer: Sean Owen 
Committed: Thu Jun 9 10:34:15 2016 +0100

--
 dev/deps/spark-deps-hadoop-2.4 | 30 +++---
 dev/deps/spark-deps-hadoop-2.6 | 30 +++---
 dev/deps/spark-deps-hadoop-2.7 | 30 +++---
 pom.xml|  6 +++---
 4 files changed, 48 insertions(+), 48 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/77c08d22/dev/deps/spark-deps-hadoop-2.4
--
diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4
index 77d5266..3df292e 100644
--- a/dev/deps/spark-deps-hadoop-2.4
+++ b/dev/deps/spark-deps-hadoop-2.4
@@ -53,21 +53,21 @@ eigenbase-properties-1.1.5.jar
 guava-14.0.1.jar
 guice-3.0.jar
 guice-servlet-3.0.jar
-hadoop-annotations-2.4.0.jar
-hadoop-auth-2.4.0.jar
-hadoop-client-2.4.0.jar
-hadoop-common-2.4.0.jar
-hadoop-hdfs-2.4.0.jar
-hadoop-mapreduce-client-app-2.4.0.jar
-hadoop-mapreduce-client-common-2.4.0.jar
-hadoop-mapreduce-client-core-2.4.0.jar
-hadoop-mapreduce-client-jobclient-2.4.0.jar
-hadoop-mapreduce-client-shuffle-2.4.0.jar
-hadoop-yarn-api-2.4.0.jar
-hadoop-yarn-client-2.4.0.jar
-hadoop-yarn-common-2.4.0.jar
-hadoop-yarn-server-common-2.4.0.jar
-hadoop-yarn-server-web-proxy-2.4.0.jar
+hadoop-annotations-2.4.1.jar
+hadoop-auth-2.4.1.jar
+hadoop-client-2.4.1.jar
+hadoop-common-2.4.1.jar
+hadoop-hdfs-2.4.1.jar
+hadoop-mapreduce-client-app-2.4.1.jar
+hadoop-mapreduce-client-common-2.4.1.jar
+hadoop-mapreduce-client-core-2.4.1.jar
+hadoop-mapreduce-client-jobclient-2.4.1.jar
+hadoop-mapreduce-client-shuffle-2.4.1.jar
+hadoop-yarn-api-2.4.1.jar
+hadoop-yarn-client-2.4.1.jar
+hadoop-yarn-common-2.4.1.jar
+hadoop-yarn-server-common-2.4.1.jar
+hadoop-yarn-server-web-proxy-2.4.1.jar
 hk2-api-2.4.0-b34.jar
 hk2-locator-2.4.0-b34.jar
 hk2-utils-2.4.0-b34.jar

http://git-wip-us.apache.org/repos/asf/spark/blob/77c08d22/dev/deps/spark-deps-hadoop-2.6
--
diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6
index 9afe50f..9540f58 100644
--- a/dev/deps/spark-deps-hadoop-2.6
+++ b/dev/deps/spark-deps-hadoop-2.6
@@ -58,21 +58,21 @@ gson-2.2.4.jar
 guava-14.0.1.jar
 guice-3.0.jar
 guice-servlet-3.0.jar
-hadoop-annotations-2.6.0.jar
-hadoop-auth-2.6.0.jar
-hadoop-client-2.6.0.jar
-hadoop-common-2.6.0.jar
-hadoop-hdfs-2.6.0.jar
-hadoop-mapreduce-client-app-2.6.0.jar
-hadoop-mapreduce-client-common-2.6.0.jar
-hadoop-mapreduce-client-core-2.6.0.jar
-hadoop-mapreduce-client-jobclient-2.6.0.jar
-hadoop-mapreduce-client-shuffle-2.6.0.jar
-hadoop-yarn-api-2.6.0.jar
-hadoop-yarn-client-2.6.0.jar
-hadoop-yarn-common-2.6.0.jar
-hadoop-yarn-server-common-2.6.0.jar

spark git commit: [SPARK-12655][GRAPHX] GraphX does not unpersist RDDs

2016-06-07 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-1.6 6a9f19dd5 -> 5830828ef


[SPARK-12655][GRAPHX] GraphX does not unpersist RDDs

Some VertexRDD and EdgeRDD are created during the intermediate step of 
g.connectedComponents() but unnecessarily left cached after the method is done. 
The fix is to unpersist these RDDs once they are no longer in use.

A test case is added to confirm the fix for the reported bug.

Author: Jason Lee 

Closes #10713 from jasoncl/SPARK-12655.

(cherry picked from commit d0a5c32bd05841f411a342a80c5da9f73f30d69a)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5830828e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5830828e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5830828e

Branch: refs/heads/branch-1.6
Commit: 5830828efbf863df510a2b5b17d76214863ff48f
Parents: 6a9f19d
Author: Jason Lee 
Authored: Fri Jan 15 12:04:05 2016 +
Committer: Sean Owen 
Committed: Tue Jun 7 09:25:04 2016 +0100

--
 .../scala/org/apache/spark/graphx/Pregel.scala |  2 +-
 .../spark/graphx/lib/ConnectedComponents.scala |  4 +++-
 .../scala/org/apache/spark/graphx/GraphSuite.scala | 17 +
 3 files changed, 21 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5830828e/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
--
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala 
b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
index 2ca60d5..8a89295 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
@@ -151,7 +151,7 @@ object Pregel extends Logging {
   // count the iteration
   i += 1
 }
-
+messages.unpersist(blocking = false)
 g
   } // end of apply
 

http://git-wip-us.apache.org/repos/asf/spark/blob/5830828e/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
--
diff --git 
a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala 
b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
index 859f896..f72cbb1 100644
--- 
a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
+++ 
b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
@@ -47,9 +47,11 @@ object ConnectedComponents {
   }
 }
 val initialMessage = Long.MaxValue
-Pregel(ccGraph, initialMessage, activeDirection = EdgeDirection.Either)(
+val pregelGraph = Pregel(ccGraph, initialMessage, activeDirection = 
EdgeDirection.Either)(
   vprog = (id, attr, msg) => math.min(attr, msg),
   sendMsg = sendMessage,
   mergeMsg = (a, b) => math.min(a, b))
+ccGraph.unpersist()
+pregelGraph
   } // end of connectedComponents
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/5830828e/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
--
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala 
b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
index 9acbd79..a46c5da 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
@@ -428,6 +428,23 @@ class GraphSuite extends SparkFunSuite with 
LocalSparkContext {
 }
   }
 
+  test("unpersist graph RDD") {
+withSpark { sc =>
+  val vert = sc.parallelize(List((1L, "a"), (2L, "b"), (3L, "c")), 1)
+  val edges = sc.parallelize(List(Edge[Long](1L, 2L), Edge[Long](1L, 3L)), 
1)
+  val g0 = Graph(vert, edges)
+  val g = g0.partitionBy(PartitionStrategy.EdgePartition2D, 2)
+  val cc = g.connectedComponents()
+  assert(sc.getPersistentRDDs.nonEmpty)
+  cc.unpersist()
+  g.unpersist()
+  g0.unpersist()
+  vert.unpersist()
+  edges.unpersist()
+  assert(sc.getPersistentRDDs.isEmpty)
+}
+  }
+
   test("SPARK-14219: pickRandomVertex") {
 withSpark { sc =>
   val vert = sc.parallelize(List((1L, "a")), 1)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR] Fix Typos 'an -> a'

2016-06-06 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 7d10e4bdd -> 90e94b826


[MINOR] Fix Typos 'an -> a'

## What changes were proposed in this pull request?

`an -> a`

Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} 
&& echo {}"` to generate candidates, and review them one by one.

## How was this patch tested?
manual tests

Author: Zheng RuiFeng 

Closes #13515 from zhengruifeng/an_a.

(cherry picked from commit fd8af397132fa1415a4c19d7f5cb5a41aa6ddb27)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/90e94b82
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/90e94b82
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/90e94b82

Branch: refs/heads/branch-2.0
Commit: 90e94b82649d9816cd4065549678b82751238552
Parents: 7d10e4b
Author: Zheng RuiFeng 
Authored: Mon Jun 6 09:35:47 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 6 09:35:57 2016 +0100

--
 R/pkg/R/utils.R   |  2 +-
 .../src/main/scala/org/apache/spark/Accumulable.scala |  2 +-
 .../org/apache/spark/api/java/JavaSparkContext.scala  |  2 +-
 .../scala/org/apache/spark/api/python/PythonRDD.scala |  2 +-
 .../scala/org/apache/spark/deploy/SparkSubmit.scala   |  6 +++---
 .../src/main/scala/org/apache/spark/rdd/JdbcRDD.scala |  6 +++---
 .../main/scala/org/apache/spark/scheduler/Pool.scala  |  2 +-
 .../org/apache/spark/broadcast/BroadcastSuite.scala   |  2 +-
 .../spark/deploy/rest/StandaloneRestSubmitSuite.scala |  2 +-
 .../test/scala/org/apache/spark/rpc/RpcEnvSuite.scala |  2 +-
 .../apache/spark/scheduler/DAGSchedulerSuite.scala|  4 ++--
 .../org/apache/spark/util/JsonProtocolSuite.scala |  2 +-
 .../spark/streaming/flume/FlumeBatchFetcher.scala |  2 +-
 .../spark/graphx/impl/VertexPartitionBaseOps.scala|  2 +-
 .../scala/org/apache/spark/ml/linalg/Vectors.scala|  2 +-
 .../src/main/scala/org/apache/spark/ml/Pipeline.scala |  2 +-
 .../spark/ml/classification/LogisticRegression.scala  |  4 ++--
 .../org/apache/spark/ml/tree/impl/RandomForest.scala  |  2 +-
 .../mllib/classification/LogisticRegression.scala |  2 +-
 .../org/apache/spark/mllib/classification/SVM.scala   |  2 +-
 .../spark/mllib/feature/VectorTransformer.scala   |  2 +-
 .../scala/org/apache/spark/mllib/linalg/Vectors.scala |  2 +-
 .../mllib/linalg/distributed/CoordinateMatrix.scala   |  2 +-
 .../apache/spark/mllib/rdd/MLPairRDDFunctions.scala   |  2 +-
 python/pyspark/ml/classification.py   |  4 ++--
 python/pyspark/ml/pipeline.py |  2 +-
 python/pyspark/mllib/classification.py|  2 +-
 python/pyspark/mllib/common.py|  2 +-
 python/pyspark/rdd.py |  4 ++--
 python/pyspark/sql/session.py |  2 +-
 python/pyspark/sql/streaming.py   |  2 +-
 python/pyspark/sql/types.py   |  2 +-
 python/pyspark/streaming/dstream.py   |  4 ++--
 .../src/main/scala/org/apache/spark/sql/Row.scala |  2 +-
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala |  4 ++--
 .../sql/catalyst/analysis/FunctionRegistry.scala  |  2 +-
 .../sql/catalyst/analysis/MultiInstanceRelation.scala |  2 +-
 .../spark/sql/catalyst/catalog/SessionCatalog.scala   |  6 +++---
 .../sql/catalyst/catalog/functionResources.scala  |  2 +-
 .../sql/catalyst/expressions/ExpectsInputTypes.scala  |  2 +-
 .../spark/sql/catalyst/expressions/Projection.scala   |  4 ++--
 .../sql/catalyst/expressions/complexTypeCreator.scala |  2 +-
 .../org/apache/spark/sql/types/AbstractDataType.scala |  2 +-
 .../scala/org/apache/spark/sql/DataFrameReader.scala  |  2 +-
 .../main/scala/org/apache/spark/sql/SQLContext.scala  | 14 +++---
 .../scala/org/apache/spark/sql/SQLImplicits.scala |  2 +-
 .../scala/org/apache/spark/sql/SparkSession.scala | 14 +++---
 .../org/apache/spark/sql/catalyst/SQLBuilder.scala|  2 +-
 .../aggregate/SortBasedAggregationIterator.scala  |  2 +-
 .../apache/spark/sql/execution/aggregate/udaf.scala   |  2 +-
 .../execution/columnar/GenerateColumnAccessor.scala   |  2 +-
 .../execution/datasources/FileSourceStrategy.scala|  2 +-
 .../execution/datasources/json/JacksonParser.scala|  2 +-
 .../datasources/parquet/CatalystRowConverter.scala|  2 +-
 .../sql/execution/exchange/ExchangeCoordinator.scala  | 10 +-
 .../spark/sql/execution/joins/SortMergeJoinExec.scala |  2 +-
 .../spark/sql/execution/r/MapPartitionsRWrapper.scala |  2 +-
 .../scala/org/apache/spark/sql/expressions/udaf.scala |  2 +-
 .../org/apache/spark/sql/internal/SharedState.scala   |  2 +-
 .../apache/spark/sql/streaming/ContinuousQuery.scala  |  

spark git commit: [MINOR] Fix Typos 'an -> a'

2016-06-06 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 32f2f95db -> fd8af3971


[MINOR] Fix Typos 'an -> a'

## What changes were proposed in this pull request?

`an -> a`

Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} 
&& echo {}"` to generate candidates, and review them one by one.

## How was this patch tested?
manual tests

Author: Zheng RuiFeng 

Closes #13515 from zhengruifeng/an_a.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fd8af397
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fd8af397
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fd8af397

Branch: refs/heads/master
Commit: fd8af397132fa1415a4c19d7f5cb5a41aa6ddb27
Parents: 32f2f95
Author: Zheng RuiFeng 
Authored: Mon Jun 6 09:35:47 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 6 09:35:47 2016 +0100

--
 R/pkg/R/utils.R   |  2 +-
 .../src/main/scala/org/apache/spark/Accumulable.scala |  2 +-
 .../org/apache/spark/api/java/JavaSparkContext.scala  |  2 +-
 .../scala/org/apache/spark/api/python/PythonRDD.scala |  2 +-
 .../scala/org/apache/spark/deploy/SparkSubmit.scala   |  6 +++---
 .../src/main/scala/org/apache/spark/rdd/JdbcRDD.scala |  6 +++---
 .../main/scala/org/apache/spark/scheduler/Pool.scala  |  2 +-
 .../org/apache/spark/broadcast/BroadcastSuite.scala   |  2 +-
 .../spark/deploy/rest/StandaloneRestSubmitSuite.scala |  2 +-
 .../test/scala/org/apache/spark/rpc/RpcEnvSuite.scala |  2 +-
 .../apache/spark/scheduler/DAGSchedulerSuite.scala|  4 ++--
 .../org/apache/spark/util/JsonProtocolSuite.scala |  2 +-
 .../spark/streaming/flume/FlumeBatchFetcher.scala |  2 +-
 .../spark/graphx/impl/VertexPartitionBaseOps.scala|  2 +-
 .../scala/org/apache/spark/ml/linalg/Vectors.scala|  2 +-
 .../src/main/scala/org/apache/spark/ml/Pipeline.scala |  2 +-
 .../spark/ml/classification/LogisticRegression.scala  |  4 ++--
 .../org/apache/spark/ml/tree/impl/RandomForest.scala  |  2 +-
 .../mllib/classification/LogisticRegression.scala |  2 +-
 .../org/apache/spark/mllib/classification/SVM.scala   |  2 +-
 .../spark/mllib/feature/VectorTransformer.scala   |  2 +-
 .../scala/org/apache/spark/mllib/linalg/Vectors.scala |  2 +-
 .../mllib/linalg/distributed/CoordinateMatrix.scala   |  2 +-
 .../apache/spark/mllib/rdd/MLPairRDDFunctions.scala   |  2 +-
 python/pyspark/ml/classification.py   |  4 ++--
 python/pyspark/ml/pipeline.py |  2 +-
 python/pyspark/mllib/classification.py|  2 +-
 python/pyspark/mllib/common.py|  2 +-
 python/pyspark/rdd.py |  4 ++--
 python/pyspark/sql/session.py |  2 +-
 python/pyspark/sql/streaming.py   |  2 +-
 python/pyspark/sql/types.py   |  2 +-
 python/pyspark/streaming/dstream.py   |  4 ++--
 .../src/main/scala/org/apache/spark/sql/Row.scala |  2 +-
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala |  4 ++--
 .../sql/catalyst/analysis/FunctionRegistry.scala  |  2 +-
 .../sql/catalyst/analysis/MultiInstanceRelation.scala |  2 +-
 .../spark/sql/catalyst/catalog/SessionCatalog.scala   |  6 +++---
 .../sql/catalyst/catalog/functionResources.scala  |  2 +-
 .../sql/catalyst/expressions/ExpectsInputTypes.scala  |  2 +-
 .../spark/sql/catalyst/expressions/Projection.scala   |  4 ++--
 .../sql/catalyst/expressions/complexTypeCreator.scala |  2 +-
 .../org/apache/spark/sql/types/AbstractDataType.scala |  2 +-
 .../scala/org/apache/spark/sql/DataFrameReader.scala  |  2 +-
 .../main/scala/org/apache/spark/sql/SQLContext.scala  | 14 +++---
 .../scala/org/apache/spark/sql/SQLImplicits.scala |  2 +-
 .../scala/org/apache/spark/sql/SparkSession.scala | 14 +++---
 .../org/apache/spark/sql/catalyst/SQLBuilder.scala|  2 +-
 .../aggregate/SortBasedAggregationIterator.scala  |  2 +-
 .../apache/spark/sql/execution/aggregate/udaf.scala   |  2 +-
 .../execution/columnar/GenerateColumnAccessor.scala   |  2 +-
 .../execution/datasources/FileSourceStrategy.scala|  2 +-
 .../execution/datasources/json/JacksonParser.scala|  2 +-
 .../datasources/parquet/CatalystRowConverter.scala|  2 +-
 .../sql/execution/exchange/ExchangeCoordinator.scala  | 10 +-
 .../spark/sql/execution/joins/SortMergeJoinExec.scala |  2 +-
 .../spark/sql/execution/r/MapPartitionsRWrapper.scala |  2 +-
 .../scala/org/apache/spark/sql/expressions/udaf.scala |  2 +-
 .../org/apache/spark/sql/internal/SharedState.scala   |  2 +-
 .../apache/spark/sql/streaming/ContinuousQuery.scala  |  2 +-
 .../org/apache/spark/sql/hive/client/HiveClient.scala |  2 +-
 .../apache/spark/sql/hive/orc/OrcFileOperator.scala   | 

spark git commit: [SPARK-15771][ML][EXAMPLES] Use 'accuracy' rather than 'precision' in many ML examples

2016-06-06 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master fd8af3971 -> a95252823


[SPARK-15771][ML][EXAMPLES] Use 'accuracy' rather than 'precision' in many ML 
examples

## What changes were proposed in this pull request?
Since [SPARK-15617](https://issues.apache.org/jira/browse/SPARK-15617) 
deprecated ```precision``` in ```MulticlassClassificationEvaluator```, many ML 
examples broken.
```python
pyspark.sql.utils.IllegalArgumentException: 
u'MulticlassClassificationEvaluator_4c3bb1d73d8cc0cedae6 parameter metricName 
given invalid value precision.'
```
We should use ```accuracy``` to replace ```precision``` in these examples.

## How was this patch tested?
Offline tests.

Author: Yanbo Liang 

Closes #13519 from yanboliang/spark-15771.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a9525282
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a9525282
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a9525282

Branch: refs/heads/master
Commit: a95252823e09939b654dd425db38dadc4100bc87
Parents: fd8af39
Author: Yanbo Liang 
Authored: Mon Jun 6 09:36:34 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 6 09:36:34 2016 +0100

--
 .../examples/ml/JavaDecisionTreeClassificationExample.java | 2 +-
 .../examples/ml/JavaGradientBoostedTreeClassifierExample.java  | 2 +-
 .../examples/ml/JavaMultilayerPerceptronClassifierExample.java | 6 +++---
 .../org/apache/spark/examples/ml/JavaNaiveBayesExample.java| 6 +++---
 .../org/apache/spark/examples/ml/JavaOneVsRestExample.java | 6 +++---
 .../spark/examples/ml/JavaRandomForestClassifierExample.java   | 2 +-
 .../src/main/python/ml/decision_tree_classification_example.py | 2 +-
 .../main/python/ml/gradient_boosted_tree_classifier_example.py | 2 +-
 .../src/main/python/ml/multilayer_perceptron_classification.py | 6 +++---
 examples/src/main/python/ml/naive_bayes_example.py | 6 +++---
 examples/src/main/python/ml/one_vs_rest_example.py | 6 +++---
 .../src/main/python/ml/random_forest_classifier_example.py | 2 +-
 .../spark/examples/ml/DecisionTreeClassificationExample.scala  | 2 +-
 .../examples/ml/GradientBoostedTreeClassifierExample.scala | 2 +-
 .../examples/ml/MultilayerPerceptronClassifierExample.scala| 6 +++---
 .../scala/org/apache/spark/examples/ml/NaiveBayesExample.scala | 6 +++---
 .../scala/org/apache/spark/examples/ml/OneVsRestExample.scala  | 6 +++---
 .../spark/examples/ml/RandomForestClassifierExample.scala  | 2 +-
 python/pyspark/ml/evaluation.py| 2 +-
 19 files changed, 37 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a9525282/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
index bdb76f0..a9c6e7f 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
@@ -90,7 +90,7 @@ public class JavaDecisionTreeClassificationExample {
 MulticlassClassificationEvaluator evaluator = new 
MulticlassClassificationEvaluator()
   .setLabelCol("indexedLabel")
   .setPredictionCol("prediction")
-  .setMetricName("precision");
+  .setMetricName("accuracy");
 double accuracy = evaluator.evaluate(predictions);
 System.out.println("Test Error = " + (1.0 - accuracy));
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a9525282/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
index 5c2e03e..3e9eb99 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
@@ -92,7 +92,7 @@ public class JavaGradientBoostedTreeClassifierExample {
 MulticlassClassificationEvaluator evaluator = new 
MulticlassClassificationEvaluator()
   .setLabelCol("indexedLabel")
   .setPredictionCol("prediction")
-  .setMetricName("precision");
+  .setMetricName("accuracy");
 double accuracy = 

spark git commit: [SPARK-15771][ML][EXAMPLES] Use 'accuracy' rather than 'precision' in many ML examples

2016-06-06 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 90e94b826 -> 86a35a229


[SPARK-15771][ML][EXAMPLES] Use 'accuracy' rather than 'precision' in many ML 
examples

## What changes were proposed in this pull request?
Since [SPARK-15617](https://issues.apache.org/jira/browse/SPARK-15617) 
deprecated ```precision``` in ```MulticlassClassificationEvaluator```, many ML 
examples broken.
```python
pyspark.sql.utils.IllegalArgumentException: 
u'MulticlassClassificationEvaluator_4c3bb1d73d8cc0cedae6 parameter metricName 
given invalid value precision.'
```
We should use ```accuracy``` to replace ```precision``` in these examples.

## How was this patch tested?
Offline tests.

Author: Yanbo Liang 

Closes #13519 from yanboliang/spark-15771.

(cherry picked from commit a95252823e09939b654dd425db38dadc4100bc87)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/86a35a22
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/86a35a22
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/86a35a22

Branch: refs/heads/branch-2.0
Commit: 86a35a22985b9e592744e6ef31453995f2322a31
Parents: 90e94b8
Author: Yanbo Liang 
Authored: Mon Jun 6 09:36:34 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 6 09:36:43 2016 +0100

--
 .../examples/ml/JavaDecisionTreeClassificationExample.java | 2 +-
 .../examples/ml/JavaGradientBoostedTreeClassifierExample.java  | 2 +-
 .../examples/ml/JavaMultilayerPerceptronClassifierExample.java | 6 +++---
 .../org/apache/spark/examples/ml/JavaNaiveBayesExample.java| 6 +++---
 .../org/apache/spark/examples/ml/JavaOneVsRestExample.java | 6 +++---
 .../spark/examples/ml/JavaRandomForestClassifierExample.java   | 2 +-
 .../src/main/python/ml/decision_tree_classification_example.py | 2 +-
 .../main/python/ml/gradient_boosted_tree_classifier_example.py | 2 +-
 .../src/main/python/ml/multilayer_perceptron_classification.py | 6 +++---
 examples/src/main/python/ml/naive_bayes_example.py | 6 +++---
 examples/src/main/python/ml/one_vs_rest_example.py | 6 +++---
 .../src/main/python/ml/random_forest_classifier_example.py | 2 +-
 .../spark/examples/ml/DecisionTreeClassificationExample.scala  | 2 +-
 .../examples/ml/GradientBoostedTreeClassifierExample.scala | 2 +-
 .../examples/ml/MultilayerPerceptronClassifierExample.scala| 6 +++---
 .../scala/org/apache/spark/examples/ml/NaiveBayesExample.scala | 6 +++---
 .../scala/org/apache/spark/examples/ml/OneVsRestExample.scala  | 6 +++---
 .../spark/examples/ml/RandomForestClassifierExample.scala  | 2 +-
 python/pyspark/ml/evaluation.py| 2 +-
 19 files changed, 37 insertions(+), 37 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/86a35a22/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
index bdb76f0..a9c6e7f 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java
@@ -90,7 +90,7 @@ public class JavaDecisionTreeClassificationExample {
 MulticlassClassificationEvaluator evaluator = new 
MulticlassClassificationEvaluator()
   .setLabelCol("indexedLabel")
   .setPredictionCol("prediction")
-  .setMetricName("precision");
+  .setMetricName("accuracy");
 double accuracy = evaluator.evaluate(predictions);
 System.out.println("Test Error = " + (1.0 - accuracy));
 

http://git-wip-us.apache.org/repos/asf/spark/blob/86a35a22/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
index 5c2e03e..3e9eb99 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java
@@ -92,7 +92,7 @@ public class JavaGradientBoostedTreeClassifierExample {
 MulticlassClassificationEvaluator evaluator = new 
MulticlassClassificationEvaluator()
   .setLabelCol("indexedLabel")
   

spark git commit: [SPARK-15793][ML] Add maxSentenceLength for ml.Word2Vec

2016-06-08 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 91fbc880b -> 87706eb66


[SPARK-15793][ML] Add maxSentenceLength for ml.Word2Vec

## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-15793

Word2vec in ML package should have maxSentenceLength method for feature parity.

## How was this patch tested?

Tested with Spark unit test.

Author: yinxusen 

Closes #13536 from yinxusen/SPARK-15793.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/87706eb6
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/87706eb6
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/87706eb6

Branch: refs/heads/master
Commit: 87706eb66cd1370862a1f8ea447484c80969e45f
Parents: 91fbc88
Author: yinxusen 
Authored: Wed Jun 8 09:18:04 2016 +0100
Committer: Sean Owen 
Committed: Wed Jun 8 09:18:04 2016 +0100

--
 .../org/apache/spark/ml/feature/Word2Vec.scala   | 19 +++
 .../apache/spark/ml/feature/Word2VecSuite.scala  |  1 +
 2 files changed, 20 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/87706eb6/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala
index 2d89eb0..33515b2 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala
@@ -87,6 +87,21 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMinCount: Int = $(minCount)
 
+  /**
+   * Sets the maximum length (in words) of each sentence in the input data.
+   * Any sentence longer than this threshold will be divided into chunks of
+   * up to `maxSentenceLength` size.
+   * Default: 1000
+   * @group param
+   */
+  final val maxSentenceLength = new IntParam(this, "maxSentenceLength", 
"Maximum length " +
+"(in words) of each sentence in the input data. Any sentence longer than 
this threshold will " +
+"be divided into chunks up to the size.")
+  setDefault(maxSentenceLength -> 1000)
+
+  /** @group getParam */
+  def getMaxSentenceLength: Int = $(maxSentenceLength)
+
   setDefault(stepSize -> 0.025)
   setDefault(maxIter -> 1)
 
@@ -137,6 +152,9 @@ final class Word2Vec(override val uid: String) extends 
Estimator[Word2VecModel]
   /** @group setParam */
   def setMinCount(value: Int): this.type = set(minCount, value)
 
+  /** @group setParam */
+  def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
+
   @Since("2.0.0")
   override def fit(dataset: Dataset[_]): Word2VecModel = {
 transformSchema(dataset.schema, logging = true)
@@ -149,6 +167,7 @@ final class Word2Vec(override val uid: String) extends 
Estimator[Word2VecModel]
   .setSeed($(seed))
   .setVectorSize($(vectorSize))
   .setWindowSize($(windowSize))
+  .setMaxSentenceLength($(maxSentenceLength))
   .fit(input)
 copyValues(new Word2VecModel(uid, wordVectors).setParent(this))
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/87706eb6/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala
--
diff --git 
a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala 
b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala
index 280a36f..16c74f6 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala
@@ -191,6 +191,7 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
   .setSeed(42L)
   .setStepSize(0.01)
   .setVectorSize(100)
+  .setMaxSentenceLength(500)
 testDefaultReadWrite(t)
   }
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r1747385 - in /spark: ./ site/ site/docs/ site/docs/2.0.0-preview/ site/docs/2.0.0-preview/api/ site/docs/2.0.0-preview/api/R/ site/docs/2.0.0-preview/api/java/ site/docs/2.0.0-preview/api

2016-06-08 Thread srowen
Author: srowen
Date: Wed Jun  8 12:04:28 2016
New Revision: 1747385

URL: http://svn.apache.org/viewvc?rev=1747385=rev
Log:
Uploaded Spark 2.0.0 preview docs and added preview docs section on site


[This commit notification would consist of 1214 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR] fix typo in documents

2016-06-07 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 57dd4efcd -> a7e9e60df


[MINOR] fix typo in documents

## What changes were proposed in this pull request?

I use spell check tools checks typo in spark documents and fix them.

## How was this patch tested?

N/A

Author: WeichenXu 

Closes #13538 from WeichenXu123/fix_doc_typo.

(cherry picked from commit 1e2c9311871968426e019164b129652fd6d0037f)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a7e9e60d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a7e9e60d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a7e9e60d

Branch: refs/heads/branch-2.0
Commit: a7e9e60df5c10a90c06883ea3203ec895b9b1f82
Parents: 57dd4ef
Author: WeichenXu 
Authored: Tue Jun 7 13:29:27 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 7 13:29:36 2016 +0100

--
 docs/graphx-programming-guide.md| 2 +-
 docs/hardware-provisioning.md   | 2 +-
 docs/streaming-programming-guide.md | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a7e9e60d/docs/graphx-programming-guide.md
--
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 9dea9b5..81cf174 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -132,7 +132,7 @@ var graph: Graph[VertexProperty, String] = null
 
 Like RDDs, property graphs are immutable, distributed, and fault-tolerant.  
Changes to the values or
 structure of the graph are accomplished by producing a new graph with the 
desired changes.  Note
-that substantial parts of the original graph (i.e., unaffected structure, 
attributes, and indicies)
+that substantial parts of the original graph (i.e., unaffected structure, 
attributes, and indices)
 are reused in the new graph reducing the cost of this inherently functional 
data structure.  The
 graph is partitioned across the executors using a range of vertex partitioning 
heuristics.  As with
 RDDs, each partition of the graph can be recreated on a different machine in 
the event of a failure.

http://git-wip-us.apache.org/repos/asf/spark/blob/a7e9e60d/docs/hardware-provisioning.md
--
diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md
index 60ecb4f..bb6f616 100644
--- a/docs/hardware-provisioning.md
+++ b/docs/hardware-provisioning.md
@@ -22,7 +22,7 @@ Hadoop and Spark on a common cluster manager like 
[Mesos](running-on-mesos.html)
 
 * If this is not possible, run Spark on different nodes in the same local-area 
network as HDFS.
 
-* For low-latency data stores like HBase, it may be preferrable to run 
computing jobs on different
+* For low-latency data stores like HBase, it may be preferable to run 
computing jobs on different
 nodes than the storage system to avoid interference.
 
 # Local Disks

http://git-wip-us.apache.org/repos/asf/spark/blob/a7e9e60d/docs/streaming-programming-guide.md
--
diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index 78ae6a7..0a6a039 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -1259,7 +1259,7 @@ dstream.foreachRDD(sendRecord)
 
 
 This is incorrect as this requires the connection object to be serialized and 
sent from the
-driver to the worker. Such connection objects are rarely transferrable across 
machines. This
+driver to the worker. Such connection objects are rarely transferable across 
machines. This
 error may manifest as serialization errors (connection object not 
serializable), initialization
 errors (connection object needs to be initialized at the workers), etc. The 
correct solution is
 to create the connection object at the worker.


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [MINOR] fix typo in documents

2016-06-07 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 5f731d685 -> 1e2c93118


[MINOR] fix typo in documents

## What changes were proposed in this pull request?

I use spell check tools checks typo in spark documents and fix them.

## How was this patch tested?

N/A

Author: WeichenXu 

Closes #13538 from WeichenXu123/fix_doc_typo.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1e2c9311
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1e2c9311
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1e2c9311

Branch: refs/heads/master
Commit: 1e2c9311871968426e019164b129652fd6d0037f
Parents: 5f731d6
Author: WeichenXu 
Authored: Tue Jun 7 13:29:27 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 7 13:29:27 2016 +0100

--
 docs/graphx-programming-guide.md| 2 +-
 docs/hardware-provisioning.md   | 2 +-
 docs/streaming-programming-guide.md | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/1e2c9311/docs/graphx-programming-guide.md
--
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 9dea9b5..81cf174 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -132,7 +132,7 @@ var graph: Graph[VertexProperty, String] = null
 
 Like RDDs, property graphs are immutable, distributed, and fault-tolerant.  
Changes to the values or
 structure of the graph are accomplished by producing a new graph with the 
desired changes.  Note
-that substantial parts of the original graph (i.e., unaffected structure, 
attributes, and indicies)
+that substantial parts of the original graph (i.e., unaffected structure, 
attributes, and indices)
 are reused in the new graph reducing the cost of this inherently functional 
data structure.  The
 graph is partitioned across the executors using a range of vertex partitioning 
heuristics.  As with
 RDDs, each partition of the graph can be recreated on a different machine in 
the event of a failure.

http://git-wip-us.apache.org/repos/asf/spark/blob/1e2c9311/docs/hardware-provisioning.md
--
diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md
index 60ecb4f..bb6f616 100644
--- a/docs/hardware-provisioning.md
+++ b/docs/hardware-provisioning.md
@@ -22,7 +22,7 @@ Hadoop and Spark on a common cluster manager like 
[Mesos](running-on-mesos.html)
 
 * If this is not possible, run Spark on different nodes in the same local-area 
network as HDFS.
 
-* For low-latency data stores like HBase, it may be preferrable to run 
computing jobs on different
+* For low-latency data stores like HBase, it may be preferable to run 
computing jobs on different
 nodes than the storage system to avoid interference.
 
 # Local Disks

http://git-wip-us.apache.org/repos/asf/spark/blob/1e2c9311/docs/streaming-programming-guide.md
--
diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index 78ae6a7..0a6a039 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -1259,7 +1259,7 @@ dstream.foreachRDD(sendRecord)
 
 
 This is incorrect as this requires the connection object to be serialized and 
sent from the
-driver to the worker. Such connection objects are rarely transferrable across 
machines. This
+driver to the worker. Such connection objects are rarely transferable across 
machines. This
 error may manifest as serialization errors (connection object not 
serializable), initialization
 errors (connection object needs to be initialized at the workers), etc. The 
correct solution is
 to create the connection object at the worker.


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-14900][ML][PYSPARK] Add accuracy and deprecate precison, recall, f1

2016-06-06 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 86a35a229 -> e38ff70e6


[SPARK-14900][ML][PYSPARK] Add accuracy and deprecate precison,recall,f1

## What changes were proposed in this pull request?
1, add accuracy for MulticlassMetrics
2, deprecate overall precision,recall,f1 and recommend accuracy usage

## How was this patch tested?
manual tests in pyspark shell

Author: Zheng RuiFeng 

Closes #13511 from zhengruifeng/deprecate_py_precisonrecall.

(cherry picked from commit 00ad4f054cd044e17d29b7c2c62efd8616462619)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e38ff70e
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e38ff70e
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e38ff70e

Branch: refs/heads/branch-2.0
Commit: e38ff70e6bacf1c85edc390d28f8a8d5ecc6cbc3
Parents: 86a35a2
Author: Zheng RuiFeng 
Authored: Mon Jun 6 15:19:22 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 6 15:19:38 2016 +0100

--
 python/pyspark/mllib/evaluation.py | 18 ++
 1 file changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e38ff70e/python/pyspark/mllib/evaluation.py
--
diff --git a/python/pyspark/mllib/evaluation.py 
b/python/pyspark/mllib/evaluation.py
index 5f32f09..2eaac87 100644
--- a/python/pyspark/mllib/evaluation.py
+++ b/python/pyspark/mllib/evaluation.py
@@ -15,6 +15,8 @@
 # limitations under the License.
 #
 
+import warnings
+
 from pyspark import since
 from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc
 from pyspark.sql import SQLContext
@@ -181,6 +183,8 @@ class MulticlassMetrics(JavaModelWrapper):
 0.66...
 >>> metrics.recall()
 0.66...
+>>> metrics.accuracy()
+0.66...
 >>> metrics.weightedFalsePositiveRate
 0.19...
 >>> metrics.weightedPrecision
@@ -233,6 +237,8 @@ class MulticlassMetrics(JavaModelWrapper):
 Returns precision or precision for a given label (category) if 
specified.
 """
 if label is None:
+# note:: Deprecated in 2.0.0. Use accuracy.
+warnings.warn("Deprecated in 2.0.0. Use accuracy.")
 return self.call("precision")
 else:
 return self.call("precision", float(label))
@@ -243,6 +249,8 @@ class MulticlassMetrics(JavaModelWrapper):
 Returns recall or recall for a given label (category) if specified.
 """
 if label is None:
+# note:: Deprecated in 2.0.0. Use accuracy.
+warnings.warn("Deprecated in 2.0.0. Use accuracy.")
 return self.call("recall")
 else:
 return self.call("recall", float(label))
@@ -254,6 +262,8 @@ class MulticlassMetrics(JavaModelWrapper):
 """
 if beta is None:
 if label is None:
+# note:: Deprecated in 2.0.0. Use accuracy.
+warnings.warn("Deprecated in 2.0.0. Use accuracy.")
 return self.call("fMeasure")
 else:
 return self.call("fMeasure", label)
@@ -263,6 +273,14 @@ class MulticlassMetrics(JavaModelWrapper):
 else:
 return self.call("fMeasure", label, beta)
 
+@since('2.0.0')
+def accuracy(self):
+"""
+Returns accuracy (equals to the total number of correctly classified 
instances
+out of the total number of instances).
+"""
+return self.call("accuracy")
+
 @property
 @since('1.4.0')
 def weightedTruePositiveRate(self):


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-14900][ML][PYSPARK] Add accuracy and deprecate precison, recall, f1

2016-06-06 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master a95252823 -> 00ad4f054


[SPARK-14900][ML][PYSPARK] Add accuracy and deprecate precison,recall,f1

## What changes were proposed in this pull request?
1, add accuracy for MulticlassMetrics
2, deprecate overall precision,recall,f1 and recommend accuracy usage

## How was this patch tested?
manual tests in pyspark shell

Author: Zheng RuiFeng 

Closes #13511 from zhengruifeng/deprecate_py_precisonrecall.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/00ad4f05
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/00ad4f05
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/00ad4f05

Branch: refs/heads/master
Commit: 00ad4f054cd044e17d29b7c2c62efd8616462619
Parents: a952528
Author: Zheng RuiFeng 
Authored: Mon Jun 6 15:19:22 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 6 15:19:22 2016 +0100

--
 python/pyspark/mllib/evaluation.py | 18 ++
 1 file changed, 18 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/00ad4f05/python/pyspark/mllib/evaluation.py
--
diff --git a/python/pyspark/mllib/evaluation.py 
b/python/pyspark/mllib/evaluation.py
index 5f32f09..2eaac87 100644
--- a/python/pyspark/mllib/evaluation.py
+++ b/python/pyspark/mllib/evaluation.py
@@ -15,6 +15,8 @@
 # limitations under the License.
 #
 
+import warnings
+
 from pyspark import since
 from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc
 from pyspark.sql import SQLContext
@@ -181,6 +183,8 @@ class MulticlassMetrics(JavaModelWrapper):
 0.66...
 >>> metrics.recall()
 0.66...
+>>> metrics.accuracy()
+0.66...
 >>> metrics.weightedFalsePositiveRate
 0.19...
 >>> metrics.weightedPrecision
@@ -233,6 +237,8 @@ class MulticlassMetrics(JavaModelWrapper):
 Returns precision or precision for a given label (category) if 
specified.
 """
 if label is None:
+# note:: Deprecated in 2.0.0. Use accuracy.
+warnings.warn("Deprecated in 2.0.0. Use accuracy.")
 return self.call("precision")
 else:
 return self.call("precision", float(label))
@@ -243,6 +249,8 @@ class MulticlassMetrics(JavaModelWrapper):
 Returns recall or recall for a given label (category) if specified.
 """
 if label is None:
+# note:: Deprecated in 2.0.0. Use accuracy.
+warnings.warn("Deprecated in 2.0.0. Use accuracy.")
 return self.call("recall")
 else:
 return self.call("recall", float(label))
@@ -254,6 +262,8 @@ class MulticlassMetrics(JavaModelWrapper):
 """
 if beta is None:
 if label is None:
+# note:: Deprecated in 2.0.0. Use accuracy.
+warnings.warn("Deprecated in 2.0.0. Use accuracy.")
 return self.call("fMeasure")
 else:
 return self.call("fMeasure", label)
@@ -263,6 +273,14 @@ class MulticlassMetrics(JavaModelWrapper):
 else:
 return self.call("fMeasure", label, beta)
 
+@since('2.0.0')
+def accuracy(self):
+"""
+Returns accuracy (equals to the total number of correctly classified 
instances
+out of the total number of instances).
+"""
+return self.call("accuracy")
+
 @property
 @since('1.4.0')
 def weightedTruePositiveRate(self):


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



svn commit: r1747061 - in /spark: downloads.md js/downloads.js site/downloads.html site/js/downloads.js

2016-06-06 Thread srowen
Author: srowen
Date: Mon Jun  6 19:56:07 2016
New Revision: 1747061

URL: http://svn.apache.org/viewvc?rev=1747061=rev
Log:
SPARK-15778 add spark-2.0.0-preview release to options and other minor related 
updates

Modified:
spark/downloads.md
spark/js/downloads.js
spark/site/downloads.html
spark/site/js/downloads.js

Modified: spark/downloads.md
URL: 
http://svn.apache.org/viewvc/spark/downloads.md?rev=1747061=1747060=1747061=diff
==
--- spark/downloads.md (original)
+++ spark/downloads.md Mon Jun  6 19:56:07 2016
@@ -16,7 +16,7 @@ $(document).ready(function() {
 
 ## Download Apache Spark
 
-Our latest version is Apache Spark 1.6.1, released on March 9, 2016
+Our latest stable version is Apache Spark 1.6.1, released on March 9, 2016
 (release notes)
 https://github.com/apache/spark/releases/tag/v1.6.1;>(git 
tag)
 
@@ -36,6 +36,17 @@ Our latest version is Apache Spark 1.6.1
 _Note: Scala 2.11 users should download the Spark source package and build
 [with Scala 2.11 
support](http://spark.apache.org/docs/latest/building-spark.html#building-for-scala-211)._
 
+### Latest Preview Release
+
+Preview releases, as the name suggests, are releases for previewing upcoming 
features.
+Unlike nightly packages, preview releases have been audited by the project's 
management committee
+to satisfy the legal requirements of Apache Software Foundation's release 
policy.
+Preview releases are not meant to be functional, i.e. they can and highly 
likely will contain
+critical bugs or documentation errors.
+
+The latest preview release is Spark 2.0.0-preview, published on May 24, 2016.
+You can select and download it above.
+
 ### Link with Spark
 Spark artifacts are [hosted in Maven 
Central](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22).
 You can add a Maven dependency with the following coordinates:
 
@@ -54,14 +65,9 @@ If you are interested in working with th
 
 Once you've downloaded Spark, you can find instructions for installing and 
building it on the documentation 
page.
 
-Stable Releases
-
-
-### Latest Preview Release (Spark 2.0.0-preview)
-Preview releases, as the name suggests, are releases for previewing upcoming 
features. Unlike nightly packages, preview releases have been audited by the 
project's management committee to satisfy the legal requirements of Apache 
Software Foundation's release policy.Preview releases are not meant to be 
functional, i.e. they can and highly likely will contain critical bugs or 
documentation errors.
-
-The latest preview release is Spark 2.0.0-preview, published on May 24, 2016. 
You can https://dist.apache.org/repos/dist/release/spark/spark-2.0.0-preview/;>download
 it here.
+### Release Notes for Stable Releases
 
+
 
 ### Nightly Packages and Artifacts
 For developers, Spark maintains nightly builds and SNAPSHOT artifacts. More 
information is available on the [Spark developer 
Wiki](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-NightlyBuilds).

Modified: spark/js/downloads.js
URL: 
http://svn.apache.org/viewvc/spark/js/downloads.js?rev=1747061=1747060=1747061=diff
==
--- spark/js/downloads.js (original)
+++ spark/js/downloads.js Mon Jun  6 19:56:07 2016
@@ -3,8 +3,8 @@
 
 releases = {};
 
-function addRelease(version, releaseDate, packages, downloadable) {
-  releases[version] = {released: releaseDate, packages: packages, 
downloadable: downloadable};
+function addRelease(version, releaseDate, packages, downloadable, stable) {
+  releases[version] = {released: releaseDate, packages: packages, 
downloadable: downloadable, stable: stable};
 }
 
 var sources = {pretty: "Source Code [can build several Hadoop versions]", tag: 
"sources"};
@@ -13,8 +13,9 @@ var hadoop1 = {pretty: "Pre-built for Ha
 var cdh4 = {pretty: "Pre-built for CDH 4", tag: "cdh4"};
 var hadoop2 = {pretty: "Pre-built for Hadoop 2.2", tag: "hadoop2"};
 var hadoop2p3 = {pretty: "Pre-built for Hadoop 2.3", tag: "hadoop2.3"};
-var hadoop2p4 = {pretty: "Pre-built for Hadoop 2.4 and later", tag: 
"hadoop2.4"};
-var hadoop2p6 = {pretty: "Pre-built for Hadoop 2.6 and later", tag: 
"hadoop2.6"};
+var hadoop2p4 = {pretty: "Pre-built for Hadoop 2.4", tag: "hadoop2.4"};
+var hadoop2p6 = {pretty: "Pre-built for Hadoop 2.6", tag: "hadoop2.6"};
+var hadoop2p7 = {pretty: "Pre-built for Hadoop 2.7 and later", tag: 
"hadoop2.7"};
 var mapr3 = {pretty: "Pre-built for MapR 3.X", tag: "mapr3"};
 var mapr4 = {pretty: "Pre-built for MapR 4.X", tag: "mapr4"};
 
@@ -31,32 +32,35 @@ var packagesV4 = [hadoop2p4, hadoop2p3,
 var packagesV5 = [hadoop2p6].concat(packages

svn commit: r1747076 - in /spark: js/downloads.js site/js/downloads.js

2016-06-06 Thread srowen
Author: srowen
Date: Mon Jun  6 20:59:54 2016
New Revision: 1747076

URL: http://svn.apache.org/viewvc?rev=1747076=rev
Log:
SPARK-15778 part 2: group preview/stable releases in download version dropdown

Modified:
spark/js/downloads.js
spark/site/js/downloads.js

Modified: spark/js/downloads.js
URL: 
http://svn.apache.org/viewvc/spark/js/downloads.js?rev=1747076=1747075=1747076=diff
==
--- spark/js/downloads.js (original)
+++ spark/js/downloads.js Mon Jun  6 20:59:54 2016
@@ -53,18 +53,18 @@ addRelease("1.1.0", new Date("9/11/2014"
 addRelease("1.0.2", new Date("8/5/2014"), sources.concat(packagesV3), true, 
true);
 addRelease("1.0.1", new Date("7/11/2014"), sources.concat(packagesV3), false, 
true);
 addRelease("1.0.0", new Date("5/30/2014"), sources.concat(packagesV2), false, 
true);
-addRelease("0.9.2", new Date("7/23/2014"), sources.concat(packagesV2), true, 
false);
-addRelease("0.9.1", new Date("4/9/2014"), sources.concat(packagesV2), false, 
false);
-addRelease("0.9.0-incubating", new Date("2/2/2014"), 
sources.concat(packagesV2), false, false);
-addRelease("0.8.1-incubating", new Date("12/19/2013"), 
sources.concat(packagesV2), true, false);
-addRelease("0.8.0-incubating", new Date("9/25/2013"), 
sources.concat(packagesV1), true, false);
-addRelease("0.7.3", new Date("7/16/2013"), sources.concat(packagesV1), true, 
false);
-addRelease("0.7.2", new Date("2/6/2013"), sources.concat(packagesV1), false, 
false);
-addRelease("0.7.0", new Date("2/27/2013"), sources, false, false);
+addRelease("0.9.2", new Date("7/23/2014"), sources.concat(packagesV2), true, 
true);
+addRelease("0.9.1", new Date("4/9/2014"), sources.concat(packagesV2), false, 
true);
+addRelease("0.9.0-incubating", new Date("2/2/2014"), 
sources.concat(packagesV2), false, true);
+addRelease("0.8.1-incubating", new Date("12/19/2013"), 
sources.concat(packagesV2), true, true);
+addRelease("0.8.0-incubating", new Date("9/25/2013"), 
sources.concat(packagesV1), true, true);
+addRelease("0.7.3", new Date("7/16/2013"), sources.concat(packagesV1), true, 
true);
+addRelease("0.7.2", new Date("2/6/2013"), sources.concat(packagesV1), false, 
true);
+addRelease("0.7.0", new Date("2/27/2013"), sources, false, true);
 
 function append(el, contents) {
-  el.innerHTML = el.innerHTML + contents;
-};
+  el.innerHTML += contents;
+}
 
 function empty(el) {
   el.innerHTML = "";
@@ -79,27 +79,25 @@ function versionShort(version) { return
 function initDownloads() {
   var versionSelect = document.getElementById("sparkVersionSelect");
 
-  // Populate versions
-  var markedDefault = false;
+  // Populate stable versions
+  append(versionSelect, "");
   for (var version in releases) {
+if (!releases[version].downloadable || !releases[version].stable) { 
continue; }
 var releaseDate = releases[version].released;
-var downloadable = releases[version].downloadable;
-var stable = releases[version].stable;
-
-if (!downloadable) { continue; }
-
-var selected = false;
-if (!markedDefault && stable) {
-  selected = true;
-  markedDefault = true;
-}
+var title = versionShort(version) + " (" + 
releaseDate.toDateString().slice(4) + ")";
+append(versionSelect, "" + title + 
"");
+  }
+  append(versionSelect, "");
 
-// Don't display incubation status here
+  // Populate other versions
+  append(versionSelect, "");
+  for (var version in releases) {
+if (!releases[version].downloadable || releases[version].stable) { 
continue; }
+var releaseDate = releases[version].released;
 var title = versionShort(version) + " (" + 
releaseDate.toDateString().slice(4) + ")";
-append(versionSelect, 
-  "" +
-  title + "");
+append(versionSelect, "" + title + 
"");
   }
+  append(versionSelect, "");
 
   // Populate packages and (transitively) releases
   onVersionSelect();

Modified: spark/site/js/downloads.js
URL: 
http://svn.apache.org/viewvc/spark/site/js/downloads.js?rev=1747076=1747075=1747076=diff
==
--- spark/site/js/downloads.js (original)
+++ spark/site/js/downloads.js Mon Jun  6 20:59:54 2016
@@ -53,18 +53,18 @@ addRelease("1.1.0", new Date("9/11/2014"
 addRelease("1.0.2", new Date("8/5/2014"), sources.concat(packagesV3), tr

spark git commit: [SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and ReplayListenerSuite

2016-06-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 d494a483a -> 879e8fd09


[SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and 
ReplayListenerSuite

## What changes were proposed in this pull request?

These tests weren't properly using `LocalSparkContext` so weren't cleaning up 
correctly when tests failed.

## How was this patch tested?

Jenkins.

Author: Imran Rashid 

Closes #13602 from squito/SPARK-15878_cleanup_replaylistener.

(cherry picked from commit 8cc22b0085475a188f229536b4f83988ae889a8e)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/879e8fd0
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/879e8fd0
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/879e8fd0

Branch: refs/heads/branch-2.0
Commit: 879e8fd09477fc78d66c9da9e0e117a513b0b046
Parents: d494a48
Author: Imran Rashid 
Authored: Sun Jun 12 12:54:57 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 12 12:55:17 2016 +0100

--
 .../org/apache/spark/scheduler/EventLoggingListenerSuite.scala | 2 +-
 .../scala/org/apache/spark/scheduler/ReplayListenerSuite.scala | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/879e8fd0/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
 
b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index 176d893..c4c80b5 100644
--- 
a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -181,7 +181,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with 
LocalSparkContext wit
 // into SPARK-6688.
 val conf = getLoggingConf(testDirPath, compressionCodec)
   .set("spark.hadoop.fs.defaultFS", "unsupported://example.com")
-val sc = new SparkContext("local-cluster[2,2,1024]", "test", conf)
+sc = new SparkContext("local-cluster[2,2,1024]", "test", conf)
 assert(sc.eventLogger.isDefined)
 val eventLogger = sc.eventLogger.get
 val eventLogPath = eventLogger.logPath

http://git-wip-us.apache.org/repos/asf/spark/blob/879e8fd0/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
index 35215c1..1732aca 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
@@ -23,7 +23,7 @@ import java.net.URI
 import org.json4s.jackson.JsonMethods._
 import org.scalatest.BeforeAndAfter
 
-import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, 
SparkFunSuite}
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils}
@@ -31,7 +31,7 @@ import org.apache.spark.util.{JsonProtocol, 
JsonProtocolSuite, Utils}
 /**
  * Test whether ReplayListenerBus replays events from logs correctly.
  */
-class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter {
+class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with 
LocalSparkContext {
   private val fileSystem = Utils.getHadoopFileSystem("/",
 SparkHadoopUtil.get.newConfiguration(new SparkConf()))
   private var testDir: File = _
@@ -101,7 +101,7 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter {
 fileSystem.mkdirs(logDirPath)
 
 val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName)
-val sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf)
+sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf)
 
 // Run a few jobs
 sc.parallelize(1 to 100, 1).count()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15806][DOCUMENTATION] update doc for SPARK_MASTER_IP

2016-06-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 3fd3ee038 -> 50248dcff


[SPARK-15806][DOCUMENTATION] update doc for SPARK_MASTER_IP

## What changes were proposed in this pull request?

SPARK_MASTER_IP is a deprecated environment variable. It is replaced by 
SPARK_MASTER_HOST according to MasterArguments.scala.

## How was this patch tested?

Manually verified.

Author: bomeng 

Closes #13543 from bomeng/SPARK-15806.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/50248dcf
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/50248dcf
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/50248dcf

Branch: refs/heads/master
Commit: 50248dcfff3ba79b73323f3a804c1e19a8be6097
Parents: 3fd3ee0
Author: bomeng 
Authored: Sun Jun 12 14:25:48 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 12 14:25:48 2016 +0100

--
 conf/spark-env.sh.template   | 2 +-
 .../org/apache/spark/deploy/master/MasterArguments.scala | 8 +++-
 docs/spark-standalone.md | 4 ++--
 sbin/start-master.sh | 6 +++---
 sbin/start-slaves.sh | 6 +++---
 5 files changed, 16 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/50248dcf/conf/spark-env.sh.template
--
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 9cffdc3..c750c72 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -42,7 +42,7 @@
 # - SPARK_DRIVER_MEMORY, Memory for Driver (e.g. 1000M, 2G) (Default: 1G)
 
 # Options for the daemons used in the standalone deploy mode
-# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname
+# - SPARK_MASTER_HOST, to bind the master to a different IP address or hostname
 # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for 
the master
 # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. 
"-Dx=y")
 # - SPARK_WORKER_CORES, to set the number of cores to use on this machine

http://git-wip-us.apache.org/repos/asf/spark/blob/50248dcf/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 585e083..c63793c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -20,18 +20,24 @@ package org.apache.spark.deploy.master
 import scala.annotation.tailrec
 
 import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
 import org.apache.spark.util.{IntParam, Utils}
 
 /**
  * Command-line parser for the master.
  */
-private[master] class MasterArguments(args: Array[String], conf: SparkConf) {
+private[master] class MasterArguments(args: Array[String], conf: SparkConf) 
extends Logging {
   var host = Utils.localHostName()
   var port = 7077
   var webUiPort = 8080
   var propertiesFile: String = null
 
   // Check for settings in environment variables
+  if (System.getenv("SPARK_MASTER_IP") != null) {
+logWarning("SPARK_MASTER_IP is deprecated, please use SPARK_MASTER_HOST")
+host = System.getenv("SPARK_MASTER_IP")
+  }
+
   if (System.getenv("SPARK_MASTER_HOST") != null) {
 host = System.getenv("SPARK_MASTER_HOST")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/50248dcf/docs/spark-standalone.md
--
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 40c7293..c864c90 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -94,8 +94,8 @@ You can optionally configure the cluster further by setting 
environment variable
 
   Environment VariableMeaning
   
-SPARK_MASTER_IP
-Bind the master to a specific IP address, for example a public 
one.
+SPARK_MASTER_HOST
+Bind the master to a specific hostname or IP address, for example a 
public one.
   
   
 SPARK_MASTER_PORT

http://git-wip-us.apache.org/repos/asf/spark/blob/50248dcf/sbin/start-master.sh
--
diff --git a/sbin/start-master.sh b/sbin/start-master.sh
index ce7f177..981cb15 100755
--- a/sbin/start-master.sh
+++ b/sbin/start-master.sh
@@ -47,8 +47,8 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then
   SPARK_MASTER_PORT=7077
 fi
 
-if [ "$SPARK_MASTER_IP" = "" ]; then
-  SPARK_MASTER_IP=`hostname`

spark git commit: [SPARK-15806][DOCUMENTATION] update doc for SPARK_MASTER_IP

2016-06-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 8c294f4ad -> b75d1c201


[SPARK-15806][DOCUMENTATION] update doc for SPARK_MASTER_IP

## What changes were proposed in this pull request?

SPARK_MASTER_IP is a deprecated environment variable. It is replaced by 
SPARK_MASTER_HOST according to MasterArguments.scala.

## How was this patch tested?

Manually verified.

Author: bomeng 

Closes #13543 from bomeng/SPARK-15806.

(cherry picked from commit 50248dcfff3ba79b73323f3a804c1e19a8be6097)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b75d1c20
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b75d1c20
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b75d1c20

Branch: refs/heads/branch-2.0
Commit: b75d1c20131b438999645d0be6ea5765a2f7da80
Parents: 8c294f4
Author: bomeng 
Authored: Sun Jun 12 14:25:48 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 12 14:25:56 2016 +0100

--
 conf/spark-env.sh.template   | 2 +-
 .../org/apache/spark/deploy/master/MasterArguments.scala | 8 +++-
 docs/spark-standalone.md | 4 ++--
 sbin/start-master.sh | 6 +++---
 sbin/start-slaves.sh | 6 +++---
 5 files changed, 16 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/b75d1c20/conf/spark-env.sh.template
--
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 9cffdc3..c750c72 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -42,7 +42,7 @@
 # - SPARK_DRIVER_MEMORY, Memory for Driver (e.g. 1000M, 2G) (Default: 1G)
 
 # Options for the daemons used in the standalone deploy mode
-# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname
+# - SPARK_MASTER_HOST, to bind the master to a different IP address or hostname
 # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for 
the master
 # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. 
"-Dx=y")
 # - SPARK_WORKER_CORES, to set the number of cores to use on this machine

http://git-wip-us.apache.org/repos/asf/spark/blob/b75d1c20/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
--
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 585e083..c63793c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -20,18 +20,24 @@ package org.apache.spark.deploy.master
 import scala.annotation.tailrec
 
 import org.apache.spark.SparkConf
+import org.apache.spark.internal.Logging
 import org.apache.spark.util.{IntParam, Utils}
 
 /**
  * Command-line parser for the master.
  */
-private[master] class MasterArguments(args: Array[String], conf: SparkConf) {
+private[master] class MasterArguments(args: Array[String], conf: SparkConf) 
extends Logging {
   var host = Utils.localHostName()
   var port = 7077
   var webUiPort = 8080
   var propertiesFile: String = null
 
   // Check for settings in environment variables
+  if (System.getenv("SPARK_MASTER_IP") != null) {
+logWarning("SPARK_MASTER_IP is deprecated, please use SPARK_MASTER_HOST")
+host = System.getenv("SPARK_MASTER_IP")
+  }
+
   if (System.getenv("SPARK_MASTER_HOST") != null) {
 host = System.getenv("SPARK_MASTER_HOST")
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/b75d1c20/docs/spark-standalone.md
--
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 40c7293..c864c90 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -94,8 +94,8 @@ You can optionally configure the cluster further by setting 
environment variable
 
   Environment VariableMeaning
   
-SPARK_MASTER_IP
-Bind the master to a specific IP address, for example a public 
one.
+SPARK_MASTER_HOST
+Bind the master to a specific hostname or IP address, for example a 
public one.
   
   
 SPARK_MASTER_PORT

http://git-wip-us.apache.org/repos/asf/spark/blob/b75d1c20/sbin/start-master.sh
--
diff --git a/sbin/start-master.sh b/sbin/start-master.sh
index ce7f177..981cb15 100755
--- a/sbin/start-master.sh
+++ b/sbin/start-master.sh
@@ -47,8 +47,8 @@ if [ 

spark git commit: [SPARK-15781][DOCUMENTATION] remove deprecated environment variable doc

2016-06-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 8cc22b008 -> 3fd3ee038


[SPARK-15781][DOCUMENTATION] remove deprecated environment variable doc

## What changes were proposed in this pull request?

Like `SPARK_JAVA_OPTS` and `SPARK_CLASSPATH`, we will remove the document for 
`SPARK_WORKER_INSTANCES` to discourage user not to use them. If they are 
actually used, SparkConf will show a warning message as before.

## How was this patch tested?

Manually tested.

Author: bomeng 

Closes #13533 from bomeng/SPARK-15781.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3fd3ee03
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3fd3ee03
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3fd3ee03

Branch: refs/heads/master
Commit: 3fd3ee038b89821f51f30a4ecd4452b5b3bc6568
Parents: 8cc22b0
Author: bomeng 
Authored: Sun Jun 12 12:58:34 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 12 12:58:34 2016 +0100

--
 docs/spark-standalone.md | 9 -
 1 file changed, 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3fd3ee03/docs/spark-standalone.md
--
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index fd94c34..40c7293 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -134,15 +134,6 @@ You can optionally configure the cluster further by 
setting environment variable
 Port for the worker web UI (default: 8081).
   
   
-SPARK_WORKER_INSTANCES
-
-  Number of worker instances to run on each machine (default: 1). You can 
make this more than 1 if
-  you have have very large machines and would like multiple Spark worker 
processes. If you do set
-  this, make sure to also set SPARK_WORKER_CORES explicitly 
to limit the cores per worker,
-  or else each worker will try to use all the cores.
-
-  
-  
 SPARK_WORKER_DIR
 Directory to run applications in, which will include both logs and 
scratch space (default: SPARK_HOME/work).
   


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15781][DOCUMENTATION] remove deprecated environment variable doc

2016-06-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 879e8fd09 -> 8c294f4ad


[SPARK-15781][DOCUMENTATION] remove deprecated environment variable doc

## What changes were proposed in this pull request?

Like `SPARK_JAVA_OPTS` and `SPARK_CLASSPATH`, we will remove the document for 
`SPARK_WORKER_INSTANCES` to discourage user not to use them. If they are 
actually used, SparkConf will show a warning message as before.

## How was this patch tested?

Manually tested.

Author: bomeng 

Closes #13533 from bomeng/SPARK-15781.

(cherry picked from commit 3fd3ee038b89821f51f30a4ecd4452b5b3bc6568)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8c294f4a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8c294f4a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8c294f4a

Branch: refs/heads/branch-2.0
Commit: 8c294f4ad95e95f6c8873d7b346394d34cc40975
Parents: 879e8fd
Author: bomeng 
Authored: Sun Jun 12 12:58:34 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 12 12:58:41 2016 +0100

--
 docs/spark-standalone.md | 9 -
 1 file changed, 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8c294f4a/docs/spark-standalone.md
--
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index fd94c34..40c7293 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -134,15 +134,6 @@ You can optionally configure the cluster further by 
setting environment variable
 Port for the worker web UI (default: 8081).
   
   
-SPARK_WORKER_INSTANCES
-
-  Number of worker instances to run on each machine (default: 1). You can 
make this more than 1 if
-  you have have very large machines and would like multiple Spark worker 
processes. If you do set
-  this, make sure to also set SPARK_WORKER_CORES explicitly 
to limit the cores per worker,
-  or else each worker will try to use all the cores.
-
-  
-  
 SPARK_WORKER_DIR
 Directory to run applications in, which will include both logs and 
scratch space (default: SPARK_HOME/work).
   


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and ReplayListenerSuite

2016-06-12 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9e204c62c -> 8cc22b008


[SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and 
ReplayListenerSuite

## What changes were proposed in this pull request?

These tests weren't properly using `LocalSparkContext` so weren't cleaning up 
correctly when tests failed.

## How was this patch tested?

Jenkins.

Author: Imran Rashid 

Closes #13602 from squito/SPARK-15878_cleanup_replaylistener.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8cc22b00
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8cc22b00
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8cc22b00

Branch: refs/heads/master
Commit: 8cc22b0085475a188f229536b4f83988ae889a8e
Parents: 9e204c6
Author: Imran Rashid 
Authored: Sun Jun 12 12:54:57 2016 +0100
Committer: Sean Owen 
Committed: Sun Jun 12 12:54:57 2016 +0100

--
 .../org/apache/spark/scheduler/EventLoggingListenerSuite.scala | 2 +-
 .../scala/org/apache/spark/scheduler/ReplayListenerSuite.scala | 6 +++---
 2 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8cc22b00/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
 
b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index 176d893..c4c80b5 100644
--- 
a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -181,7 +181,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with 
LocalSparkContext wit
 // into SPARK-6688.
 val conf = getLoggingConf(testDirPath, compressionCodec)
   .set("spark.hadoop.fs.defaultFS", "unsupported://example.com")
-val sc = new SparkContext("local-cluster[2,2,1024]", "test", conf)
+sc = new SparkContext("local-cluster[2,2,1024]", "test", conf)
 assert(sc.eventLogger.isDefined)
 val eventLogger = sc.eventLogger.get
 val eventLogPath = eventLogger.logPath

http://git-wip-us.apache.org/repos/asf/spark/blob/8cc22b00/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
index 35215c1..1732aca 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala
@@ -23,7 +23,7 @@ import java.net.URI
 import org.json4s.jackson.JsonMethods._
 import org.scalatest.BeforeAndAfter
 
-import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, 
SparkFunSuite}
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.util.{JsonProtocol, JsonProtocolSuite, Utils}
@@ -31,7 +31,7 @@ import org.apache.spark.util.{JsonProtocol, 
JsonProtocolSuite, Utils}
 /**
  * Test whether ReplayListenerBus replays events from logs correctly.
  */
-class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter {
+class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with 
LocalSparkContext {
   private val fileSystem = Utils.getHadoopFileSystem("/",
 SparkHadoopUtil.get.newConfiguration(new SparkConf()))
   private var testDir: File = _
@@ -101,7 +101,7 @@ class ReplayListenerSuite extends SparkFunSuite with 
BeforeAndAfter {
 fileSystem.mkdirs(logDirPath)
 
 val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName)
-val sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf)
+sc = new SparkContext("local-cluster[2,1,1024]", "Test replay", conf)
 
 // Run a few jobs
 sc.parallelize(1 to 100, 1).count()


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15837][ML][PYSPARK] Word2vec python add maxsentence parameter

2016-06-10 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 16ca32eac -> cdd7f5a57


[SPARK-15837][ML][PYSPARK] Word2vec python add maxsentence parameter

## What changes were proposed in this pull request?

Word2vec python add maxsentence parameter.

## How was this patch tested?

Existing test.

Author: WeichenXu 

Closes #13578 from WeichenXu123/word2vec_python_add_maxsentence.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cdd7f5a5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cdd7f5a5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cdd7f5a5

Branch: refs/heads/master
Commit: cdd7f5a57a21d4a8f93456d149f65859c96190cf
Parents: 16ca32e
Author: WeichenXu 
Authored: Fri Jun 10 12:26:53 2016 +0100
Committer: Sean Owen 
Committed: Fri Jun 10 12:26:53 2016 +0100

--
 python/pyspark/ml/feature.py | 29 -
 1 file changed, 24 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cdd7f5a5/python/pyspark/ml/feature.py
--
diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py
index ebe1300..bfb2fb7 100755
--- a/python/pyspark/ml/feature.py
+++ b/python/pyspark/ml/feature.py
@@ -2244,28 +2244,33 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, 
HasSeed, HasInputCol, Has
 windowSize = Param(Params._dummy(), "windowSize",
"the window size (context words from [-window, 
window]). Default value is 5",
typeConverter=TypeConverters.toInt)
+maxSentenceLength = Param(Params._dummy(), "maxSentenceLength",
+  "Maximum length (in words) of each sentence in 
the input data. " +
+  "Any sentence longer than this threshold will " +
+  "be divided into chunks up to the size.",
+  typeConverter=TypeConverters.toInt)
 
 @keyword_only
 def __init__(self, vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1,
- seed=None, inputCol=None, outputCol=None, windowSize=5):
+ seed=None, inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000):
 """
 __init__(self, vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1, \
- seed=None, inputCol=None, outputCol=None, windowSize=5)
+ seed=None, inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000)
 """
 super(Word2Vec, self).__init__()
 self._java_obj = 
self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid)
 self._setDefault(vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1,
- seed=None, windowSize=5)
+ seed=None, windowSize=5, maxSentenceLength=1000)
 kwargs = self.__init__._input_kwargs
 self.setParams(**kwargs)
 
 @keyword_only
 @since("1.4.0")
 def setParams(self, vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1,
-  seed=None, inputCol=None, outputCol=None, windowSize=5):
+  seed=None, inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000):
 """
 setParams(self, minCount=5, numPartitions=1, stepSize=0.025, 
maxIter=1, seed=None, \
- inputCol=None, outputCol=None, windowSize=5)
+ inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000)
 Sets params for this Word2Vec.
 """
 kwargs = self.setParams._input_kwargs
@@ -2327,6 +2332,20 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, 
HasSeed, HasInputCol, Has
 """
 return self.getOrDefault(self.windowSize)
 
+@since("2.0.0")
+def setMaxSentenceLength(self, value):
+"""
+Sets the value of :py:attr:`maxSentenceLength`.
+"""
+return self._set(maxSentenceLength=value)
+
+@since("2.0.0")
+def getMaxSentenceLength(self):
+"""
+Gets the value of maxSentenceLength or its default value.
+"""
+return self.getOrDefault(self.maxSentenceLength)
+
 def _create_model(self, java_model):
 return Word2VecModel(java_model)
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15837][ML][PYSPARK] Word2vec python add maxsentence parameter

2016-06-10 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 6709ce1ae -> 54b4763d2


[SPARK-15837][ML][PYSPARK] Word2vec python add maxsentence parameter

## What changes were proposed in this pull request?

Word2vec python add maxsentence parameter.

## How was this patch tested?

Existing test.

Author: WeichenXu 

Closes #13578 from WeichenXu123/word2vec_python_add_maxsentence.

(cherry picked from commit cdd7f5a57a21d4a8f93456d149f65859c96190cf)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/54b4763d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/54b4763d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/54b4763d

Branch: refs/heads/branch-2.0
Commit: 54b4763d295d6aeab6105d0430470343dd4ca3a3
Parents: 6709ce1
Author: WeichenXu 
Authored: Fri Jun 10 12:26:53 2016 +0100
Committer: Sean Owen 
Committed: Fri Jun 10 12:27:04 2016 +0100

--
 python/pyspark/ml/feature.py | 29 -
 1 file changed, 24 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/54b4763d/python/pyspark/ml/feature.py
--
diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py
index ebe1300..bfb2fb7 100755
--- a/python/pyspark/ml/feature.py
+++ b/python/pyspark/ml/feature.py
@@ -2244,28 +2244,33 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, 
HasSeed, HasInputCol, Has
 windowSize = Param(Params._dummy(), "windowSize",
"the window size (context words from [-window, 
window]). Default value is 5",
typeConverter=TypeConverters.toInt)
+maxSentenceLength = Param(Params._dummy(), "maxSentenceLength",
+  "Maximum length (in words) of each sentence in 
the input data. " +
+  "Any sentence longer than this threshold will " +
+  "be divided into chunks up to the size.",
+  typeConverter=TypeConverters.toInt)
 
 @keyword_only
 def __init__(self, vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1,
- seed=None, inputCol=None, outputCol=None, windowSize=5):
+ seed=None, inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000):
 """
 __init__(self, vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1, \
- seed=None, inputCol=None, outputCol=None, windowSize=5)
+ seed=None, inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000)
 """
 super(Word2Vec, self).__init__()
 self._java_obj = 
self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid)
 self._setDefault(vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1,
- seed=None, windowSize=5)
+ seed=None, windowSize=5, maxSentenceLength=1000)
 kwargs = self.__init__._input_kwargs
 self.setParams(**kwargs)
 
 @keyword_only
 @since("1.4.0")
 def setParams(self, vectorSize=100, minCount=5, numPartitions=1, 
stepSize=0.025, maxIter=1,
-  seed=None, inputCol=None, outputCol=None, windowSize=5):
+  seed=None, inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000):
 """
 setParams(self, minCount=5, numPartitions=1, stepSize=0.025, 
maxIter=1, seed=None, \
- inputCol=None, outputCol=None, windowSize=5)
+ inputCol=None, outputCol=None, windowSize=5, 
maxSentenceLength=1000)
 Sets params for this Word2Vec.
 """
 kwargs = self.setParams._input_kwargs
@@ -2327,6 +2332,20 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, 
HasSeed, HasInputCol, Has
 """
 return self.getOrDefault(self.windowSize)
 
+@since("2.0.0")
+def setMaxSentenceLength(self, value):
+"""
+Sets the value of :py:attr:`maxSentenceLength`.
+"""
+return self._set(maxSentenceLength=value)
+
+@since("2.0.0")
+def getMaxSentenceLength(self):
+"""
+Gets the value of maxSentenceLength or its default value.
+"""
+return self.getOrDefault(self.maxSentenceLength)
+
 def _create_model(self, java_model):
 return Word2VecModel(java_model)
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15823][PYSPARK][ML] Add @property for 'accuracy' in MulticlassMetrics

2016-06-10 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 84a8421e5 -> 6709ce1ae


[SPARK-15823][PYSPARK][ML] Add @property for 'accuracy' in MulticlassMetrics

## What changes were proposed in this pull request?
`accuracy` should be decorated with `property` to keep step with other methods 
in `pyspark.MulticlassMetrics`, like `weightedPrecision`, `weightedRecall`, etc

## How was this patch tested?
manual tests

Author: Zheng RuiFeng 

Closes #13560 from zhengruifeng/add_accuracy_property.

(cherry picked from commit 16ca32eace39c423224b0ec25922038fd45c501a)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6709ce1a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6709ce1a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6709ce1a

Branch: refs/heads/branch-2.0
Commit: 6709ce1aea4a8d7438722f48fd7f2ed0fc7fa5be
Parents: 84a8421
Author: Zheng RuiFeng 
Authored: Fri Jun 10 10:09:19 2016 +0100
Committer: Sean Owen 
Committed: Fri Jun 10 10:09:29 2016 +0100

--
 python/pyspark/mllib/evaluation.py | 7 ++-
 1 file changed, 2 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6709ce1a/python/pyspark/mllib/evaluation.py
--
diff --git a/python/pyspark/mllib/evaluation.py 
b/python/pyspark/mllib/evaluation.py
index 2eaac87..fc2a0b3 100644
--- a/python/pyspark/mllib/evaluation.py
+++ b/python/pyspark/mllib/evaluation.py
@@ -179,11 +179,7 @@ class MulticlassMetrics(JavaModelWrapper):
 1.0...
 >>> metrics.fMeasure(0.0, 2.0)
 0.52...
->>> metrics.precision()
-0.66...
->>> metrics.recall()
-0.66...
->>> metrics.accuracy()
+>>> metrics.accuracy
 0.66...
 >>> metrics.weightedFalsePositiveRate
 0.19...
@@ -273,6 +269,7 @@ class MulticlassMetrics(JavaModelWrapper):
 else:
 return self.call("fMeasure", label, beta)
 
+@property
 @since('2.0.0')
 def accuracy(self):
 """


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15823][PYSPARK][ML] Add @property for 'accuracy' in MulticlassMetrics

2016-06-10 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 675a73715 -> 16ca32eac


[SPARK-15823][PYSPARK][ML] Add @property for 'accuracy' in MulticlassMetrics

## What changes were proposed in this pull request?
`accuracy` should be decorated with `property` to keep step with other methods 
in `pyspark.MulticlassMetrics`, like `weightedPrecision`, `weightedRecall`, etc

## How was this patch tested?
manual tests

Author: Zheng RuiFeng 

Closes #13560 from zhengruifeng/add_accuracy_property.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/16ca32ea
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/16ca32ea
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/16ca32ea

Branch: refs/heads/master
Commit: 16ca32eace39c423224b0ec25922038fd45c501a
Parents: 675a737
Author: Zheng RuiFeng 
Authored: Fri Jun 10 10:09:19 2016 +0100
Committer: Sean Owen 
Committed: Fri Jun 10 10:09:19 2016 +0100

--
 python/pyspark/mllib/evaluation.py | 7 ++-
 1 file changed, 2 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/16ca32ea/python/pyspark/mllib/evaluation.py
--
diff --git a/python/pyspark/mllib/evaluation.py 
b/python/pyspark/mllib/evaluation.py
index 2eaac87..fc2a0b3 100644
--- a/python/pyspark/mllib/evaluation.py
+++ b/python/pyspark/mllib/evaluation.py
@@ -179,11 +179,7 @@ class MulticlassMetrics(JavaModelWrapper):
 1.0...
 >>> metrics.fMeasure(0.0, 2.0)
 0.52...
->>> metrics.precision()
-0.66...
->>> metrics.recall()
-0.66...
->>> metrics.accuracy()
+>>> metrics.accuracy
 0.66...
 >>> metrics.weightedFalsePositiveRate
 0.19...
@@ -273,6 +269,7 @@ class MulticlassMetrics(JavaModelWrapper):
 else:
 return self.call("fMeasure", label, beta)
 
+@property
 @since('2.0.0')
 def accuracy(self):
 """


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: Log warnings for numIterations * miniBatchFraction < 1.0

2016-05-25 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 9c297df3d -> 589cce93c


Log warnings for numIterations * miniBatchFraction < 1.0

## What changes were proposed in this pull request?

Add a warning log for the case that `numIterations * miniBatchFraction <1.0` 
during gradient descent. If the product of those two numbers is less than 
`1.0`, then not all training examples will be used during optimization. To put 
this concretely, suppose that `numExamples = 100`, `miniBatchFraction = 0.2` 
and `numIterations = 3`. Then, 3 iterations will occur each sampling 
approximately 6 examples each. In the best case, each of the 6 examples are 
unique; hence 18/100 examples are used.

This may be counter-intuitive to most users and led to the issue during the 
development of another Spark  ML model: 
https://github.com/zhengruifeng/spark-libFM/issues/11. If a user actually does 
not require the training data set, it would be easier and more intuitive to use 
`RDD.sample`.

## How was this patch tested?

`build/mvn -DskipTests clean package` build succeeds

Author: Gio Borje 

Closes #13265 from Hydrotoast/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/589cce93
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/589cce93
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/589cce93

Branch: refs/heads/master
Commit: 589cce93c821ac28e9090a478f6e7465398b7c30
Parents: 9c297df
Author: Gio Borje 
Authored: Wed May 25 16:52:31 2016 -0500
Committer: Sean Owen 
Committed: Wed May 25 16:52:31 2016 -0500

--
 .../org/apache/spark/mllib/optimization/GradientDescent.scala   | 5 +
 1 file changed, 5 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/589cce93/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
 
b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
index a67ea83..735e780 100644
--- 
a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
@@ -197,6 +197,11 @@ object GradientDescent extends Logging {
 "< 1.0 can be unstable because of the stochasticity in sampling.")
 }
 
+if (numIterations * miniBatchFraction < 1.0) {
+  logWarning("Not all examples will be used if numIterations * 
miniBatchFraction < 1.0: " +
+s"numIterations=$numIterations and 
miniBatchFraction=$miniBatchFraction")
+}
+
 val stochasticLossHistory = new ArrayBuffer[Double](numIterations)
 // Record previous weight and current one to calculate solution vector 
difference
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: Log warnings for numIterations * miniBatchFraction < 1.0

2016-05-25 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 f7158c482 -> 0064a4dcb


Log warnings for numIterations * miniBatchFraction < 1.0

## What changes were proposed in this pull request?

Add a warning log for the case that `numIterations * miniBatchFraction <1.0` 
during gradient descent. If the product of those two numbers is less than 
`1.0`, then not all training examples will be used during optimization. To put 
this concretely, suppose that `numExamples = 100`, `miniBatchFraction = 0.2` 
and `numIterations = 3`. Then, 3 iterations will occur each sampling 
approximately 6 examples each. In the best case, each of the 6 examples are 
unique; hence 18/100 examples are used.

This may be counter-intuitive to most users and led to the issue during the 
development of another Spark  ML model: 
https://github.com/zhengruifeng/spark-libFM/issues/11. If a user actually does 
not require the training data set, it would be easier and more intuitive to use 
`RDD.sample`.

## How was this patch tested?

`build/mvn -DskipTests clean package` build succeeds

Author: Gio Borje 

Closes #13265 from Hydrotoast/master.

(cherry picked from commit 589cce93c821ac28e9090a478f6e7465398b7c30)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0064a4dc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0064a4dc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0064a4dc

Branch: refs/heads/branch-2.0
Commit: 0064a4dcbed1d91732a29c2cede464b8d148aeca
Parents: f7158c4
Author: Gio Borje 
Authored: Wed May 25 16:52:31 2016 -0500
Committer: Sean Owen 
Committed: Wed May 25 16:52:48 2016 -0500

--
 .../org/apache/spark/mllib/optimization/GradientDescent.scala   | 5 +
 1 file changed, 5 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/0064a4dc/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
 
b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
index a67ea83..735e780 100644
--- 
a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
@@ -197,6 +197,11 @@ object GradientDescent extends Logging {
 "< 1.0 can be unstable because of the stochasticity in sampling.")
 }
 
+if (numIterations * miniBatchFraction < 1.0) {
+  logWarning("Not all examples will be used if numIterations * 
miniBatchFraction < 1.0: " +
+s"numIterations=$numIterations and 
miniBatchFraction=$miniBatchFraction")
+}
+
 val stochasticLossHistory = new ArrayBuffer[Double](numIterations)
 // Record previous weight and current one to calculate solution vector 
difference
 


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15610][ML] update error message for k in pca

2016-05-27 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 88c9c467a -> 9893dc975


[SPARK-15610][ML] update error message for k in pca

## What changes were proposed in this pull request?
Fix the wrong bound of `k` in `PCA`
`require(k <= sources.first().size, ...`  ->  `require(k < sources.first().size`

BTW, remove unused import in `ml.ElementwiseProduct`

## How was this patch tested?

manual tests

Author: Zheng RuiFeng 

Closes #13356 from zhengruifeng/fix_pca.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9893dc97
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9893dc97
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9893dc97

Branch: refs/heads/master
Commit: 9893dc975784551a62f65bbd709f8972e0204b2a
Parents: 88c9c46
Author: Zheng RuiFeng 
Authored: Fri May 27 21:57:41 2016 -0500
Committer: Sean Owen 
Committed: Fri May 27 21:57:41 2016 -0500

--
 .../scala/org/apache/spark/ml/feature/ElementwiseProduct.scala | 1 -
 mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala  | 6 +++---
 2 files changed, 3 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/9893dc97/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala 
b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
index 91989c3..9d2e60f 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
@@ -23,7 +23,6 @@ import org.apache.spark.ml.linalg.{Vector, VectorUDT}
 import org.apache.spark.ml.param.Param
 import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, 
Identifiable}
 import org.apache.spark.mllib.feature
-import org.apache.spark.mllib.linalg.{Vectors => OldVectors}
 import org.apache.spark.mllib.linalg.VectorImplicits._
 import org.apache.spark.sql.types.DataType
 

http://git-wip-us.apache.org/repos/asf/spark/blob/9893dc97/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
index 30c403e..15b7220 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
@@ -40,8 +40,9 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) {
*/
   @Since("1.4.0")
   def fit(sources: RDD[Vector]): PCAModel = {
-require(k <= sources.first().size,
-  s"source vector size is ${sources.first().size} must be greater than 
k=$k")
+val numFeatures = sources.first().size
+require(k <= numFeatures,
+  s"source vector size $numFeatures must be no less than k=$k")
 
 val mat = new RowMatrix(sources)
 val (pc, explainedVariance) = 
mat.computePrincipalComponentsAndExplainedVariance(k)
@@ -58,7 +59,6 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) {
   case m =>
 throw new IllegalArgumentException("Unsupported matrix format. 
Expected " +
   s"SparseMatrix or DenseMatrix. Instead got: ${m.getClass}")
-
 }
 val denseExplainedVariance = explainedVariance match {
   case dv: DenseVector =>


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15610][ML] update error message for k in pca

2016-05-27 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 6d82e0c1b -> 3801fb4f3


[SPARK-15610][ML] update error message for k in pca

## What changes were proposed in this pull request?
Fix the wrong bound of `k` in `PCA`
`require(k <= sources.first().size, ...`  ->  `require(k < sources.first().size`

BTW, remove unused import in `ml.ElementwiseProduct`

## How was this patch tested?

manual tests

Author: Zheng RuiFeng 

Closes #13356 from zhengruifeng/fix_pca.

(cherry picked from commit 9893dc975784551a62f65bbd709f8972e0204b2a)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3801fb4f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3801fb4f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3801fb4f

Branch: refs/heads/branch-2.0
Commit: 3801fb4f35ba1ffb8dbaf8326eff927b738551f2
Parents: 6d82e0c
Author: Zheng RuiFeng 
Authored: Fri May 27 21:57:41 2016 -0500
Committer: Sean Owen 
Committed: Fri May 27 21:57:48 2016 -0500

--
 .../scala/org/apache/spark/ml/feature/ElementwiseProduct.scala | 1 -
 mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala  | 6 +++---
 2 files changed, 3 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3801fb4f/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala 
b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
index 91989c3..9d2e60f 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala
@@ -23,7 +23,6 @@ import org.apache.spark.ml.linalg.{Vector, VectorUDT}
 import org.apache.spark.ml.param.Param
 import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, 
Identifiable}
 import org.apache.spark.mllib.feature
-import org.apache.spark.mllib.linalg.{Vectors => OldVectors}
 import org.apache.spark.mllib.linalg.VectorImplicits._
 import org.apache.spark.sql.types.DataType
 

http://git-wip-us.apache.org/repos/asf/spark/blob/3801fb4f/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
--
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala 
b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
index 30c403e..15b7220 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala
@@ -40,8 +40,9 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) {
*/
   @Since("1.4.0")
   def fit(sources: RDD[Vector]): PCAModel = {
-require(k <= sources.first().size,
-  s"source vector size is ${sources.first().size} must be greater than 
k=$k")
+val numFeatures = sources.first().size
+require(k <= numFeatures,
+  s"source vector size $numFeatures must be no less than k=$k")
 
 val mat = new RowMatrix(sources)
 val (pc, explainedVariance) = 
mat.computePrincipalComponentsAndExplainedVariance(k)
@@ -58,7 +59,6 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) {
   case m =>
 throw new IllegalArgumentException("Unsupported matrix format. 
Expected " +
   s"SparseMatrix or DenseMatrix. Instead got: ${m.getClass}")
-
 }
 val denseExplainedVariance = explainedVariance match {
   case dv: DenseVector =>


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15449][MLLIB][EXAMPLE] Wrong Data Format - Documentation Issue

2016-05-27 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 4a2fb8b87 -> 5d4dafe8f


[SPARK-15449][MLLIB][EXAMPLE] Wrong Data Format - Documentation Issue

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)
In the MLLib naivebayes example, scala and python example doesn't use libsvm 
data, but Java does.

I make changes in scala and python example to use the libsvm data as the same 
as Java example.

## How was this patch tested?

Manual tests

Author: wm...@hotmail.com 

Closes #13301 from wangmiao1981/example.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5d4dafe8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5d4dafe8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5d4dafe8

Branch: refs/heads/master
Commit: 5d4dafe8fdea49dcbd6b0e4c23e3791fa30c8911
Parents: 4a2fb8b
Author: wm...@hotmail.com 
Authored: Fri May 27 20:59:24 2016 -0500
Committer: Sean Owen 
Committed: Fri May 27 20:59:24 2016 -0500

--
 data/mllib/sample_naive_bayes_data.txt| 12 
 .../spark/examples/mllib/JavaNaiveBayesExample.java   |  4 ++--
 examples/src/main/python/mllib/naive_bayes_example.py | 13 -
 .../spark/examples/mllib/NaiveBayesExample.scala  | 14 --
 4 files changed, 10 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/5d4dafe8/data/mllib/sample_naive_bayes_data.txt
--
diff --git a/data/mllib/sample_naive_bayes_data.txt 
b/data/mllib/sample_naive_bayes_data.txt
deleted file mode 100644
index bd22bea..000
--- a/data/mllib/sample_naive_bayes_data.txt
+++ /dev/null
@@ -1,12 +0,0 @@
-0,1 0 0
-0,2 0 0
-0,3 0 0
-0,4 0 0
-1,0 1 0
-1,0 2 0
-1,0 3 0
-1,0 4 0
-2,0 0 1
-2,0 0 2
-2,0 0 3
-2,0 0 4
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/5d4dafe8/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
 
b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
index 2b17dbb..f4ec04b 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
@@ -36,9 +36,9 @@ public class JavaNaiveBayesExample {
 SparkConf sparkConf = new SparkConf().setAppName("JavaNaiveBayesExample");
 JavaSparkContext jsc = new JavaSparkContext(sparkConf);
 // $example on$
-String path = "data/mllib/sample_naive_bayes_data.txt";
+String path = "data/mllib/sample_libsvm_data.txt";
 JavaRDD inputData = MLUtils.loadLibSVMFile(jsc.sc(), 
path).toJavaRDD();
-JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 
0.4}, 12345);
+JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 
0.4});
 JavaRDD training = tmp[0]; // training set
 JavaRDD test = tmp[1]; // test set
 final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0);

http://git-wip-us.apache.org/repos/asf/spark/blob/5d4dafe8/examples/src/main/python/mllib/naive_bayes_example.py
--
diff --git a/examples/src/main/python/mllib/naive_bayes_example.py 
b/examples/src/main/python/mllib/naive_bayes_example.py
index 35724f7..749353b 100644
--- a/examples/src/main/python/mllib/naive_bayes_example.py
+++ b/examples/src/main/python/mllib/naive_bayes_example.py
@@ -29,15 +29,9 @@ import shutil
 from pyspark import SparkContext
 # $example on$
 from pyspark.mllib.classification import NaiveBayes, NaiveBayesModel
-from pyspark.mllib.linalg import Vectors
-from pyspark.mllib.regression import LabeledPoint
+from pyspark.mllib.util import MLUtils
 
 
-def parseLine(line):
-parts = line.split(',')
-label = float(parts[0])
-features = Vectors.dense([float(x) for x in parts[1].split(' ')])
-return LabeledPoint(label, features)
 # $example off$
 
 if __name__ == "__main__":
@@ -45,10 +39,11 @@ if __name__ == "__main__":
 sc = SparkContext(appName="PythonNaiveBayesExample")
 
 # $example on$
-data = sc.textFile('data/mllib/sample_naive_bayes_data.txt').map(parseLine)
+# Load and parse the data file.
+data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt")
 
 # Split data approximately into training (60%) and test (40%)
-training, test = data.randomSplit([0.6, 0.4], seed=0)
+training, test = data.randomSplit([0.6, 0.4])
 
 # Train a naive Bayes model.
 model = NaiveBayes.train(training, 

spark git commit: [SPARK-15562][ML] Delete temp directory after program exit in DataFrameExample

2016-05-27 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 8467e2102 -> 6d82e0c1b


[SPARK-15562][ML] Delete temp directory after program exit in DataFrameExample

## What changes were proposed in this pull request?
Temp directory used to save records is not deleted after program exit in 
DataFrameExample. Although it called deleteOnExit, it doesn't work as the 
directory is not empty. Similar things happend in ContextCleanerSuite. Update 
the code to make sure temp directory is deleted after program exit.

## How was this patch tested?

unit tests and local build.

Author: dding3 

Closes #13328 from dding3/master.

(cherry picked from commit 88c9c467a31630c558719679ca0894873a268b27)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/6d82e0c1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/6d82e0c1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/6d82e0c1

Branch: refs/heads/branch-2.0
Commit: 6d82e0c1b8b4368e91aeebfc80430a61762c7e88
Parents: 8467e21
Author: dding3 
Authored: Fri May 27 21:01:50 2016 -0500
Committer: Sean Owen 
Committed: Fri May 27 21:01:56 2016 -0500

--
 core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala   | 4 ++--
 .../scala/org/apache/spark/examples/ml/DataFrameExample.scala| 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/6d82e0c1/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala 
b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
index 69ff6c7..6724af9 100644
--- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
@@ -32,6 +32,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData}
 import org.apache.spark.shuffle.sort.SortShuffleManager
 import org.apache.spark.storage._
+import org.apache.spark.util.Utils
 
 /**
  * An abstract base class for context cleaner tests, which sets up a context 
with a config
@@ -206,8 +207,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase {
   }
 
   test("automatically cleanup normal checkpoint") {
-val checkpointDir = java.io.File.createTempFile("temp", "")
-checkpointDir.deleteOnExit()
+val checkpointDir = Utils.createTempDir()
 checkpointDir.delete()
 var rdd = newPairRDD()
 sc.setCheckpointDir(checkpointDir.toString)

http://git-wip-us.apache.org/repos/asf/spark/blob/6d82e0c1/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
--
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala 
b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
index c69027b..11faa61 100644
--- 
a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
+++ 
b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
@@ -28,6 +28,7 @@ import org.apache.spark.ml.linalg.Vector
 import org.apache.spark.mllib.linalg.Vectors
 import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer
 import org.apache.spark.sql.{DataFrame, Row, SparkSession}
+import org.apache.spark.util.Utils
 
 /**
  * An example of how to use [[org.apache.spark.sql.DataFrame]] for ML. Run with
@@ -86,8 +87,7 @@ object DataFrameExample {
 println(s"Selected features column with average values:\n 
${featureSummary.mean.toString}")
 
 // Save the records in a parquet file.
-val tmpDir = Files.createTempDir()
-tmpDir.deleteOnExit()
+val tmpDir = Utils.createTempDir()
 val outputDir = new File(tmpDir, "dataframe").toString
 println(s"Saving to $outputDir as Parquet file.")
 df.write.parquet(outputDir)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15562][ML] Delete temp directory after program exit in DataFrameExample

2016-05-27 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 5d4dafe8f -> 88c9c467a


[SPARK-15562][ML] Delete temp directory after program exit in DataFrameExample

## What changes were proposed in this pull request?
Temp directory used to save records is not deleted after program exit in 
DataFrameExample. Although it called deleteOnExit, it doesn't work as the 
directory is not empty. Similar things happend in ContextCleanerSuite. Update 
the code to make sure temp directory is deleted after program exit.

## How was this patch tested?

unit tests and local build.

Author: dding3 

Closes #13328 from dding3/master.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/88c9c467
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/88c9c467
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/88c9c467

Branch: refs/heads/master
Commit: 88c9c467a31630c558719679ca0894873a268b27
Parents: 5d4dafe
Author: dding3 
Authored: Fri May 27 21:01:50 2016 -0500
Committer: Sean Owen 
Committed: Fri May 27 21:01:50 2016 -0500

--
 core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala   | 4 ++--
 .../scala/org/apache/spark/examples/ml/DataFrameExample.scala| 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/88c9c467/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
--
diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala 
b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
index 69ff6c7..6724af9 100644
--- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala
@@ -32,6 +32,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData}
 import org.apache.spark.shuffle.sort.SortShuffleManager
 import org.apache.spark.storage._
+import org.apache.spark.util.Utils
 
 /**
  * An abstract base class for context cleaner tests, which sets up a context 
with a config
@@ -206,8 +207,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase {
   }
 
   test("automatically cleanup normal checkpoint") {
-val checkpointDir = java.io.File.createTempFile("temp", "")
-checkpointDir.deleteOnExit()
+val checkpointDir = Utils.createTempDir()
 checkpointDir.delete()
 var rdd = newPairRDD()
 sc.setCheckpointDir(checkpointDir.toString)

http://git-wip-us.apache.org/repos/asf/spark/blob/88c9c467/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
--
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala 
b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
index c69027b..11faa61 100644
--- 
a/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
+++ 
b/examples/src/main/scala/org/apache/spark/examples/ml/DataFrameExample.scala
@@ -28,6 +28,7 @@ import org.apache.spark.ml.linalg.Vector
 import org.apache.spark.mllib.linalg.Vectors
 import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer
 import org.apache.spark.sql.{DataFrame, Row, SparkSession}
+import org.apache.spark.util.Utils
 
 /**
  * An example of how to use [[org.apache.spark.sql.DataFrame]] for ML. Run with
@@ -86,8 +87,7 @@ object DataFrameExample {
 println(s"Selected features column with average values:\n 
${featureSummary.mean.toString}")
 
 // Save the records in a parquet file.
-val tmpDir = Files.createTempDir()
-tmpDir.deleteOnExit()
+val tmpDir = Utils.createTempDir()
 val outputDir = new File(tmpDir, "dataframe").toString
 println(s"Saving to $outputDir as Parquet file.")
 df.write.parquet(outputDir)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15449][MLLIB][EXAMPLE] Wrong Data Format - Documentation Issue

2016-05-27 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 80a40e8e2 -> 8467e2102


[SPARK-15449][MLLIB][EXAMPLE] Wrong Data Format - Documentation Issue

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)
In the MLLib naivebayes example, scala and python example doesn't use libsvm 
data, but Java does.

I make changes in scala and python example to use the libsvm data as the same 
as Java example.

## How was this patch tested?

Manual tests

Author: wm...@hotmail.com 

Closes #13301 from wangmiao1981/example.

(cherry picked from commit 5d4dafe8fdea49dcbd6b0e4c23e3791fa30c8911)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8467e210
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8467e210
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8467e210

Branch: refs/heads/branch-2.0
Commit: 8467e2102886da1cefb43f2aaa69864375fe91bc
Parents: 80a40e8
Author: wm...@hotmail.com 
Authored: Fri May 27 20:59:24 2016 -0500
Committer: Sean Owen 
Committed: Fri May 27 20:59:34 2016 -0500

--
 data/mllib/sample_naive_bayes_data.txt| 12 
 .../spark/examples/mllib/JavaNaiveBayesExample.java   |  4 ++--
 examples/src/main/python/mllib/naive_bayes_example.py | 13 -
 .../spark/examples/mllib/NaiveBayesExample.scala  | 14 --
 4 files changed, 10 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8467e210/data/mllib/sample_naive_bayes_data.txt
--
diff --git a/data/mllib/sample_naive_bayes_data.txt 
b/data/mllib/sample_naive_bayes_data.txt
deleted file mode 100644
index bd22bea..000
--- a/data/mllib/sample_naive_bayes_data.txt
+++ /dev/null
@@ -1,12 +0,0 @@
-0,1 0 0
-0,2 0 0
-0,3 0 0
-0,4 0 0
-1,0 1 0
-1,0 2 0
-1,0 3 0
-1,0 4 0
-2,0 0 1
-2,0 0 2
-2,0 0 3
-2,0 0 4
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/spark/blob/8467e210/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
--
diff --git 
a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
 
b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
index 2b17dbb..f4ec04b 100644
--- 
a/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
+++ 
b/examples/src/main/java/org/apache/spark/examples/mllib/JavaNaiveBayesExample.java
@@ -36,9 +36,9 @@ public class JavaNaiveBayesExample {
 SparkConf sparkConf = new SparkConf().setAppName("JavaNaiveBayesExample");
 JavaSparkContext jsc = new JavaSparkContext(sparkConf);
 // $example on$
-String path = "data/mllib/sample_naive_bayes_data.txt";
+String path = "data/mllib/sample_libsvm_data.txt";
 JavaRDD inputData = MLUtils.loadLibSVMFile(jsc.sc(), 
path).toJavaRDD();
-JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 
0.4}, 12345);
+JavaRDD[] tmp = inputData.randomSplit(new double[]{0.6, 
0.4});
 JavaRDD training = tmp[0]; // training set
 JavaRDD test = tmp[1]; // test set
 final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0);

http://git-wip-us.apache.org/repos/asf/spark/blob/8467e210/examples/src/main/python/mllib/naive_bayes_example.py
--
diff --git a/examples/src/main/python/mllib/naive_bayes_example.py 
b/examples/src/main/python/mllib/naive_bayes_example.py
index 35724f7..749353b 100644
--- a/examples/src/main/python/mllib/naive_bayes_example.py
+++ b/examples/src/main/python/mllib/naive_bayes_example.py
@@ -29,15 +29,9 @@ import shutil
 from pyspark import SparkContext
 # $example on$
 from pyspark.mllib.classification import NaiveBayes, NaiveBayesModel
-from pyspark.mllib.linalg import Vectors
-from pyspark.mllib.regression import LabeledPoint
+from pyspark.mllib.util import MLUtils
 
 
-def parseLine(line):
-parts = line.split(',')
-label = float(parts[0])
-features = Vectors.dense([float(x) for x in parts[1].split(' ')])
-return LabeledPoint(label, features)
 # $example off$
 
 if __name__ == "__main__":
@@ -45,10 +39,11 @@ if __name__ == "__main__":
 sc = SparkContext(appName="PythonNaiveBayesExample")
 
 # $example on$
-data = sc.textFile('data/mllib/sample_naive_bayes_data.txt').map(parseLine)
+# Load and parse the data file.
+data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt")
 
 # Split data approximately into training (60%) and test (40%)
-training, test = data.randomSplit([0.6, 0.4], seed=0)
+  

spark git commit: [SPARK-15813] Improve Canceling log message to make it less ambiguous

2016-06-13 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 b96e7f6aa -> 41f309bfb


[SPARK-15813] Improve Canceling log message to make it less ambiguous

## What changes were proposed in this pull request?
Add new desired executor number to make the log message less ambiguous.

## How was this patch tested?
This is a trivial change

Author: Peter Ableda 

Closes #13552 from peterableda/patch-1.

(cherry picked from commit d681742b2d37bd68cf5d8d3161e0f48846f6f9d4)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/41f309bf
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/41f309bf
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/41f309bf

Branch: refs/heads/branch-2.0
Commit: 41f309bfbcefcc9612efb7c0571a4009147e5896
Parents: b96e7f6
Author: Peter Ableda 
Authored: Mon Jun 13 09:40:17 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 13 09:40:25 2016 +0100

--
 .../main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/41f309bf/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
--
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index b110d82..1b80071 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -354,7 +354,8 @@ private[yarn] class YarnAllocator(
 
 } else if (missing < 0) {
   val numToCancel = math.min(numPendingAllocate, -missing)
-  logInfo(s"Canceling requests for $numToCancel executor containers")
+  logInfo(s"Canceling requests for $numToCancel executor container(s) to 
have a new desired " +
+s"total $targetNumExecutors executors.")
 
   val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, 
ANY_HOST, resource)
   if (!matchingRequests.isEmpty) {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15813] Improve Canceling log message to make it less ambiguous

2016-06-13 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master e2ab79d5e -> d681742b2


[SPARK-15813] Improve Canceling log message to make it less ambiguous

## What changes were proposed in this pull request?
Add new desired executor number to make the log message less ambiguous.

## How was this patch tested?
This is a trivial change

Author: Peter Ableda 

Closes #13552 from peterableda/patch-1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d681742b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d681742b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d681742b

Branch: refs/heads/master
Commit: d681742b2d37bd68cf5d8d3161e0f48846f6f9d4
Parents: e2ab79d
Author: Peter Ableda 
Authored: Mon Jun 13 09:40:17 2016 +0100
Committer: Sean Owen 
Committed: Mon Jun 13 09:40:17 2016 +0100

--
 .../main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/d681742b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
--
diff --git 
a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala 
b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index b110d82..1b80071 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -354,7 +354,8 @@ private[yarn] class YarnAllocator(
 
 } else if (missing < 0) {
   val numToCancel = math.min(numPendingAllocate, -missing)
-  logInfo(s"Canceling requests for $numToCancel executor containers")
+  logInfo(s"Canceling requests for $numToCancel executor container(s) to 
have a new desired " +
+s"total $targetNumExecutors executors.")
 
   val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, 
ANY_HOST, resource)
   if (!matchingRequests.isEmpty) {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15883][MLLIB][DOCS] Fix broken links in mllib documents

2016-06-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 3761330dd -> ad102af16


[SPARK-15883][MLLIB][DOCS] Fix broken links in mllib documents

## What changes were proposed in this pull request?

This issue fixes all broken links on Spark 2.0 preview MLLib documents. Also, 
this contains some editorial change.

**Fix broken links**
  * mllib-data-types.md
  * mllib-decision-tree.md
  * mllib-ensembles.md
  * mllib-feature-extraction.md
  * mllib-pmml-model-export.md
  * mllib-statistics.md

**Fix malformed section header and scala coding style**
  * mllib-linear-methods.md

**Replace indirect forward links with direct one**
  * ml-classification-regression.md

## How was this patch tested?

Manual tests (with `cd docs; jekyll build`.)

Author: Dongjoon Hyun 

Closes #13608 from dongjoon-hyun/SPARK-15883.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ad102af1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ad102af1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ad102af1

Branch: refs/heads/master
Commit: ad102af169c7344b30d3b84aa16452fcdc22542c
Parents: 3761330
Author: Dongjoon Hyun 
Authored: Sat Jun 11 12:55:38 2016 +0100
Committer: Sean Owen 
Committed: Sat Jun 11 12:55:38 2016 +0100

--
 docs/ml-classification-regression.md |  4 ++--
 docs/mllib-data-types.md | 16 ++--
 docs/mllib-decision-tree.md  |  6 +++---
 docs/mllib-ensembles.md  |  6 +++---
 docs/mllib-feature-extraction.md |  2 +-
 docs/mllib-linear-methods.md | 10 +-
 docs/mllib-pmml-model-export.md  |  2 +-
 docs/mllib-statistics.md |  8 
 8 files changed, 25 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/ad102af1/docs/ml-classification-regression.md
--
diff --git a/docs/ml-classification-regression.md 
b/docs/ml-classification-regression.md
index 88457d4..d7e5521 100644
--- a/docs/ml-classification-regression.md
+++ b/docs/ml-classification-regression.md
@@ -815,7 +815,7 @@ The main differences between this API and the [original 
MLlib ensembles API](mll
 ## Random Forests
 
 [Random forests](http://en.wikipedia.org/wiki/Random_forest)
-are ensembles of [decision trees](ml-decision-tree.html).
+are ensembles of [decision 
trees](ml-classification-regression.html#decision-trees).
 Random forests combine many decision trees in order to reduce the risk of 
overfitting.
 The `spark.ml` implementation supports random forests for binary and 
multiclass classification and for regression,
 using both continuous and categorical features.
@@ -896,7 +896,7 @@ All output columns are optional; to exclude an output 
column, set its correspond
 ## Gradient-Boosted Trees (GBTs)
 
 [Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting)
-are ensembles of [decision trees](ml-decision-tree.html).
+are ensembles of [decision 
trees](ml-classification-regression.html#decision-trees).
 GBTs iteratively train decision trees in order to minimize a loss function.
 The `spark.ml` implementation supports GBTs for binary classification and for 
regression,
 using both continuous and categorical features.

http://git-wip-us.apache.org/repos/asf/spark/blob/ad102af1/docs/mllib-data-types.md
--
diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md
index 2ffe0f1..ef56aeb 100644
--- a/docs/mllib-data-types.md
+++ b/docs/mllib-data-types.md
@@ -33,7 +33,7 @@ implementations: 
[`DenseVector`](api/scala/index.html#org.apache.spark.mllib.lin
 using the factory methods implemented in
 [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) to 
create local vectors.
 
-Refer to the [`Vector` Scala 
docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` 
Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors) for 
details on the API.
+Refer to the [`Vector` Scala 
docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` 
Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) for 
details on the API.
 
 {% highlight scala %}
 import org.apache.spark.mllib.linalg.{Vector, Vectors}
@@ -199,7 +199,7 @@ After loading, the feature indices are converted to 
zero-based.
 
[`MLUtils.loadLibSVMFile`](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$)
 reads training
 examples stored in LIBSVM format.
 
-Refer to the [`MLUtils` Scala 
docs](api/scala/index.html#org.apache.spark.mllib.util.MLUtils) for details on 
the API.
+Refer to the [`MLUtils` Scala 

spark git commit: [SPARK-15879][DOCS][UI] Update logo in UI and docs to add "Apache"

2016-06-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 f0fa0a894 -> 4c29c55f2


[SPARK-15879][DOCS][UI] Update logo in UI and docs to add "Apache"

## What changes were proposed in this pull request?

Use new Spark logo including "Apache" (now, with crushed PNGs). Remove old 
unreferenced logo files.

## How was this patch tested?

Manual check of generated HTML site and Spark UI. I searched for references to 
the deleted files to make sure they were not used.

Author: Sean Owen <so...@cloudera.com>

Closes #13609 from srowen/SPARK-15879.

(cherry picked from commit 3761330dd0151d7369d7fba4d4c344e9863990ef)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4c29c55f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4c29c55f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4c29c55f

Branch: refs/heads/branch-2.0
Commit: 4c29c55f22d57c5fbadd0b759155fbab4b07a70a
Parents: f0fa0a8
Author: Sean Owen <so...@cloudera.com>
Authored: Sat Jun 11 12:46:07 2016 +0100
Committer: Sean Owen <so...@cloudera.com>
Committed: Sat Jun 11 12:46:21 2016 +0100

--
 .../spark/ui/static/spark-logo-77x50px-hd.png   | Bin 3536 -> 4182 bytes
 .../org/apache/spark/ui/static/spark_logo.png   | Bin 14233 -> 0 bytes
 docs/img/incubator-logo.png | Bin 11651 -> 0 bytes
 docs/img/spark-logo-100x40px.png| Bin 3635 -> 0 bytes
 docs/img/spark-logo-77x40px-hd.png  | Bin 1904 -> 0 bytes
 docs/img/spark-logo-77x50px-hd.png  | Bin 3536 -> 0 bytes
 docs/img/spark-logo-hd.png  | Bin 13512 -> 16418 bytes
 7 files changed, 0 insertions(+), 0 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png
--
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png 
b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png
index 6c5f099..ffe2550 100644
Binary files 
a/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png 
and 
b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png 
differ

http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png
--
diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png 
b/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png
deleted file mode 100644
index 4b18734..000
Binary files 
a/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png and 
/dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/docs/img/incubator-logo.png
--
diff --git a/docs/img/incubator-logo.png b/docs/img/incubator-logo.png
deleted file mode 100644
index 33ca7f6..000
Binary files a/docs/img/incubator-logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/docs/img/spark-logo-100x40px.png
--
diff --git a/docs/img/spark-logo-100x40px.png b/docs/img/spark-logo-100x40px.png
deleted file mode 100644
index 54c3187..000
Binary files a/docs/img/spark-logo-100x40px.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/docs/img/spark-logo-77x40px-hd.png
--
diff --git a/docs/img/spark-logo-77x40px-hd.png 
b/docs/img/spark-logo-77x40px-hd.png
deleted file mode 100644
index 270402f..000
Binary files a/docs/img/spark-logo-77x40px-hd.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/docs/img/spark-logo-77x50px-hd.png
--
diff --git a/docs/img/spark-logo-77x50px-hd.png 
b/docs/img/spark-logo-77x50px-hd.png
deleted file mode 100644
index 6c5f099..000
Binary files a/docs/img/spark-logo-77x50px-hd.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/4c29c55f/docs/img/spark-logo-hd.png
--
diff --git a/docs/img/spark-logo-hd.png b/docs/img/spark-logo-hd.png
index 1381e30..e4508e7 100644
Binary files a/docs/img/spark-logo-hd.png and b/docs/img/spark-logo-hd.png 
differ


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15879][DOCS][UI] Update logo in UI and docs to add "Apache"

2016-06-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 7504bc73f -> 3761330dd


[SPARK-15879][DOCS][UI] Update logo in UI and docs to add "Apache"

## What changes were proposed in this pull request?

Use new Spark logo including "Apache" (now, with crushed PNGs). Remove old 
unreferenced logo files.

## How was this patch tested?

Manual check of generated HTML site and Spark UI. I searched for references to 
the deleted files to make sure they were not used.

Author: Sean Owen <so...@cloudera.com>

Closes #13609 from srowen/SPARK-15879.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3761330d
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3761330d
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3761330d

Branch: refs/heads/master
Commit: 3761330dd0151d7369d7fba4d4c344e9863990ef
Parents: 7504bc7
Author: Sean Owen <so...@cloudera.com>
Authored: Sat Jun 11 12:46:07 2016 +0100
Committer: Sean Owen <so...@cloudera.com>
Committed: Sat Jun 11 12:46:07 2016 +0100

--
 .../spark/ui/static/spark-logo-77x50px-hd.png   | Bin 3536 -> 4182 bytes
 .../org/apache/spark/ui/static/spark_logo.png   | Bin 14233 -> 0 bytes
 docs/img/incubator-logo.png | Bin 11651 -> 0 bytes
 docs/img/spark-logo-100x40px.png| Bin 3635 -> 0 bytes
 docs/img/spark-logo-77x40px-hd.png  | Bin 1904 -> 0 bytes
 docs/img/spark-logo-77x50px-hd.png  | Bin 3536 -> 0 bytes
 docs/img/spark-logo-hd.png  | Bin 13512 -> 16418 bytes
 7 files changed, 0 insertions(+), 0 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png
--
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png 
b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png
index 6c5f099..ffe2550 100644
Binary files 
a/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png 
and 
b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png 
differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png
--
diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png 
b/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png
deleted file mode 100644
index 4b18734..000
Binary files 
a/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png and 
/dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/docs/img/incubator-logo.png
--
diff --git a/docs/img/incubator-logo.png b/docs/img/incubator-logo.png
deleted file mode 100644
index 33ca7f6..000
Binary files a/docs/img/incubator-logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/docs/img/spark-logo-100x40px.png
--
diff --git a/docs/img/spark-logo-100x40px.png b/docs/img/spark-logo-100x40px.png
deleted file mode 100644
index 54c3187..000
Binary files a/docs/img/spark-logo-100x40px.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/docs/img/spark-logo-77x40px-hd.png
--
diff --git a/docs/img/spark-logo-77x40px-hd.png 
b/docs/img/spark-logo-77x40px-hd.png
deleted file mode 100644
index 270402f..000
Binary files a/docs/img/spark-logo-77x40px-hd.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/docs/img/spark-logo-77x50px-hd.png
--
diff --git a/docs/img/spark-logo-77x50px-hd.png 
b/docs/img/spark-logo-77x50px-hd.png
deleted file mode 100644
index 6c5f099..000
Binary files a/docs/img/spark-logo-77x50px-hd.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/spark/blob/3761330d/docs/img/spark-logo-hd.png
--
diff --git a/docs/img/spark-logo-hd.png b/docs/img/spark-logo-hd.png
index 1381e30..e4508e7 100644
Binary files a/docs/img/spark-logo-hd.png and b/docs/img/spark-logo-hd.png 
differ


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: [SPARK-15883][MLLIB][DOCS] Fix broken links in mllib documents

2016-06-11 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 4c29c55f2 -> 8cf33fb8a


[SPARK-15883][MLLIB][DOCS] Fix broken links in mllib documents

## What changes were proposed in this pull request?

This issue fixes all broken links on Spark 2.0 preview MLLib documents. Also, 
this contains some editorial change.

**Fix broken links**
  * mllib-data-types.md
  * mllib-decision-tree.md
  * mllib-ensembles.md
  * mllib-feature-extraction.md
  * mllib-pmml-model-export.md
  * mllib-statistics.md

**Fix malformed section header and scala coding style**
  * mllib-linear-methods.md

**Replace indirect forward links with direct one**
  * ml-classification-regression.md

## How was this patch tested?

Manual tests (with `cd docs; jekyll build`.)

Author: Dongjoon Hyun 

Closes #13608 from dongjoon-hyun/SPARK-15883.

(cherry picked from commit ad102af169c7344b30d3b84aa16452fcdc22542c)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8cf33fb8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8cf33fb8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8cf33fb8

Branch: refs/heads/branch-2.0
Commit: 8cf33fb8a945e8f76833f68fc99b1ad5dee13641
Parents: 4c29c55
Author: Dongjoon Hyun 
Authored: Sat Jun 11 12:55:38 2016 +0100
Committer: Sean Owen 
Committed: Sat Jun 11 12:55:48 2016 +0100

--
 docs/ml-classification-regression.md |  4 ++--
 docs/mllib-data-types.md | 16 ++--
 docs/mllib-decision-tree.md  |  6 +++---
 docs/mllib-ensembles.md  |  6 +++---
 docs/mllib-feature-extraction.md |  2 +-
 docs/mllib-linear-methods.md | 10 +-
 docs/mllib-pmml-model-export.md  |  2 +-
 docs/mllib-statistics.md |  8 
 8 files changed, 25 insertions(+), 29 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/8cf33fb8/docs/ml-classification-regression.md
--
diff --git a/docs/ml-classification-regression.md 
b/docs/ml-classification-regression.md
index 88457d4..d7e5521 100644
--- a/docs/ml-classification-regression.md
+++ b/docs/ml-classification-regression.md
@@ -815,7 +815,7 @@ The main differences between this API and the [original 
MLlib ensembles API](mll
 ## Random Forests
 
 [Random forests](http://en.wikipedia.org/wiki/Random_forest)
-are ensembles of [decision trees](ml-decision-tree.html).
+are ensembles of [decision 
trees](ml-classification-regression.html#decision-trees).
 Random forests combine many decision trees in order to reduce the risk of 
overfitting.
 The `spark.ml` implementation supports random forests for binary and 
multiclass classification and for regression,
 using both continuous and categorical features.
@@ -896,7 +896,7 @@ All output columns are optional; to exclude an output 
column, set its correspond
 ## Gradient-Boosted Trees (GBTs)
 
 [Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting)
-are ensembles of [decision trees](ml-decision-tree.html).
+are ensembles of [decision 
trees](ml-classification-regression.html#decision-trees).
 GBTs iteratively train decision trees in order to minimize a loss function.
 The `spark.ml` implementation supports GBTs for binary classification and for 
regression,
 using both continuous and categorical features.

http://git-wip-us.apache.org/repos/asf/spark/blob/8cf33fb8/docs/mllib-data-types.md
--
diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md
index 2ffe0f1..ef56aeb 100644
--- a/docs/mllib-data-types.md
+++ b/docs/mllib-data-types.md
@@ -33,7 +33,7 @@ implementations: 
[`DenseVector`](api/scala/index.html#org.apache.spark.mllib.lin
 using the factory methods implemented in
 [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) to 
create local vectors.
 
-Refer to the [`Vector` Scala 
docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` 
Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors) for 
details on the API.
+Refer to the [`Vector` Scala 
docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) and [`Vectors` 
Scala docs](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) for 
details on the API.
 
 {% highlight scala %}
 import org.apache.spark.mllib.linalg.{Vector, Vectors}
@@ -199,7 +199,7 @@ After loading, the feature indices are converted to 
zero-based.
 
[`MLUtils.loadLibSVMFile`](api/scala/index.html#org.apache.spark.mllib.util.MLUtils$)
 reads training
 examples stored in LIBSVM format.
 
-Refer to the [`MLUtils` Scala 

spark git commit: [DOCUMENTATION] fixed typos in python programming guide

2016-06-14 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 974be6241 -> cf52375b9


[DOCUMENTATION] fixed typos in python programming guide

## What changes were proposed in this pull request?

minor typo

## How was this patch tested?

minor typo in the doc, should be self explanatory

Author: Mortada Mehyar 

Closes #13639 from mortada/typo.

(cherry picked from commit a87a56f5c70792eccbb57046f6b26d40494c380a)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cf52375b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cf52375b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cf52375b

Branch: refs/heads/branch-2.0
Commit: cf52375b9f3da84d6aad31134d4f2859de7d447c
Parents: 974be62
Author: Mortada Mehyar 
Authored: Tue Jun 14 09:45:46 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 14 09:45:56 2016 +0100

--
 docs/programming-guide.md | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/cf52375b/docs/programming-guide.md
--
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
index 3f081a0..97bcb51 100644
--- a/docs/programming-guide.md
+++ b/docs/programming-guide.md
@@ -491,7 +491,7 @@ for examples of using Cassandra / HBase ```InputFormat``` 
and ```OutputFormat```
 
 RDDs support two types of operations: *transformations*, which create a new 
dataset from an existing one, and *actions*, which return a value to the driver 
program after running a computation on the dataset. For example, `map` is a 
transformation that passes each dataset element through a function and returns 
a new RDD representing the results. On the other hand, `reduce` is an action 
that aggregates all the elements of the RDD using some function and returns the 
final result to the driver program (although there is also a parallel 
`reduceByKey` that returns a distributed dataset).
 
-All transformations in Spark are lazy, in that they do not compute 
their results right away. Instead, they just remember the transformations 
applied to some base dataset (e.g. a file). The transformations are only 
computed when an action requires a result to be returned to the driver program. 
This design enables Spark to run more efficiently -- for example, we can 
realize that a dataset created through `map` will be used in a `reduce` and 
return only the result of the `reduce` to the driver, rather than the larger 
mapped dataset.
+All transformations in Spark are lazy, in that they do not compute 
their results right away. Instead, they just remember the transformations 
applied to some base dataset (e.g. a file). The transformations are only 
computed when an action requires a result to be returned to the driver program. 
This design enables Spark to run more efficiently. For example, we can realize 
that a dataset created through `map` will be used in a `reduce` and return only 
the result of the `reduce` to the driver, rather than the larger mapped dataset.
 
 By default, each transformed RDD may be recomputed each time you run an action 
on it. However, you may also *persist* an RDD in memory using the `persist` (or 
`cache`) method, in which case Spark will keep the elements around on the 
cluster for much faster access the next time you query it. There is also 
support for persisting RDDs on disk, or replicated across multiple nodes.
 
@@ -618,7 +618,7 @@ class MyClass {
 }
 {% endhighlight %}
 
-Here, if we create a `new MyClass` and call `doStuff` on it, the `map` inside 
there references the
+Here, if we create a new `MyClass` instance and call `doStuff` on it, the 
`map` inside there references the
 `func1` method *of that `MyClass` instance*, so the whole object needs to be 
sent to the cluster. It is
 similar to writing `rdd.map(x => this.func1(x))`.
 
@@ -1156,7 +1156,7 @@ to disk, incurring the additional overhead of disk I/O 
and increased garbage col
 Shuffle also generates a large number of intermediate files on disk. As of 
Spark 1.3, these files
 are preserved until the corresponding RDDs are no longer used and are garbage 
collected.
 This is done so the shuffle files don't need to be re-created if the lineage 
is re-computed.
-Garbage collection may happen only after a long period time, if the 
application retains references
+Garbage collection may happen only after a long period of time, if the 
application retains references
 to these RDDs or if GC does not kick in frequently. This means that 
long-running Spark jobs may
 consume a large amount of disk space. The temporary storage directory is 
specified by the
 `spark.local.dir` configuration 

spark git commit: [DOCUMENTATION] fixed typos in python programming guide

2016-06-14 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 688b6ef9d -> a87a56f5c


[DOCUMENTATION] fixed typos in python programming guide

## What changes were proposed in this pull request?

minor typo

## How was this patch tested?

minor typo in the doc, should be self explanatory

Author: Mortada Mehyar 

Closes #13639 from mortada/typo.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a87a56f5
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a87a56f5
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a87a56f5

Branch: refs/heads/master
Commit: a87a56f5c70792eccbb57046f6b26d40494c380a
Parents: 688b6ef
Author: Mortada Mehyar 
Authored: Tue Jun 14 09:45:46 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 14 09:45:46 2016 +0100

--
 docs/programming-guide.md | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a87a56f5/docs/programming-guide.md
--
diff --git a/docs/programming-guide.md b/docs/programming-guide.md
index 3f081a0..97bcb51 100644
--- a/docs/programming-guide.md
+++ b/docs/programming-guide.md
@@ -491,7 +491,7 @@ for examples of using Cassandra / HBase ```InputFormat``` 
and ```OutputFormat```
 
 RDDs support two types of operations: *transformations*, which create a new 
dataset from an existing one, and *actions*, which return a value to the driver 
program after running a computation on the dataset. For example, `map` is a 
transformation that passes each dataset element through a function and returns 
a new RDD representing the results. On the other hand, `reduce` is an action 
that aggregates all the elements of the RDD using some function and returns the 
final result to the driver program (although there is also a parallel 
`reduceByKey` that returns a distributed dataset).
 
-All transformations in Spark are lazy, in that they do not compute 
their results right away. Instead, they just remember the transformations 
applied to some base dataset (e.g. a file). The transformations are only 
computed when an action requires a result to be returned to the driver program. 
This design enables Spark to run more efficiently -- for example, we can 
realize that a dataset created through `map` will be used in a `reduce` and 
return only the result of the `reduce` to the driver, rather than the larger 
mapped dataset.
+All transformations in Spark are lazy, in that they do not compute 
their results right away. Instead, they just remember the transformations 
applied to some base dataset (e.g. a file). The transformations are only 
computed when an action requires a result to be returned to the driver program. 
This design enables Spark to run more efficiently. For example, we can realize 
that a dataset created through `map` will be used in a `reduce` and return only 
the result of the `reduce` to the driver, rather than the larger mapped dataset.
 
 By default, each transformed RDD may be recomputed each time you run an action 
on it. However, you may also *persist* an RDD in memory using the `persist` (or 
`cache`) method, in which case Spark will keep the elements around on the 
cluster for much faster access the next time you query it. There is also 
support for persisting RDDs on disk, or replicated across multiple nodes.
 
@@ -618,7 +618,7 @@ class MyClass {
 }
 {% endhighlight %}
 
-Here, if we create a `new MyClass` and call `doStuff` on it, the `map` inside 
there references the
+Here, if we create a new `MyClass` instance and call `doStuff` on it, the 
`map` inside there references the
 `func1` method *of that `MyClass` instance*, so the whole object needs to be 
sent to the cluster. It is
 similar to writing `rdd.map(x => this.func1(x))`.
 
@@ -1156,7 +1156,7 @@ to disk, incurring the additional overhead of disk I/O 
and increased garbage col
 Shuffle also generates a large number of intermediate files on disk. As of 
Spark 1.3, these files
 are preserved until the corresponding RDDs are no longer used and are garbage 
collected.
 This is done so the shuffle files don't need to be re-created if the lineage 
is re-computed.
-Garbage collection may happen only after a long period time, if the 
application retains references
+Garbage collection may happen only after a long period of time, if the 
application retains references
 to these RDDs or if GC does not kick in frequently. This means that 
long-running Spark jobs may
 consume a large amount of disk space. The temporary storage directory is 
specified by the
 `spark.local.dir` configuration parameter when configuring the Spark context.


-
To 

spark git commit: [SPARK-15821][DOCS] Include parallel build info

2016-06-14 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 96c3500c6 -> a431e3f1f


[SPARK-15821][DOCS] Include parallel build info

## What changes were proposed in this pull request?

We should mention that users can build Spark using multiple threads to decrease 
build times; either here or in "Building Spark"

## How was this patch tested?

Built on machines with between one core to 192 cores using mvn -T 1C and 
observed faster build times with no loss in stability

In response to the question here 
https://issues.apache.org/jira/browse/SPARK-15821 I think we should suggest 
this option as we know it works for Spark and can result in faster builds

Author: Adam Roberts 

Closes #13562 from a-roberts/patch-3.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a431e3f1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a431e3f1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a431e3f1

Branch: refs/heads/master
Commit: a431e3f1f8575e2498650ac767e69fbc903e9929
Parents: 96c3500
Author: Adam Roberts 
Authored: Tue Jun 14 13:59:01 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 14 13:59:01 2016 +0100

--
 README.md| 2 ++
 dev/make-distribution.sh | 4 ++--
 2 files changed, 4 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/a431e3f1/README.md
--
diff --git a/README.md b/README.md
index d5804d1..c77c429 100644
--- a/README.md
+++ b/README.md
@@ -25,6 +25,8 @@ To build Spark and its example programs, run:
 build/mvn -DskipTests clean package
 
 (You do not need to do this if you downloaded a pre-built package.)
+
+You can build Spark using more than one thread by using the -T option with 
Maven, see ["Parallel builds in Maven 
3"](https://cwiki.apache.org/confluence/display/MAVEN/Parallel+builds+in+Maven+3).
 More detailed documentation is available from the project site, at
 ["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html).
 For developing Spark using an IDE, see 
[Eclipse](https://cwiki.apache.org/confluence/display/SPARK/Useful+Developer+Tools#UsefulDeveloperTools-Eclipse)

http://git-wip-us.apache.org/repos/asf/spark/blob/a431e3f1/dev/make-distribution.sh
--
diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh
index 4f7544f..9be4fdf 100755
--- a/dev/make-distribution.sh
+++ b/dev/make-distribution.sh
@@ -53,7 +53,7 @@ while (( "$#" )); do
 --hadoop)
   echo "Error: '--hadoop' is no longer supported:"
   echo "Error: use Maven profiles and options -Dhadoop.version and 
-Dyarn.version instead."
-  echo "Error: Related profiles include hadoop-2.2, hadoop-2.3 and 
hadoop-2.4."
+  echo "Error: Related profiles include hadoop-2.2, hadoop-2.3, 
hadoop-2.4, hadoop-2.6 and hadoop-2.7."
   exit_with_usage
   ;;
 --with-yarn)
@@ -150,7 +150,7 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g 
-XX:MaxPermSize=512M -XX:ReservedCodeCac
 # Store the command as an array because $MVN variable might have spaces in it.
 # Normal quoting tricks don't work.
 # See: http://mywiki.wooledge.org/BashFAQ/050
-BUILD_COMMAND=("$MVN" clean package -DskipTests $@)
+BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@)
 
 # Actually build the jar
 echo -e "\nBuilding with..."


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



spark git commit: doc fix of HiveThriftServer

2016-06-14 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 0d80bc291 -> e90ba2287


doc fix of HiveThriftServer

## What changes were proposed in this pull request?

Just minor doc fix.

\cc yhuai

Author: Jeff Zhang 

Closes #13659 from zjffdu/doc_fix.

(cherry picked from commit 53bb03084796231f724ff8369490df520e1ee33c)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e90ba228
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e90ba228
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e90ba228

Branch: refs/heads/branch-2.0
Commit: e90ba228787c0a8b50855bafb0bc16eddee8329b
Parents: 0d80bc2
Author: Jeff Zhang 
Authored: Tue Jun 14 14:28:40 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 14 14:28:54 2016 +0100

--
 .../apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala | 2 +-
 .../spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 4 ++--
 .../apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala  | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/e90ba228/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
index c82fa4e..2e0fa1e 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
@@ -30,7 +30,7 @@ import org.apache.spark.ui._
 import org.apache.spark.ui.UIUtils._
 
 
-/** Page for Spark Web UI that shows statistics of a thrift server */
+/** Page for Spark Web UI that shows statistics of the thrift server */
 private[ui] class ThriftServerPage(parent: ThriftServerTab) extends 
WebUIPage("") with Logging {
 
   private val listener = parent.listener

http://git-wip-us.apache.org/repos/asf/spark/blob/e90ba228/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
index 008108a..f39e9dc 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
@@ -29,7 +29,7 @@ import 
org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo,
 import org.apache.spark.ui._
 import org.apache.spark.ui.UIUtils._
 
-/** Page for Spark Web UI that shows statistics of a streaming job */
+/** Page for Spark Web UI that shows statistics of jobs running in the thrift 
server */
 private[ui] class ThriftServerSessionPage(parent: ThriftServerTab)
   extends WebUIPage("session") with Logging {
 
@@ -60,7 +60,7 @@ private[ui] class ThriftServerSessionPage(parent: 
ThriftServerTab)
 UIUtils.headerSparkPage("JDBC/ODBC Session", content, parent, Some(5000))
   }
 
-  /** Generate basic stats of the streaming program */
+  /** Generate basic stats of the thrift server program */
   private def generateBasicStats(): Seq[Node] = {
 val timeSinceStart = System.currentTimeMillis() - startTime.getTime
 

http://git-wip-us.apache.org/repos/asf/spark/blob/e90ba228/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
index 923ba8a..db20660 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
@@ -24,7 +24,7 @@ import 
org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._
 import org.apache.spark.ui.{SparkUI, SparkUITab}
 
 /**
- * Spark Web UI tab that shows statistics of a streaming job.
+ * Spark Web UI tab that shows statistics of jobs running in the thrift server.
  * 

spark git commit: doc fix of HiveThriftServer

2016-06-14 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master a431e3f1f -> 53bb03084


doc fix of HiveThriftServer

## What changes were proposed in this pull request?

Just minor doc fix.

\cc yhuai

Author: Jeff Zhang 

Closes #13659 from zjffdu/doc_fix.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/53bb0308
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/53bb0308
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/53bb0308

Branch: refs/heads/master
Commit: 53bb03084796231f724ff8369490df520e1ee33c
Parents: a431e3f
Author: Jeff Zhang 
Authored: Tue Jun 14 14:28:40 2016 +0100
Committer: Sean Owen 
Committed: Tue Jun 14 14:28:40 2016 +0100

--
 .../apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala | 2 +-
 .../spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 4 ++--
 .../apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala  | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/53bb0308/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
index c82fa4e..2e0fa1e 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala
@@ -30,7 +30,7 @@ import org.apache.spark.ui._
 import org.apache.spark.ui.UIUtils._
 
 
-/** Page for Spark Web UI that shows statistics of a thrift server */
+/** Page for Spark Web UI that shows statistics of the thrift server */
 private[ui] class ThriftServerPage(parent: ThriftServerTab) extends 
WebUIPage("") with Logging {
 
   private val listener = parent.listener

http://git-wip-us.apache.org/repos/asf/spark/blob/53bb0308/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
index 008108a..f39e9dc 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala
@@ -29,7 +29,7 @@ import 
org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.{ExecutionInfo,
 import org.apache.spark.ui._
 import org.apache.spark.ui.UIUtils._
 
-/** Page for Spark Web UI that shows statistics of a streaming job */
+/** Page for Spark Web UI that shows statistics of jobs running in the thrift 
server */
 private[ui] class ThriftServerSessionPage(parent: ThriftServerTab)
   extends WebUIPage("session") with Logging {
 
@@ -60,7 +60,7 @@ private[ui] class ThriftServerSessionPage(parent: 
ThriftServerTab)
 UIUtils.headerSparkPage("JDBC/ODBC Session", content, parent, Some(5000))
   }
 
-  /** Generate basic stats of the streaming program */
+  /** Generate basic stats of the thrift server program */
   private def generateBasicStats(): Seq[Node] = {
 val timeSinceStart = System.currentTimeMillis() - startTime.getTime
 

http://git-wip-us.apache.org/repos/asf/spark/blob/53bb0308/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
--
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
index 923ba8a..db20660 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala
@@ -24,7 +24,7 @@ import 
org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab._
 import org.apache.spark.ui.{SparkUI, SparkUITab}
 
 /**
- * Spark Web UI tab that shows statistics of a streaming job.
+ * Spark Web UI tab that shows statistics of jobs running in the thrift server.
  * This assumes the given SparkContext has enabled its SparkUI.
  */
 private[thriftserver] class ThriftServerTab(sparkContext: 

spark git commit: [MINOR] Clean up several build warnings, mostly due to internal use of old accumulators

2016-06-14 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 e03c25193 -> 24539223b


[MINOR] Clean up several build warnings, mostly due to internal use of old 
accumulators

Another PR to clean up recent build warnings. This particularly cleans up 
several instances of the old accumulator API usage in tests that are 
straightforward to update. I think this qualifies as "minor".

Jenkins

Author: Sean Owen <so...@cloudera.com>

Closes #13642 from srowen/BuildWarnings.

(cherry picked from commit 6151d2641f91c8e3ec0c324e78afb46cdb2ef111)
Signed-off-by: Sean Owen <so...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/24539223
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/24539223
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/24539223

Branch: refs/heads/branch-2.0
Commit: 24539223b043b621a377251bdab206833af78d0c
Parents: e03c251
Author: Sean Owen <so...@cloudera.com>
Authored: Tue Jun 14 09:40:07 2016 -0700
Committer: Sean Owen <so...@cloudera.com>
Committed: Tue Jun 14 20:36:30 2016 +0100

--
 core/pom.xml|   6 +-
 .../spark/scheduler/DAGSchedulerSuite.scala |  12 +--
 .../spark/scheduler/TaskContextSuite.scala  |   9 +-
 .../spark/sql/execution/debug/package.scala |  34 +++---
 .../sql/execution/metric/SQLMetricsSuite.scala  | 105 +--
 .../spark/deploy/yarn/YarnAllocatorSuite.scala  |   1 +
 6 files changed, 31 insertions(+), 136 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/24539223/core/pom.xml
--
diff --git a/core/pom.xml b/core/pom.xml
index f5fdb40..90c8f97 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -356,12 +356,12 @@
 generate-resources
 
   
-  
+  
 
   
-  
+  
 
-  
+  
 
 
   run

http://git-wip-us.apache.org/repos/asf/spark/blob/24539223/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 5bcc8ff..ce4e7a2 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -1593,13 +1593,11 @@ class DAGSchedulerSuite extends SparkFunSuite with 
LocalSparkContext with Timeou
   }
 
   test("misbehaved accumulator should not crash DAGScheduler and 
SparkContext") {
-val acc = new Accumulator[Int](0, new AccumulatorParam[Int] {
-  override def addAccumulator(t1: Int, t2: Int): Int = t1 + t2
-  override def zero(initialValue: Int): Int = 0
-  override def addInPlace(r1: Int, r2: Int): Int = {
-throw new DAGSchedulerSuiteDummyException
-  }
-})
+val acc = new LongAccumulator {
+  override def add(v: java.lang.Long): Unit = throw new 
DAGSchedulerSuiteDummyException
+  override def add(v: Long): Unit = throw new 
DAGSchedulerSuiteDummyException
+}
+sc.register(acc)
 
 // Run this on executors
 sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) }

http://git-wip-us.apache.org/repos/asf/spark/blob/24539223/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
--
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala 
b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
index 368668b..9eda79a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -146,14 +146,13 @@ class TaskContextSuite extends SparkFunSuite with 
BeforeAndAfter with LocalSpark
   test("accumulators are updated on exception failures") {
 // This means use 1 core and 4 max task failures
 sc = new SparkContext("local[1,4]", "test")
-val param = AccumulatorParam.LongAccumulatorParam
 // Create 2 accumulators, one that counts failed values and another that 
doesn't
-val acc1 = new Accumulator(0L, param, Some("x"), countFailedValues = true)
-val acc2 = new Accumulator(0L, param, Some("y"), countFailedValues = false)
+val acc1 = AccumulatorSuite.createLongAccum("x", true)
+val acc2 = AccumulatorSuite.createLongAccum("y", false)
 // Fail first 3 attempts 

spark git commit: [SPARK-15617][ML][DOC] Clarify that fMeasure in MulticlassMetrics is "micro" f1_score

2016-06-04 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 cf8782116 -> 729730159


[SPARK-15617][ML][DOC] Clarify that fMeasure in MulticlassMetrics is "micro" 
f1_score

## What changes were proposed in this pull request?
1, del precision,recall in  `ml.MulticlassClassificationEvaluator`
2, update user guide for `mlllib.weightedFMeasure`

## How was this patch tested?
local build

Author: Ruifeng Zheng 

Closes #13390 from zhengruifeng/clarify_f1.

(cherry picked from commit 2099e05f93067937cdf6cedcf493afd66e212abe)
Signed-off-by: Sean Owen 


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/72973015
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/72973015
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/72973015

Branch: refs/heads/branch-2.0
Commit: 729730159c6236cb437d215388d444f16849f405
Parents: cf87821
Author: Ruifeng Zheng 
Authored: Sat Jun 4 13:56:04 2016 +0100
Committer: Sean Owen 
Committed: Sat Jun 4 13:56:16 2016 +0100

--
 docs/mllib-evaluation-metrics.md| 16 +++-
 .../MulticlassClassificationEvaluator.scala | 12 +---
 .../MulticlassClassificationEvaluatorSuite.scala|  2 +-
 python/pyspark/ml/evaluation.py |  4 +---
 4 files changed, 10 insertions(+), 24 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/72973015/docs/mllib-evaluation-metrics.md
--
diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md
index a269dbf..c49bc4f 100644
--- a/docs/mllib-evaluation-metrics.md
+++ b/docs/mllib-evaluation-metrics.md
@@ -140,7 +140,7 @@ definitions of positive and negative labels is 
straightforward.
  Label based metrics
 
 Opposed to binary classification where there are only two possible labels, 
multiclass classification problems have many
-possible labels and so the concept of label-based metrics is introduced. 
Overall precision measures precision across all
+possible labels and so the concept of label-based metrics is introduced. 
Accuracy measures precision across all
 labels -  the number of times any class was predicted correctly (true 
positives) normalized by the number of data
 points. Precision by label considers only one class, and measures the number 
of time a specific label was predicted
 correctly normalized by the number of times that label appears in the output.
@@ -182,21 +182,11 @@ $$\hat{\delta}(x) = \begin{cases}1 & \text{if $x = 0$}, 
\\ 0 & \text{otherwise}.
   
 
 
-  Overall Precision
-  $PPV = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} 
\hat{\delta}\left(\hat{\mathbf{y}}_i -
-\mathbf{y}_i\right)$
-
-
-  Overall Recall
-  $TPR = \frac{TP}{TP + FN} = \frac{1}{N}\sum_{i=0}^{N-1} 
\hat{\delta}\left(\hat{\mathbf{y}}_i -
+  Accuracy
+  $ACC = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} 
\hat{\delta}\left(\hat{\mathbf{y}}_i -
 \mathbf{y}_i\right)$
 
 
-  Overall F1-measure
-  $F1 = 2 \cdot \left(\frac{PPV \cdot TPR}
-  {PPV + TPR}\right)$
-
-
   Precision by label
   $PPV(\ell) = \frac{TP}{TP + FP} =
   \frac{\sum_{i=0}^{N-1} \hat{\delta}(\hat{\mathbf{y}}_i - \ell) \cdot 
\hat{\delta}(\mathbf{y}_i - \ell)}

http://git-wip-us.apache.org/repos/asf/spark/blob/72973015/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
 
b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
index 0b84e0a..794b1e7 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
@@ -39,16 +39,16 @@ class MulticlassClassificationEvaluator @Since("1.5.0") 
(@Since("1.5.0") overrid
   def this() = this(Identifiable.randomUID("mcEval"))
 
   /**
-   * param for metric name in evaluation (supports `"f1"` (default), 
`"precision"`, `"recall"`,
-   * `"weightedPrecision"`, `"weightedRecall"`, `"accuracy"`)
+   * param for metric name in evaluation (supports `"f1"` (default), 
`"weightedPrecision"`,
+   * `"weightedRecall"`, `"accuracy"`)
* @group param
*/
   @Since("1.5.0")
   val metricName: Param[String] = {
-val allowedParams = ParamValidators.inArray(Array("f1", "precision",
-  "recall", "weightedPrecision", "weightedRecall", "accuracy"))
+val allowedParams = 

spark git commit: [SPARK-15617][ML][DOC] Clarify that fMeasure in MulticlassMetrics is "micro" f1_score

2016-06-04 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 2ca563cc4 -> 2099e05f9


[SPARK-15617][ML][DOC] Clarify that fMeasure in MulticlassMetrics is "micro" 
f1_score

## What changes were proposed in this pull request?
1, del precision,recall in  `ml.MulticlassClassificationEvaluator`
2, update user guide for `mlllib.weightedFMeasure`

## How was this patch tested?
local build

Author: Ruifeng Zheng 

Closes #13390 from zhengruifeng/clarify_f1.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2099e05f
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2099e05f
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2099e05f

Branch: refs/heads/master
Commit: 2099e05f93067937cdf6cedcf493afd66e212abe
Parents: 2ca563c
Author: Ruifeng Zheng 
Authored: Sat Jun 4 13:56:04 2016 +0100
Committer: Sean Owen 
Committed: Sat Jun 4 13:56:04 2016 +0100

--
 docs/mllib-evaluation-metrics.md| 16 +++-
 .../MulticlassClassificationEvaluator.scala | 12 +---
 .../MulticlassClassificationEvaluatorSuite.scala|  2 +-
 python/pyspark/ml/evaluation.py |  4 +---
 4 files changed, 10 insertions(+), 24 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/2099e05f/docs/mllib-evaluation-metrics.md
--
diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md
index a269dbf..c49bc4f 100644
--- a/docs/mllib-evaluation-metrics.md
+++ b/docs/mllib-evaluation-metrics.md
@@ -140,7 +140,7 @@ definitions of positive and negative labels is 
straightforward.
  Label based metrics
 
 Opposed to binary classification where there are only two possible labels, 
multiclass classification problems have many
-possible labels and so the concept of label-based metrics is introduced. 
Overall precision measures precision across all
+possible labels and so the concept of label-based metrics is introduced. 
Accuracy measures precision across all
 labels -  the number of times any class was predicted correctly (true 
positives) normalized by the number of data
 points. Precision by label considers only one class, and measures the number 
of time a specific label was predicted
 correctly normalized by the number of times that label appears in the output.
@@ -182,21 +182,11 @@ $$\hat{\delta}(x) = \begin{cases}1 & \text{if $x = 0$}, 
\\ 0 & \text{otherwise}.
   
 
 
-  Overall Precision
-  $PPV = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} 
\hat{\delta}\left(\hat{\mathbf{y}}_i -
-\mathbf{y}_i\right)$
-
-
-  Overall Recall
-  $TPR = \frac{TP}{TP + FN} = \frac{1}{N}\sum_{i=0}^{N-1} 
\hat{\delta}\left(\hat{\mathbf{y}}_i -
+  Accuracy
+  $ACC = \frac{TP}{TP + FP} = \frac{1}{N}\sum_{i=0}^{N-1} 
\hat{\delta}\left(\hat{\mathbf{y}}_i -
 \mathbf{y}_i\right)$
 
 
-  Overall F1-measure
-  $F1 = 2 \cdot \left(\frac{PPV \cdot TPR}
-  {PPV + TPR}\right)$
-
-
   Precision by label
   $PPV(\ell) = \frac{TP}{TP + FP} =
   \frac{\sum_{i=0}^{N-1} \hat{\delta}(\hat{\mathbf{y}}_i - \ell) \cdot 
\hat{\delta}(\mathbf{y}_i - \ell)}

http://git-wip-us.apache.org/repos/asf/spark/blob/2099e05f/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
--
diff --git 
a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
 
b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
index 0b84e0a..794b1e7 100644
--- 
a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
+++ 
b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala
@@ -39,16 +39,16 @@ class MulticlassClassificationEvaluator @Since("1.5.0") 
(@Since("1.5.0") overrid
   def this() = this(Identifiable.randomUID("mcEval"))
 
   /**
-   * param for metric name in evaluation (supports `"f1"` (default), 
`"precision"`, `"recall"`,
-   * `"weightedPrecision"`, `"weightedRecall"`, `"accuracy"`)
+   * param for metric name in evaluation (supports `"f1"` (default), 
`"weightedPrecision"`,
+   * `"weightedRecall"`, `"accuracy"`)
* @group param
*/
   @Since("1.5.0")
   val metricName: Param[String] = {
-val allowedParams = ParamValidators.inArray(Array("f1", "precision",
-  "recall", "weightedPrecision", "weightedRecall", "accuracy"))
+val allowedParams = ParamValidators.inArray(Array("f1", 
"weightedPrecision",
+  "weightedRecall", "accuracy"))
 new Param(this, "metricName", "metric name in 

spark git commit: [MINOR] Fix Java Lint errors introduced by #13286 and #13280

2016-06-08 Thread srowen
Repository: spark
Updated Branches:
  refs/heads/master 87706eb66 -> f958c1c3e


[MINOR] Fix Java Lint errors introduced by #13286 and #13280

## What changes were proposed in this pull request?

revived #13464

Fix Java Lint errors introduced by #13286 and #13280
Before:
```
Using `mvn` from path: 
/Users/pichu/Project/spark/build/apache-maven-3.3.9/bin/mvn
Java HotSpot(TM) 64-Bit Server VM warning: ignoring option MaxPermSize=512M; 
support was removed in 8.0
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[340,5] 
(whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[341,5] 
(whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[342,5] 
(whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/launcher/LauncherServer.java:[343,5] 
(whitespace) FileTabCharacter: Line contains a tab character.
[ERROR] src/main/java/org/apache/spark/sql/streaming/OutputMode.java:[41,28] 
(naming) MethodName: Method name 'Append' must match pattern 
'^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/java/org/apache/spark/sql/streaming/OutputMode.java:[52,28] 
(naming) MethodName: Method name 'Complete' must match pattern 
'^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] 
src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[61,8]
 (imports) UnusedImports: Unused import - 
org.apache.parquet.schema.PrimitiveType.
[ERROR] 
src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[62,8]
 (imports) UnusedImports: Unused import - org.apache.parquet.schema.Type.
```

## How was this patch tested?
ran `dev/lint-java` locally

Author: Sandeep Singh 

Closes #13559 from techaddict/minor-3.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f958c1c3
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f958c1c3
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f958c1c3

Branch: refs/heads/master
Commit: f958c1c3e292aba98d283637606890f353a9836c
Parents: 87706eb
Author: Sandeep Singh 
Authored: Wed Jun 8 14:51:00 2016 +0100
Committer: Sean Owen 
Committed: Wed Jun 8 14:51:00 2016 +0100

--
 dev/checkstyle-suppressions.xml  | 2 ++
 .../main/java/org/apache/spark/launcher/LauncherServer.java  | 8 
 .../datasources/parquet/SpecificParquetRecordReaderBase.java | 2 --
 3 files changed, 6 insertions(+), 6 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/f958c1c3/dev/checkstyle-suppressions.xml
--
diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml
index bfc2e73..31656ca 100644
--- a/dev/checkstyle-suppressions.xml
+++ b/dev/checkstyle-suppressions.xml
@@ -42,4 +42,6 @@
   
files="src/main/java/org/apache/hive/service/auth/PasswdAuthenticationProvider.java"/>
 
+
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f958c1c3/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
--
diff --git 
a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java 
b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
index 28e9420..ae43f56 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/LauncherServer.java
@@ -337,10 +337,10 @@ class LauncherServer implements Closeable {
   }
   super.close();
   if (handle != null) {
-   if (!handle.getState().isFinal()) {
- LOG.log(Level.WARNING, "Lost connection to spark application.");
- handle.setState(SparkAppHandle.State.LOST);
-   }
+if (!handle.getState().isFinal()) {
+  LOG.log(Level.WARNING, "Lost connection to spark application.");
+  handle.setState(SparkAppHandle.State.LOST);
+}
 handle.disconnect();
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f958c1c3/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
--
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
index 3f7a872..14626e5 100644
--- 

<    3   4   5   6   7   8   9   10   11   12   >