spark git commit: [SPARK-12349][ML] Make spark.ml PCAModel load backwards compatible
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
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 IshizakiCloses #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
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: CK50Closes #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
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: CK50Closes #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
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-borckmansCloses #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
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 SalianCloses #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 …
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 ZhangCloses #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 …
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 ZhangCloses #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
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 OwenProject: 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
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 OwenCommitted: 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
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: toddwanCloses #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
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: toddwanCloses #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.
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 SharmaCloses #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.
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 SharmaCloses #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
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 YangCloses #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
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 HoffmannCloses #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
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 HoffmannCloses #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.
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 SharmaCloses #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.
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 SharmaCloses #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
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 KulhariaCloses #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
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 KulhariaCloses #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
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 KulhariaCloses #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
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 KulhariaCloses #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
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: somideshmukhCloses #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
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: somideshmukhCloses #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)
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
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
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
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
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 FangCloses #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
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 FangCloses #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
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: BrianLondonCloses #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()
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 KleinCloses #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()
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 KleinCloses #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()
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 KleinCloses #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
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
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"
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 YUCloses #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"
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 YUCloses #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
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 YuCloses #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
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 YuCloses #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
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
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.
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 YangCloses #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.
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 YangCloses #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
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
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 …
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 RandallCloses #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 …
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 RandallCloses #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 …
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 RandallCloses #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
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: prabsAuthor: 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
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: prabsAuthor: 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
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 RobertsCloses #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
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 RobertsCloses #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
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 LeeCloses #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'
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 RuiFengCloses #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'
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 RuiFengCloses #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
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 LiangCloses #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
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 LiangCloses #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
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: yinxusenCloses #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
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
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: WeichenXuCloses #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
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: WeichenXuCloses #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
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 RuiFengCloses #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
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 RuiFengCloses #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
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
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
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 RashidCloses #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
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: bomengCloses #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
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: bomengCloses #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
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: bomengCloses #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
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: bomengCloses #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
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 RashidCloses #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
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: WeichenXuCloses #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
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: WeichenXuCloses #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
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 RuiFengCloses #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
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 RuiFengCloses #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
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 BorjeCloses #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
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 BorjeCloses #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
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 RuiFengCloses #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
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 RuiFengCloses #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
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.comCloses #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
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: dding3Closes #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
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: dding3Closes #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
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.comCloses #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
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 AbledaCloses #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
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 AbledaCloses #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
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 HyunCloses #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"
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"
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
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 HyunCloses #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
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 MehyarCloses #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
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 MehyarCloses #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
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 RobertsCloses #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
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 ZhangCloses #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
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 ZhangCloses #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
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
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 ZhengCloses #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
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 ZhengCloses #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
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 SinghCloses #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 ---