spark git commit: [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests
Repository: spark Updated Branches: refs/heads/branch-1.6 24d59fb64 -> 60e095b9b [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests ## What changes were proposed in this pull request? Make spill tests wait until job has completed before returning the number of stages that spilled ## How was this patch tested? Existing Jenkins tests. Author: Sean Owen Closes #13896 from srowen/SPARK-16193. (cherry picked from commit e87741589a24821b5fe73e5d9ee2164247998580) 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/60e095b9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/60e095b9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/60e095b9 Branch: refs/heads/branch-1.6 Commit: 60e095b9bea3caa3e9d1e768d116f911a048d8ec Parents: 24d59fb Author: Sean Owen Authored: Sat Jun 25 12:14:14 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 12:14:40 2016 +0100 -- core/src/main/scala/org/apache/spark/TestUtils.scala | 13 - 1 file changed, 12 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/60e095b9/core/src/main/scala/org/apache/spark/TestUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 43c89b2..871b9d1 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -22,6 +22,7 @@ import java.net.{URI, URL} import java.nio.charset.StandardCharsets import java.nio.file.Paths import java.util.Arrays +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConverters._ @@ -190,8 +191,14 @@ private[spark] object TestUtils { private class SpillListener extends SparkListener { private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]] private val spilledStageIds = new mutable.HashSet[Int] + private val stagesDone = new CountDownLatch(1) - def numSpilledStages: Int = spilledStageIds.size + def numSpilledStages: Int = { +// Long timeout, just in case somehow the job end isn't notified. +// Fails if a timeout occurs +assert(stagesDone.await(10, TimeUnit.SECONDS)) +spilledStageIds.size + } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { stageIdToTaskMetrics.getOrElseUpdate( @@ -206,4 +213,8 @@ private class SpillListener extends SparkListener { spilledStageIds += stageId } } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { +stagesDone.countDown() + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests
Repository: spark Updated Branches: refs/heads/master 3ee9695d1 -> e87741589 [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests ## What changes were proposed in this pull request? Make spill tests wait until job has completed before returning the number of stages that spilled ## How was this patch tested? Existing Jenkins tests. Author: Sean Owen Closes #13896 from srowen/SPARK-16193. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e8774158 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e8774158 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e8774158 Branch: refs/heads/master Commit: e87741589a24821b5fe73e5d9ee2164247998580 Parents: 3ee9695 Author: Sean Owen Authored: Sat Jun 25 12:14:14 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 12:14:14 2016 +0100 -- core/src/main/scala/org/apache/spark/TestUtils.scala | 13 - 1 file changed, 12 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e8774158/core/src/main/scala/org/apache/spark/TestUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 43c89b2..871b9d1 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -22,6 +22,7 @@ import java.net.{URI, URL} import java.nio.charset.StandardCharsets import java.nio.file.Paths import java.util.Arrays +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConverters._ @@ -190,8 +191,14 @@ private[spark] object TestUtils { private class SpillListener extends SparkListener { private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]] private val spilledStageIds = new mutable.HashSet[Int] + private val stagesDone = new CountDownLatch(1) - def numSpilledStages: Int = spilledStageIds.size + def numSpilledStages: Int = { +// Long timeout, just in case somehow the job end isn't notified. +// Fails if a timeout occurs +assert(stagesDone.await(10, TimeUnit.SECONDS)) +spilledStageIds.size + } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { stageIdToTaskMetrics.getOrElseUpdate( @@ -206,4 +213,8 @@ private class SpillListener extends SparkListener { spilledStageIds += stageId } } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { +stagesDone.countDown() + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests
Repository: spark Updated Branches: refs/heads/branch-2.0 cbfcdcfb6 -> b03b0976f [SPARK-16193][TESTS] Address flaky ExternalAppendOnlyMapSuite spilling tests ## What changes were proposed in this pull request? Make spill tests wait until job has completed before returning the number of stages that spilled ## How was this patch tested? Existing Jenkins tests. Author: Sean Owen Closes #13896 from srowen/SPARK-16193. (cherry picked from commit e87741589a24821b5fe73e5d9ee2164247998580) 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/b03b0976 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b03b0976 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b03b0976 Branch: refs/heads/branch-2.0 Commit: b03b0976fac878bf7e5d1721441179a4d4d9c317 Parents: cbfcdcf Author: Sean Owen Authored: Sat Jun 25 12:14:14 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 12:14:24 2016 +0100 -- core/src/main/scala/org/apache/spark/TestUtils.scala | 13 - 1 file changed, 12 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b03b0976/core/src/main/scala/org/apache/spark/TestUtils.scala -- diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 43c89b2..871b9d1 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -22,6 +22,7 @@ import java.net.{URI, URL} import java.nio.charset.StandardCharsets import java.nio.file.Paths import java.util.Arrays +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConverters._ @@ -190,8 +191,14 @@ private[spark] object TestUtils { private class SpillListener extends SparkListener { private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]] private val spilledStageIds = new mutable.HashSet[Int] + private val stagesDone = new CountDownLatch(1) - def numSpilledStages: Int = spilledStageIds.size + def numSpilledStages: Int = { +// Long timeout, just in case somehow the job end isn't notified. +// Fails if a timeout occurs +assert(stagesDone.await(10, TimeUnit.SECONDS)) +spilledStageIds.size + } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { stageIdToTaskMetrics.getOrElseUpdate( @@ -206,4 +213,8 @@ private class SpillListener extends SparkListener { spilledStageIds += stageId } } + + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { +stagesDone.countDown() + } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-1301][WEB UI] Added anchor links to Accumulators and Tasks on StagePage
Repository: spark Updated Branches: refs/heads/master bf665a958 -> 3ee9695d1 [SPARK-1301][WEB UI] Added anchor links to Accumulators and Tasks on StagePage ## What changes were proposed in this pull request? Sometimes the "Aggregated Metrics by Executor" table on the Stage page can get very long so actor links to the Accumulators and Tasks tables below it have been added to the summary at the top of the page. This has been done in the same way as the Jobs and Stages pages. Note: the Accumulators link only displays when the table exists. ## How was this patch tested? Manually Tested and dev/run-tests ![justtasks](https://cloud.githubusercontent.com/assets/13952758/15165269/6e8efe8c-16c9-11e6-9784-cffe966fdcf0.png) ![withaccumulators](https://cloud.githubusercontent.com/assets/13952758/15165270/7019ec9e-16c9-11e6-8649-db69ed7a317d.png) Author: Alex Bozarth Closes #13037 from ajbozarth/spark1301. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3ee9695d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3ee9695d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3ee9695d Branch: refs/heads/master Commit: 3ee9695d1fcf3750cbf7896a56f8a1ba93f4e82f Parents: bf665a9 Author: Alex Bozarth Authored: Sat Jun 25 09:27:22 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 09:27:22 2016 +0100 -- .../org/apache/spark/ui/static/webui.css| 4 +- .../org/apache/spark/ui/static/webui.js | 47 .../scala/org/apache/spark/ui/UIUtils.scala | 1 + .../org/apache/spark/ui/jobs/StagePage.scala| 16 ++- 4 files changed, 64 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3ee9695d/core/src/main/resources/org/apache/spark/ui/static/webui.css -- diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 595e80a..b157f3e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -155,7 +155,7 @@ pre { display: none; } -span.expand-additional-metrics, span.expand-dag-viz { +span.expand-additional-metrics, span.expand-dag-viz, span.collapse-table { cursor: pointer; } @@ -163,7 +163,7 @@ span.additional-metric-title { cursor: pointer; } -.additional-metrics.collapsed { +.additional-metrics.collapsed, .collapsible-table.collapsed { display: none; } http://git-wip-us.apache.org/repos/asf/spark/blob/3ee9695d/core/src/main/resources/org/apache/spark/ui/static/webui.js -- diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js new file mode 100644 index 000..e37307a --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -0,0 +1,47 @@ +/* + * 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. + */ + +function collapseTablePageLoad(name, table){ + if (window.localStorage.getItem(name) == "true") { +// Set it to false so that the click function can revert it +window.localStorage.setItem(name, "false"); +collapseTable(name, table); + } +} + +function collapseTable(thisName, table){ +var status = window.localStorage.getItem(thisName) == "true"; +status = !status; + +thisClass = '.' + thisName + +// Expand the list of additional metrics. +var tableDiv = $(thisClass).parent().find('.' + table); +$(tableDiv).toggleClass('collapsed'); + +// Switch the class of the arrow from open to closed. +$(thisClass).find('.collapse-table-arrow').toggleClass('arrow-open'); +$(thisClass).find('.collapse-table-arrow').toggleClass('arrow-closed'); + +window.localStorage.setItem(thisName, "" + status); +} + +// Add a call to collapseTablePageLoad() on each collapsible table +// to remember if it's collapsed on each page re
spark git commit: [SPARK-15958] Make initial buffer size for the Sorter configurable
Repository: spark Updated Branches: refs/heads/master a3c7b4187 -> bf665a958 [SPARK-15958] Make initial buffer size for the Sorter configurable ## What changes were proposed in this pull request? Currently the initial buffer size in the sorter is hard coded inside the code and is too small for large workload. As a result, the sorter spends significant time expanding the buffer size and copying the data. It would be useful to have it configurable. ## How was this patch tested? Tested by running a job on the cluster. Author: Sital Kedia Closes #13699 from sitalkedia/config_sort_buffer_upstream. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bf665a95 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bf665a95 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bf665a95 Branch: refs/heads/master Commit: bf665a958631125a1670504ef5966ef1a0e14798 Parents: a3c7b41 Author: Sital Kedia Authored: Sat Jun 25 09:13:39 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 09:13:39 2016 +0100 -- .../org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java| 7 +-- .../apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java | 4 ++-- .../apache/spark/sql/execution/UnsafeExternalRowSorter.java | 4 +++- .../apache/spark/sql/execution/UnsafeKVExternalSorter.java| 7 +-- 4 files changed, 15 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bf665a95/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java -- diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index daa63d4..05fa04c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -61,7 +61,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); @VisibleForTesting - static final int INITIAL_SORT_BUFFER_SIZE = 4096; + static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; private final BlockManager blockManager; private final IndexShuffleBlockResolver shuffleBlockResolver; @@ -74,6 +74,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final TaskContext taskContext; private final SparkConf sparkConf; private final boolean transferToEnabled; + private final int initialSortBufferSize; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -122,6 +123,8 @@ public class UnsafeShuffleWriter extends ShuffleWriter { this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); +this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", + DEFAULT_INITIAL_SORT_BUFFER_SIZE); open(); } @@ -187,7 +190,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { memoryManager, blockManager, taskContext, - INITIAL_SORT_BUFFER_SIZE, + initialSortBufferSize, partitioner.numPartitions(), sparkConf, writeMetrics); http://git-wip-us.apache.org/repos/asf/spark/blob/bf665a95/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java -- diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 7dd61f8..daeb467 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -413,10 +413,10 @@ public class UnsafeShuffleWriterSuite { } private void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception { -memoryManager.limit(UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE * 16); +memoryManager.limit(UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); -for (int i = 0; i < UnsafeShuffleWriter.INITIAL_SORT_BUFFER_SIZE + 1; i++) { +for (int i = 0; i < UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { dataToWrite.add(new Tuple2(i, i)); } writer.write(dataToWrite.iterator()); http://git-wip-us.apache.org/repos/asf/spark/blob/bf665a95/sql/catalyst/src/main/java/org/apache/spark/sq
spark git commit: [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution.
Repository: spark Updated Branches: refs/heads/branch-2.0 d079b5de7 -> cbfcdcfb6 [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution. ## What changes were proposed in this pull request? Just adjust the size of an array in line 58 so it does not cause an ArrayOutOfBoundsException in line 66. ## How was this patch tested? Manual tests. I have recompiled the entire project with the fix, it has been built successfully and I have run the code, also with good results. line 66: val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 crashes because trueWeights has length "nfeatures + 1" while "x" has length "features", and they should have the same length. To fix this just make trueWeights be the same length as x. I have recompiled the project with the change and it is working now: [spark-1.6.1]$ spark-submit --master local[*] --class org.apache.spark.mllib.util.SVMDataGenerator mllib/target/spark-mllib_2.11-1.6.1.jar local /home/user/test And it generates the data successfully now in the specified folder. Author: José Antonio Closes #13895 from j4munoz/patch-2. (cherry picked from commit a3c7b4187bad00dad87df7e3b5929a44d29568ed) 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/cbfcdcfb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cbfcdcfb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cbfcdcfb Branch: refs/heads/branch-2.0 Commit: cbfcdcfb60d41126e17cddda52922d6058f1a401 Parents: d079b5d Author: José Antonio Authored: Sat Jun 25 09:11:25 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 09:11:35 2016 +0100 -- .../main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cbfcdcfb/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index cde5979..c946860 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -55,7 +55,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) -val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) +val trueWeights = Array.fill[Double](nfeatures)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution.
Repository: spark Updated Branches: refs/heads/branch-1.6 b7acc1b71 -> 24d59fb64 [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution. ## What changes were proposed in this pull request? Just adjust the size of an array in line 58 so it does not cause an ArrayOutOfBoundsException in line 66. ## How was this patch tested? Manual tests. I have recompiled the entire project with the fix, it has been built successfully and I have run the code, also with good results. line 66: val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 crashes because trueWeights has length "nfeatures + 1" while "x" has length "features", and they should have the same length. To fix this just make trueWeights be the same length as x. I have recompiled the project with the change and it is working now: [spark-1.6.1]$ spark-submit --master local[*] --class org.apache.spark.mllib.util.SVMDataGenerator mllib/target/spark-mllib_2.11-1.6.1.jar local /home/user/test And it generates the data successfully now in the specified folder. Author: José Antonio Closes #13895 from j4munoz/patch-2. (cherry picked from commit a3c7b4187bad00dad87df7e3b5929a44d29568ed) 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/24d59fb6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/24d59fb6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/24d59fb6 Branch: refs/heads/branch-1.6 Commit: 24d59fb64770fb8951794df9ee6398329838359a Parents: b7acc1b Author: José Antonio Authored: Sat Jun 25 09:11:25 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 09:11:47 2016 +0100 -- .../main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/24d59fb6/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index cde5979..c946860 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -55,7 +55,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) -val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) +val trueWeights = Array.fill[Double](nfeatures)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution.
Repository: spark Updated Branches: refs/heads/master a7d29499d -> a3c7b4187 [MLLIB] org.apache.spark.mllib.util.SVMDataGenerator generates ArrayIndexOutOfBoundsException. I have found the bug and tested the solution. ## What changes were proposed in this pull request? Just adjust the size of an array in line 58 so it does not cause an ArrayOutOfBoundsException in line 66. ## How was this patch tested? Manual tests. I have recompiled the entire project with the fix, it has been built successfully and I have run the code, also with good results. line 66: val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 crashes because trueWeights has length "nfeatures + 1" while "x" has length "features", and they should have the same length. To fix this just make trueWeights be the same length as x. I have recompiled the project with the change and it is working now: [spark-1.6.1]$ spark-submit --master local[*] --class org.apache.spark.mllib.util.SVMDataGenerator mllib/target/spark-mllib_2.11-1.6.1.jar local /home/user/test And it generates the data successfully now in the specified folder. Author: José Antonio Closes #13895 from j4munoz/patch-2. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a3c7b418 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a3c7b418 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a3c7b418 Branch: refs/heads/master Commit: a3c7b4187bad00dad87df7e3b5929a44d29568ed Parents: a7d2949 Author: José Antonio Authored: Sat Jun 25 09:11:25 2016 +0100 Committer: Sean Owen Committed: Sat Jun 25 09:11:25 2016 +0100 -- .../main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a3c7b418/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala -- diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index cde5979..c946860 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -55,7 +55,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) -val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) +val trueWeights = Array.fill[Double](nfeatures)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
svn commit: r1750186 - in /spark/site/docs/1.6.2: ./ api/ api/R/ api/java/ api/java/lib/ api/java/org/ api/java/org/apache/ api/java/org/apache/spark/ api/java/org/apache/spark/annotation/ api/java/or
Author: rxin Date: Sat Jun 25 08:02:58 2016 New Revision: 1750186 URL: http://svn.apache.org/viewvc?rev=1750186&view=rev Log: add 1.6.2 doc [This commit notification would consist of 931 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
svn commit: r14146 - /dev/spark/spark-1.6.2/ /release/spark/spark-1.6.2/
Author: rxin Date: Sat Jun 25 07:16:01 2016 New Revision: 14146 Log: Spark 1.6.2 release Added: release/spark/spark-1.6.2/ - copied from r14145, dev/spark/spark-1.6.2/ Removed: dev/spark/spark-1.6.2/ - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
svn commit: r14145 - /dev/spark/spark-1.6.2/
Author: rxin Date: Sat Jun 25 07:07:25 2016 New Revision: 14145 Log: Add spark-1.6.2 Added: dev/spark/spark-1.6.2/ dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.sha dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz.sha dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.3.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.3.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.3.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.3.tgz.sha dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.4.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.4.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.4.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.4.tgz.sha dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.6.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.6.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.6.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop2.6.tgz.sha dev/spark/spark-1.6.2/spark-1.6.2-bin-without-hadoop.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2-bin-without-hadoop.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2-bin-without-hadoop.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2-bin-without-hadoop.tgz.sha dev/spark/spark-1.6.2/spark-1.6.2.tgz (with props) dev/spark/spark-1.6.2/spark-1.6.2.tgz.asc dev/spark/spark-1.6.2/spark-1.6.2.tgz.md5 dev/spark/spark-1.6.2/spark-1.6.2.tgz.sha Added: dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz == Binary file - no diff available. Propchange: dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz -- svn:mime-type = application/octet-stream Added: dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.asc == --- dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.asc (added) +++ dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.asc Sat Jun 25 07:07:25 2016 @@ -0,0 +1,11 @@ +-BEGIN PGP SIGNATURE- +Version: GnuPG v2.0.14 (GNU/Linux) + +iQEcBAABAgAGBQJXafRUAAoJEHxsEF/8jtCJOVcH/10JgFjieKK801hEvTK85UWg +OPqvVEq6whELA5vn6zbvbX/ti86NNHmGpTqqn2jz5BwKzr6czZecF+u8JVj/cf9+ +HHbEJcDav2NW89xZBZH+YApSOTft2ePRTy9vgd6YelrXNSQ/UZWG69sfrBoNBjol +Ph/FbOBww2dSppuriHZ4PgDGI7cKmNXsGaFrbhD5eYApS3pvrtY7M864xLdDR8yR +a/Tr5Z1NVMSpb9out9qt+zoIbAMCw6B3LCjeUftoIiIko0GXmmXkphzrE2CzkCoG +JWGOBAAYuUpFuEefJ0b3eDw5vMPBsxWW9hNxLHcBcq3MNObcJHbDRPMLuT1WHbQ= +=THQi +-END PGP SIGNATURE- Added: dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.md5 == --- dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.md5 (added) +++ dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.md5 Sat Jun 25 07:07:25 2016 @@ -0,0 +1 @@ +spark-1.6.2-bin-cdh4.tgz: 09 32 D6 CE 2A D7 01 45 B8 0A 09 D8 49 45 32 7C Added: dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.sha == --- dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.sha (added) +++ dev/spark/spark-1.6.2/spark-1.6.2-bin-cdh4.tgz.sha Sat Jun 25 07:07:25 2016 @@ -0,0 +1,3 @@ +spark-1.6.2-bin-cdh4.tgz: BCD0C9F9 EF4FD6CD 8F144FBD 73DA4F4C 2345C37C 56EA54A9 + 33B9568D 68FB7935 DB9F2761 9DF39F8D 01D49D81 204E421E + F6EA9E23 DF751025 A3EB58BC 592A64F4 Added: dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz == Binary file - no diff available. Propchange: dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz -- svn:mime-type = application/octet-stream Added: dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz.asc == --- dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz.asc (added) +++ dev/spark/spark-1.6.2/spark-1.6.2-bin-hadoop1-scala2.11.tgz.asc Sat Jun 25 07:07:25 2016 @@ -0,0 +1,11 @@ +-BEGIN PGP SIGNATURE- +Version: GnuPG v2.0.14 (GNU/Linux) + +iQEcBAABAgAGBQJXafQQAAoJEHxsEF/8jtCJljYH/0IY+Z3VnzqypFAKciclVtjj +NETCNISEs3qq+yJOlahBPh159p/D9LNMeSQDixJD98Nfm5Hac+PGeRhsTiBgodF3 +ajU9y2vRNb35+V63Jqmz8DBj95VFXCCTBJj8UsSpKG7ITti0kvyk1u81P1JUbQ3e +tkvegxby6Bl4REyoDtTekk9OifRqvwTBPknqWum+r7xhThqvZjdf/9aCvo7Rpffe +gDU/dLg8VUXSYooV1kYht36PZiOaW3YJ4ys4c4grbWAVadoVKlUE1