Skip to content
Snippets Groups Projects
  1. May 01, 2017
    • Ryan Blue's avatar
      [SPARK-20540][CORE] Fix unstable executor requests. · 2b2dd08e
      Ryan Blue authored
      There are two problems fixed in this commit. First, the
      ExecutorAllocationManager sets a timeout to avoid requesting executors
      too often. However, the timeout is always updated based on its value and
      a timeout, not the current time. If the call is delayed by locking for
      more than the ongoing scheduler timeout, the manager will request more
      executors on every run. This seems to be the main cause of SPARK-20540.
      
      The second problem is that the total number of requested executors is
      not tracked by the CoarseGrainedSchedulerBackend. Instead, it calculates
      the value based on the current status of 3 variables: the number of
      known executors, the number of executors that have been killed, and the
      number of pending executors. But, the number of pending executors is
      never less than 0, even though there may be more known than requested.
      When executors are killed and not replaced, this can cause the request
      sent to YARN to be incorrect because there were too many executors due
      to the scheduler's state being slightly out of date. This is fixed by tracking
      the currently requested size explicitly.
      
      ## How was this patch tested?
      
      Existing tests.
      
      Author: Ryan Blue <blue@apache.org>
      
      Closes #17813 from rdblue/SPARK-20540-fix-dynamic-allocation.
      2b2dd08e
    • Kunal Khamar's avatar
      [SPARK-20464][SS] Add a job group and description for streaming queries and... · 6fc6cf88
      Kunal Khamar authored
      [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
      
      ## What changes were proposed in this pull request?
      
      Job group: adding a job group is required to properly cancel running jobs related to a query.
      Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.
      
      ## How was this patch tested?
      
      - Unit tests
      - UI screenshot
      
        - Order by job id:
      ![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)
      
        - Order by description:
      ![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)
      
        - Order by job id (no query name):
      ![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)
      
        - Order by description (no query name):
      ![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)
      
      Author: Kunal Khamar <kkhamar@outlook.com>
      
      Closes #17765 from kunalkhamar/sc-6696.
      6fc6cf88
    • jerryshao's avatar
      [SPARK-20517][UI] Fix broken history UI download link · ab30590f
      jerryshao authored
      The download link in history server UI is concatenated with:
      
      ```
       <td><a href="{{uiroot}}/api/v1/applications/{{id}}/{{num}}/logs" class="btn btn-info btn-mini">Download</a></td>
      ```
      
      Here `num` field represents number of attempts, this is not equal to REST APIs. In the REST API, if attempt id is not existed the URL should be `api/v1/applications/<id>/logs`, otherwise the URL should be `api/v1/applications/<id>/<attemptId>/logs`. Using `<num>` to represent `<attemptId>` will lead to the issue of "no such app".
      
      Manual verification.
      
      CC ajbozarth can you please review this change, since you add this feature before? Thanks!
      
      Author: jerryshao <sshao@hortonworks.com>
      
      Closes #17795 from jerryshao/SPARK-20517.
      ab30590f
  2. Apr 28, 2017
    • Aaditya Ramesh's avatar
      [SPARK-19525][CORE] Add RDD checkpoint compression support · 77bcd77e
      Aaditya Ramesh authored
      ## What changes were proposed in this pull request?
      
      This PR adds RDD checkpoint compression support and add a new config `spark.checkpoint.compress` to enable/disable it. Credit goes to aramesh117
      
      Closes #17024
      
      ## How was this patch tested?
      
      The new unit test.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      Author: Aaditya Ramesh <aramesh@conviva.com>
      
      Closes #17789 from zsxwing/pr17024.
      77bcd77e
    • Mark Grover's avatar
      [SPARK-20514][CORE] Upgrade Jetty to 9.3.11.v20160721 · 5d71f3db
      Mark Grover authored
      Upgrade Jetty so it can work with Hadoop 3 (alpha 2 release, in particular).
      Without this change, because of incompatibily between Jetty versions,
      Spark fails to compile when built against Hadoop 3
      
      ## How was this patch tested?
      Unit tests being run.
      
      Author: Mark Grover <mark@apache.org>
      
      Closes #17790 from markgrover/spark-20514.
      5d71f3db
    • hyukjinkwon's avatar
      [SPARK-20465][CORE] Throws a proper exception when any temp directory could not be got · 8c911ada
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR proposes to throw an exception with better message rather than `ArrayIndexOutOfBoundsException` when temp directories could not be created.
      
      Running the commands below:
      
      ```bash
      ./bin/spark-shell --conf spark.local.dir=/NONEXISTENT_DIR_ONE,/NONEXISTENT_DIR_TWO
      ```
      
      produces ...
      
      **Before**
      
      ```
      Exception in thread "main" java.lang.ExceptionInInitializerError
              ...
      Caused by: java.lang.ArrayIndexOutOfBoundsException: 0
              ...
      ```
      
      **After**
      
      ```
      Exception in thread "main" java.lang.ExceptionInInitializerError
              ...
      Caused by: java.io.IOException: Failed to get a temp directory under [/NONEXISTENT_DIR_ONE,/NONEXISTENT_DIR_TWO].
              ...
      ```
      
      ## How was this patch tested?
      
      Unit tests in `LocalDirsSuite.scala`.
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #17768 from HyukjinKwon/throws-temp-dir-exception.
      8c911ada
  3. Apr 27, 2017
    • Wenchen Fan's avatar
      [SPARK-12837][CORE] Do not send the name of internal accumulator to executor side · b90bf520
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      When sending accumulator updates back to driver, the network overhead is pretty big as there are a lot of accumulators, e.g. `TaskMetrics` will send about 20 accumulators everytime, there may be a lot of `SQLMetric` if the query plan is complicated.
      
      Therefore, it's critical to reduce the size of serialized accumulator. A simple way is to not send the name of internal accumulators to executor side, as it's unnecessary. When executor sends accumulator updates back to driver, we can look up the accumulator name in `AccumulatorContext` easily. Note that, we still need to send names of normal accumulators, as the user code run at executor side may rely on accumulator names.
      
      In the future, we should reimplement `TaskMetrics` to not rely on accumulators and use custom serialization.
      
      Tried on the example in https://issues.apache.org/jira/browse/SPARK-12837, the size of serialized accumulator has been cut down by about 40%.
      
      ## How was this patch tested?
      
      existing tests.
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #17596 from cloud-fan/oom.
      b90bf520
    • Shixiong Zhu's avatar
      [SPARK-20461][CORE][SS] Use UninterruptibleThread for Executor and fix the... · 01c999e7
      Shixiong Zhu authored
      [SPARK-20461][CORE][SS] Use UninterruptibleThread for Executor and fix the potential hang in CachedKafkaConsumer
      
      ## What changes were proposed in this pull request?
      
      This PR changes Executor's threads to `UninterruptibleThread` so that we can use `runUninterruptibly` in `CachedKafkaConsumer`. However, this is just best effort to avoid hanging forever. If the user uses`CachedKafkaConsumer` in another thread (e.g., create a new thread or Future), the potential hang may still happen.
      
      ## How was this patch tested?
      
      The new added test.
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #17761 from zsxwing/int.
      01c999e7
    • jinxing's avatar
      [SPARK-20426] Lazy initialization of FileSegmentManagedBuffer for shuffle service. · 85c6ce61
      jinxing authored
      ## What changes were proposed in this pull request?
      When application contains large amount of shuffle blocks. NodeManager requires lots of memory to keep metadata(`FileSegmentManagedBuffer`) in `StreamManager`. When the number of shuffle blocks is big enough. NodeManager can run OOM. This pr proposes to do lazy initialization of `FileSegmentManagedBuffer` in shuffle service.
      
      ## How was this patch tested?
      
      Manually test.
      
      Author: jinxing <jinxing6042@126.com>
      
      Closes #17744 from jinxing64/SPARK-20426.
      85c6ce61
    • Marcelo Vanzin's avatar
      [SPARK-20421][CORE] Mark internal listeners as deprecated. · 561e9cc3
      Marcelo Vanzin authored
      These listeners weren't really meant for external consumption, but they're
      public and marked with DeveloperApi. Adding the deprecated tag warns people
      that they may soon go away (as they will as part of the work for SPARK-18085).
      
      Note that not all types made public by https://github.com/apache/spark/pull/648
      are being deprecated. Some remaining types are still exposed through the
      SparkListener API.
      
      Also note the text for StorageStatus is a tiny bit different, since I'm not
      so sure I'll be able to remove it. But the effect for the users should be the
      same (they should stop trying to use it).
      
      Author: Marcelo Vanzin <vanzin@cloudera.com>
      
      Closes #17766 from vanzin/SPARK-20421.
      561e9cc3
  4. Apr 26, 2017
    • Mark Grover's avatar
      [SPARK-20435][CORE] More thorough redaction of sensitive information · 66636ef0
      Mark Grover authored
      This change does a more thorough redaction of sensitive information from logs and UI
      Add unit tests that ensure that no regressions happen that leak sensitive information to the logs.
      
      The motivation for this change was appearance of password like so in `SparkListenerEnvironmentUpdate` in event logs under some JVM configurations:
      `"sun.java.command":"org.apache.spark.deploy.SparkSubmit ... --conf spark.executorEnv.HADOOP_CREDSTORE_PASSWORD=secret_password ..."
      `
      Previously redaction logic was only checking if the key matched the secret regex pattern, it'd redact it's value. That worked for most cases. However, in the above case, the key (sun.java.command) doesn't tell much, so the value needs to be searched. This PR expands the check to check for values as well.
      
      ## How was this patch tested?
      
      New unit tests added that ensure that no sensitive information is present in the event logs or the yarn logs. Old unit test in UtilsSuite was modified because the test was asserting that a non-sensitive property's value won't be redacted. However, the non-sensitive value had the literal "secret" in it which was causing it to redact. Simply updating the non-sensitive property's value to another arbitrary value (that didn't have "secret" in it) fixed it.
      
      Author: Mark Grover <mark@apache.org>
      
      Closes #17725 from markgrover/spark-20435.
      66636ef0
    • jerryshao's avatar
      [SPARK-20391][CORE] Rename memory related fields in ExecutorSummay · 66dd5b83
      jerryshao authored
      ## What changes were proposed in this pull request?
      
      This is a follow-up of #14617 to make the name of memory related fields more meaningful.
      
      Here  for the backward compatibility, I didn't change `maxMemory` and `memoryUsed` fields.
      
      ## How was this patch tested?
      
      Existing UT and local verification.
      
      CC squito and tgravescs .
      
      Author: jerryshao <sshao@hortonworks.com>
      
      Closes #17700 from jerryshao/SPARK-20391.
      66dd5b83
  5. Apr 25, 2017
    • ding's avatar
      [SPARK-5484][GRAPHX] Periodically do checkpoint in Pregel · 0a7f5f27
      ding authored
      ## What changes were proposed in this pull request?
      
      Pregel-based iterative algorithms with more than ~50 iterations begin to slow down and eventually fail with a StackOverflowError due to Spark's lack of support for long lineage chains.
      
      This PR causes Pregel to checkpoint the graph periodically if the checkpoint directory is set.
      This PR moves PeriodicGraphCheckpointer.scala from mllib to graphx, moves PeriodicRDDCheckpointer.scala, PeriodicCheckpointer.scala from mllib to core
      ## How was this patch tested?
      
      unit tests, manual tests
      (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
      
      (If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
      
      Author: ding <ding@localhost.localdomain>
      Author: dding3 <ding.ding@intel.com>
      Author: Michael Allman <michael@videoamp.com>
      
      Closes #15125 from dding3/cp2_pregel.
      0a7f5f27
    • Sergey Zhemzhitsky's avatar
      [SPARK-20404][CORE] Using Option(name) instead of Some(name) · 0bc7a902
      Sergey Zhemzhitsky authored
      Using Option(name) instead of Some(name) to prevent runtime failures when using accumulators created like the following
      ```
      sparkContext.accumulator(0, null)
      ```
      
      Author: Sergey Zhemzhitsky <szhemzhitski@gmail.com>
      
      Closes #17740 from szhem/SPARK-20404-null-acc-names.
      0bc7a902
  6. Apr 24, 2017
    • Josh Rosen's avatar
      [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT · f44c8a84
      Josh Rosen authored
      This patch bumps the master branch version to `2.3.0-SNAPSHOT`.
      
      Author: Josh Rosen <joshrosen@databricks.com>
      
      Closes #17753 from JoshRosen/SPARK-20453.
      f44c8a84
    • jerryshao's avatar
      [SPARK-20239][CORE] Improve HistoryServer's ACL mechanism · 5280d93e
      jerryshao authored
      ## What changes were proposed in this pull request?
      
      Current SHS (Spark History Server) two different ACLs:
      
      * ACL of base URL, it is controlled by "spark.acls.enabled" or "spark.ui.acls.enabled", and with this enabled, only user configured with "spark.admin.acls" (or group) or "spark.ui.view.acls" (or group), or the user who started SHS could list all the applications, otherwise none of them can be listed. This will also affect REST APIs which listing the summary of all apps and one app.
      * Per application ACL. This is controlled by "spark.history.ui.acls.enabled". With this enabled only history admin user and user/group who ran this app can access the details of this app.
      
      With this two ACLs, we may encounter several unexpected behaviors:
      
      1. if base URL's ACL (`spark.acls.enable`) is enabled but user A has no view permission. User "A" cannot see the app list but could still access details of it's own app.
      2. if ACLs of base URL (`spark.acls.enable`) is disabled, then user "A" could download any application's event log, even it is not run by user "A".
      3. The changes of Live UI's ACL will affect History UI's ACL which share the same conf file.
      
      The unexpected behaviors is mainly because we have two different ACLs, ideally we should have only one to manage all.
      
      So to improve SHS's ACL mechanism, here in this PR proposed to:
      
      1. Disable "spark.acls.enable" and only use "spark.history.ui.acls.enable" for history server.
      2. Check permission for event-log download REST API.
      
      With this PR:
      
      1. Admin user could see/download the list of all applications, as well as application details.
      2. Normal user could see the list of all applications, but can only download and check the details of applications accessible to him.
      
      ## How was this patch tested?
      
      New UTs are added, also verified in real cluster.
      
      CC tgravescs vanzin please help to review, this PR changes the semantics you did previously. Thanks a lot.
      
      Author: jerryshao <sshao@hortonworks.com>
      
      Closes #17582 from jerryshao/SPARK-20239.
      5280d93e
  7. Apr 23, 2017
    • 郭小龙 10207633's avatar
      [SPARK-20385][WEB-UI] Submitted Time' field, the date format needs to be... · 2eaf4f3f
      郭小龙 10207633 authored
      [SPARK-20385][WEB-UI] Submitted Time' field, the date format needs to be formatted, in running Drivers table or Completed Drivers table in master web ui.
      
      ## What changes were proposed in this pull request?
      Submitted Time' field, the date format **needs to be formatted**, in running Drivers table or Completed Drivers table in master web ui.
      Before fix this problem  e.g.
      
      Completed Drivers
      Submission ID	             **Submitted Time**  	             Worker	                            State	   Cores	   Memory	       Main Class
      driver-20170419145755-0005	 **Wed Apr 19 14:57:55 CST 2017**	 worker-20170419145250-zdh120-40412	FAILED	   1	       1024.0 MB	   cn.zte.HdfsTest
      
      please see the  attachment:https://issues.apache.org/jira/secure/attachment/12863977/before_fix.png
      
      After fix this problem e.g.
      
      Completed Drivers
      Submission ID	             **Submitted Time**  	             Worker	                            State	   Cores	   Memory	       Main Class
      driver-20170419145755-0006	 **2017/04/19 16:01:25**	 worker-20170419145250-zdh120-40412	         FAILED	   1	       1024.0 MB	   cn.zte.HdfsTest
      
      please see the  attachment:https://issues.apache.org/jira/secure/attachment/12863976/after_fix.png
      
      'Submitted Time' field, the date format **has been formatted**, in running Applications table or Completed Applicationstable in master web ui, **it is correct.**
      e.g.
      Running Applications
      Application ID	                Name	                Cores	Memory per Executor	   **Submitted Time**	      User	   State	        Duration
      app-20170419160910-0000 (kill)	SparkSQL::10.43.183.120	1	    5.0 GB	               **2017/04/19 16:09:10**	  root	   RUNNING	    53 s
      
      **Format after the time easier to observe, and consistent with the applications table,so I think it's worth fixing.**
      
      ## How was this patch tested?
      
      (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
      (If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
      Author: guoxiaolong <guo.xiaolong1@zte.com.cn>
      Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>
      
      Closes #17682 from guoxiaolongzte/SPARK-20385.
      2eaf4f3f
  8. Apr 22, 2017
    • eatoncys's avatar
      [SPARK-20386][SPARK CORE] modify the log info if the block exists on the slave already · 05a45149
      eatoncys authored
      ## What changes were proposed in this pull request?
      Modify the added memory size to memSize-originalMemSize if the  block exists on the slave already
      since if the  block exists, the added memory size should be memSize-originalMemSize; if originalMemSize is bigger than memSize ,then the log info should be Removed memory, removed size should be originalMemSize-memSize
      
      ## How was this patch tested?
      Multiple runs on existing unit tests
      
      (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
      (If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: eatoncys <chen.yanshan@zte.com.cn>
      
      Closes #17683 from eatoncys/SPARK-20386.
      05a45149
  9. Apr 20, 2017
    • jerryshao's avatar
      [SPARK-20172][CORE] Add file permission check when listing files in FsHistoryProvider · 592f5c89
      jerryshao authored
      ## What changes were proposed in this pull request?
      
      In the current Spark's HistoryServer we expected to get `AccessControlException` during listing all the files, but unfortunately it was not worked because we actually doesn't check the access permission and no other calls will throw such exception. What was worse is that this check will be deferred until reading files, which is not necessary and quite verbose, since it will be printed out the exception in every 10 seconds when checking the files.
      
      So here with this fix, we actually check the read permission during listing the files, which could avoid unnecessary file read later on and suppress the verbose log.
      
      ## How was this patch tested?
      
      Add unit test to verify.
      
      Author: jerryshao <sshao@hortonworks.com>
      
      Closes #17495 from jerryshao/SPARK-20172.
      592f5c89
    • Eric Liang's avatar
      [SPARK-20358][CORE] Executors failing stage on interrupted exception thrown by cancelled tasks · b2ebadfd
      Eric Liang authored
      ## What changes were proposed in this pull request?
      
      This was a regression introduced by my earlier PR here: https://github.com/apache/spark/pull/17531
      
      It turns out NonFatal() does not in fact catch InterruptedException.
      
      ## How was this patch tested?
      
      Extended cancellation unit test coverage. The first test fails before this patch.
      
      cc JoshRosen mridulm
      
      Author: Eric Liang <ekl@databricks.com>
      
      Closes #17659 from ericl/spark-20358.
      b2ebadfd
  10. Apr 18, 2017
    • 郭小龙 10207633's avatar
      [SPARK-20354][CORE][REST-API] When I request access to the 'http:... · 1f81dda3
      郭小龙 10207633 authored
      [SPARK-20354][CORE][REST-API] When I request access to the 'http: //ip:port/api/v1/applications' link, return 'sparkUser' is empty in REST API.
      
      ## What changes were proposed in this pull request?
      
      When I request access to the 'http: //ip:port/api/v1/applications' link, get the json. I need the 'sparkUser' field specific value, because my Spark big data management platform needs to filter through this field which user submits the application to facilitate my administration and query, but the current return of the json string is empty, causing me this Function can not be achieved, that is, I do not know who the specific application is submitted by this REST Api.
      
      **current return json:**
      [ {
        "id" : "app-20170417152053-0000",
        "name" : "KafkaWordCount",
        "attempts" : [ {
          "startTime" : "2017-04-17T07:20:51.395GMT",
          "endTime" : "1969-12-31T23:59:59.999GMT",
          "lastUpdated" : "2017-04-17T07:20:51.395GMT",
          "duration" : 0,
          **"sparkUser" : "",**
          "completed" : false,
          "endTimeEpoch" : -1,
          "startTimeEpoch" : 1492413651395,
          "lastUpdatedEpoch" : 1492413651395
        } ]
      } ]
      
      **When I fix this question, return json:**
      [ {
        "id" : "app-20170417154201-0000",
        "name" : "KafkaWordCount",
        "attempts" : [ {
          "startTime" : "2017-04-17T07:41:57.335GMT",
          "endTime" : "1969-12-31T23:59:59.999GMT",
          "lastUpdated" : "2017-04-17T07:41:57.335GMT",
          "duration" : 0,
          **"sparkUser" : "mr",**
          "completed" : false,
          "startTimeEpoch" : 1492414917335,
          "endTimeEpoch" : -1,
          "lastUpdatedEpoch" : 1492414917335
        } ]
      } ]
      
      ## How was this patch tested?
      
      manual tests
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
      Author: guoxiaolong <guo.xiaolong1@zte.com.cn>
      Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>
      
      Closes #17656 from guoxiaolongzte/SPARK-20354.
      1f81dda3
    • Robert Stupp's avatar
      [SPARK-20344][SCHEDULER] Duplicate call in FairSchedulableBuilder.addTaskSetManager · 07fd94e0
      Robert Stupp authored
      ## What changes were proposed in this pull request?
      
      Eliminate the duplicate call to `Pool.getSchedulableByName()` in `FairSchedulableBuilder.addTaskSetManager`
      
      ## How was this patch tested?
      
      ./dev/run-tests
      
      Author: Robert Stupp <snazy@snazy.de>
      
      Closes #17647 from snazy/20344-dup-call-master.
      07fd94e0
  11. Apr 16, 2017
  12. Apr 13, 2017
    • Sergei Lebedev's avatar
      [SPARK-20284][CORE] Make {Des,S}erializationStream extend Closeable · a4293c28
      Sergei Lebedev authored
      ## What changes were proposed in this pull request?
      
      This PR allows to use `SerializationStream` and `DeserializationStream` in try-with-resources.
      
      ## How was this patch tested?
      
      `core` unit tests.
      
      Author: Sergei Lebedev <s.lebedev@criteo.com>
      
      Closes #17598 from superbobry/compression-stream-closeable.
      a4293c28
  13. Apr 12, 2017
    • Shixiong Zhu's avatar
      [SPARK-20131][CORE] Don't use `this` lock in StandaloneSchedulerBackend.stop · c5f1cc37
      Shixiong Zhu authored
      ## What changes were proposed in this pull request?
      
      `o.a.s.streaming.StreamingContextSuite.SPARK-18560 Receiver data should be deserialized properly` is flaky is because there is a potential dead-lock in StandaloneSchedulerBackend which causes `await` timeout. Here is the related stack trace:
      ```
      "Thread-31" #211 daemon prio=5 os_prio=31 tid=0x00007fedd4808000 nid=0x16403 waiting on condition [0x00007000239b7000]
         java.lang.Thread.State: TIMED_WAITING (parking)
      	at sun.misc.Unsafe.park(Native Method)
      	- parking to wait for  <0x000000079b49ca10> (a scala.concurrent.impl.Promise$CompletionLatch)
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
      	at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:208)
      	at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:218)
      	at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223)
      	at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:201)
      	at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
      	at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:92)
      	at org.apache.spark.rpc.RpcEndpointRef.askSync(RpcEndpointRef.scala:76)
      	at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend.stop(CoarseGrainedSchedulerBackend.scala:402)
      	at org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend.org$apache$spark$scheduler$cluster$StandaloneSchedulerBackend$$stop(StandaloneSchedulerBackend.scala:213)
      	- locked <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend)
      	at org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend.stop(StandaloneSchedulerBackend.scala:116)
      	- locked <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend)
      	at org.apache.spark.scheduler.TaskSchedulerImpl.stop(TaskSchedulerImpl.scala:517)
      	at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1657)
      	at org.apache.spark.SparkContext$$anonfun$stop$8.apply$mcV$sp(SparkContext.scala:1921)
      	at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1302)
      	at org.apache.spark.SparkContext.stop(SparkContext.scala:1920)
      	at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:708)
      	at org.apache.spark.streaming.StreamingContextSuite$$anonfun$43$$anonfun$apply$mcV$sp$66$$anon$3.run(StreamingContextSuite.scala:827)
      
      "dispatcher-event-loop-3" #18 daemon prio=5 os_prio=31 tid=0x00007fedd603a000 nid=0x6203 waiting for monitor entry [0x0000700003be4000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint.org$apache$spark$scheduler$cluster$CoarseGrainedSchedulerBackend$DriverEndpoint$$makeOffers(CoarseGrainedSchedulerBackend.scala:253)
      	- waiting to lock <0x00000007066fca38> (a org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend)
      	at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anonfun$receive$1.applyOrElse(CoarseGrainedSchedulerBackend.scala:124)
      	at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117)
      	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205)
      	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101)
      	at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:213)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      ```
      
      This PR removes `synchronized` and changes `stopping` to AtomicBoolean to ensure idempotent to fix the dead-lock.
      
      ## How was this patch tested?
      
      Jenkins
      
      Author: Shixiong Zhu <shixiong@databricks.com>
      
      Closes #17610 from zsxwing/SPARK-20131.
      c5f1cc37
    • Wenchen Fan's avatar
      [SPARK-15354][FLAKY-TEST] TopologyAwareBlockReplicationPolicyBehavior.Peers in 2 racks · a7b430b5
      Wenchen Fan authored
      ## What changes were proposed in this pull request?
      
      `TopologyAwareBlockReplicationPolicyBehavior.Peers in 2 racks` is failing occasionally: https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.storage.TopologyAwareBlockReplicationPolicyBehavior&test_name=Peers+in+2+racks.
      
      This is because, when we generate 10 block manager id to test, they may all belong to the same rack, as the rack is randomly picked. This PR fixes this problem by forcing each rack to be picked at least once.
      
      ## How was this patch tested?
      
      N/A
      
      Author: Wenchen Fan <wenchen@databricks.com>
      
      Closes #17624 from cloud-fan/test.
      a7b430b5
    • hyukjinkwon's avatar
      [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkins · ceaf77ae
      hyukjinkwon authored
      ## What changes were proposed in this pull request?
      
      This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.
      
      There are several problems with it:
      
      - It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".
      
      - > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.
      
        (see  joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))
      
      To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.
      
      There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013
      
      Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.
      
      ## How was this patch tested?
      
      Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.
      
      This was tested via manually adding `time.time()` as below:
      
      ```diff
           profiles_and_goals = build_profiles + sbt_goals
      
           print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
                 " ".join(profiles_and_goals))
      
      +    import time
      +    st = time.time()
           exec_sbt(profiles_and_goals)
      +    print("Elapsed :[%s]" % str(time.time() - st))
      ```
      
      produces
      
      ```
      ...
      ========================================================================
      Building Unidoc API Documentation
      ========================================================================
      ...
      [info] Main Java API documentation successful.
      ...
      Elapsed :[94.8746569157]
      ...
      
      Author: hyukjinkwon <gurwls223@gmail.com>
      
      Closes #17477 from HyukjinKwon/SPARK-18692.
      ceaf77ae
  14. Apr 10, 2017
    • Sean Owen's avatar
      [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish... · a26e3ed5
      Sean Owen authored
      [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
      
      ## What changes were proposed in this pull request?
      
      Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").
      
      The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.
      
      ## How was this patch tested?
      
      Existing tests.
      
      Author: Sean Owen <sowen@cloudera.com>
      
      Closes #17527 from srowen/SPARK-20156.
      a26e3ed5
    • Bogdan Raducanu's avatar
      [SPARK-20243][TESTS] DebugFilesystem.assertNoOpenStreams thread race · 4f7d49b9
      Bogdan Raducanu authored
      ## What changes were proposed in this pull request?
      
      Synchronize access to openStreams map.
      
      ## How was this patch tested?
      
      Existing tests.
      
      Author: Bogdan Raducanu <bogdan@databricks.com>
      
      Closes #17592 from bogdanrdc/SPARK-20243.
      4f7d49b9
  15. Apr 09, 2017
  16. Apr 06, 2017
    • jerryshao's avatar
      [SPARK-17019][CORE] Expose on-heap and off-heap memory usage in various places · a4491626
      jerryshao authored
      ## What changes were proposed in this pull request?
      
      With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992), Spark supports persisting data into off-heap memory, but the usage of on-heap and off-heap memory is not exposed currently, it is not so convenient for user to monitor and profile, so here propose to expose off-heap memory as well as on-heap memory usage in various places:
      1. Spark UI's executor page will display both on-heap and off-heap memory usage.
      2. REST request returns both on-heap and off-heap memory.
      3. Also this can be gotten from MetricsSystem.
      4. Last this usage can be obtained programmatically from SparkListener.
      
      Attach the UI changes:
      
      ![screen shot 2016-08-12 at 11 20 44 am](https://cloud.githubusercontent.com/assets/850797/17612032/6c2f4480-607f-11e6-82e8-a27fb8cbb4ae.png)
      
      Backward compatibility is also considered for event-log and REST API. Old event log can still be replayed with off-heap usage displayed as 0. For REST API, only adds the new fields, so JSON backward compatibility can still be kept.
      ## How was this patch tested?
      
      Unit test added and manual verification.
      
      Author: jerryshao <sshao@hortonworks.com>
      
      Closes #14617 from jerryshao/SPARK-17019.
      a4491626
  17. Apr 05, 2017
    • Eric Liang's avatar
      [SPARK-20217][CORE] Executor should not fail stage if killed task throws non-interrupted exception · 5142e5d4
      Eric Liang authored
      ## What changes were proposed in this pull request?
      
      If tasks throw non-interrupted exceptions on kill (e.g. java.nio.channels.ClosedByInterruptException), their death is reported back as TaskFailed instead of TaskKilled. This causes stage failure in some cases.
      
      This is reproducible as follows. Run the following, and then use SparkContext.killTaskAttempt to kill one of the tasks. The entire stage will fail since we threw a RuntimeException instead of InterruptedException.
      
      ```
      spark.range(100).repartition(100).foreach { i =>
        try {
          Thread.sleep(10000000)
        } catch {
          case t: InterruptedException =>
            throw new RuntimeException(t)
        }
      }
      ```
      Based on the code in TaskSetManager, I think this also affects kills of speculative tasks. However, since the number of speculated tasks is few, and usually you need to fail a task a few times before the stage is cancelled, it unlikely this would be noticed in production unless both speculation was enabled and the num allowed task failures was = 1.
      
      We should probably unconditionally return TaskKilled instead of TaskFailed if the task was killed by the driver, regardless of the actual exception thrown.
      
      ## How was this patch tested?
      
      Unit test. The test fails before the change in Executor.scala
      
      cc JoshRosen
      
      Author: Eric Liang <ekl@databricks.com>
      
      Closes #17531 from ericl/fix-task-interrupt.
      5142e5d4
    • Dilip Biswal's avatar
      [SPARK-20204][SQL][FOLLOWUP] SQLConf should react to change in default timezone settings · 9d68c672
      Dilip Biswal authored
      ## What changes were proposed in this pull request?
      Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf.
      
      ## How was this patch tested?
      Added an unit test in ConfigEntrySuite
      
      Author: Dilip Biswal <dbiswal@us.ibm.com>
      
      Closes #17537 from dilipbiswal/timezone_debug.
      9d68c672
    • shaolinliu's avatar
      [SPARK-19807][WEB UI] Add reason for cancellation when a stage is killed using web UI · 71c3c481
      shaolinliu authored
      ## What changes were proposed in this pull request?
      
      When a user kills a stage using web UI (in Stages page), StagesTab.handleKillRequest requests SparkContext to cancel the stage without giving a reason. SparkContext has cancelStage(stageId: Int, reason: String) that Spark could use to pass the information for monitoring/debugging purposes.
      
      ## How was this patch tested?
      
      manual tests
      
      Please review http://spark.apache.org/contributing.html before opening a pull request.
      
      Author: shaolinliu <liu.shaolin1@zte.com.cn>
      Author: lvdongr <lv.dongdong@zte.com.cn>
      
      Closes #17258 from shaolinliu/SPARK-19807.
      71c3c481
    • Oliver Köth's avatar
      [SPARK-20042][WEB UI] Fix log page buttons for reverse proxy mode · 6f09dc70
      Oliver Köth authored
      with spark.ui.reverseProxy=true, full path URLs like /log will point to
      the master web endpoint which is serving the worker UI as reverse proxy.
      To access a REST endpoint in the worker in reverse proxy mode , the
      leading /proxy/"target"/ part of the base URI must be retained.
      
      Added logic to log-view.js to handle this, similar to executorspage.js
      
      Patch was tested manually
      
      Author: Oliver Köth <okoeth@de.ibm.com>
      
      Closes #17370 from okoethibm/master.
      6f09dc70
  18. Apr 03, 2017
    • Denis Bolshakov's avatar
      [SPARK-9002][CORE] KryoSerializer initialization does not include 'Array[Int]' · fb5869f2
      Denis Bolshakov authored
      [SPARK-9002][CORE] KryoSerializer initialization does not include 'Array[Int]'
      
      ## What changes were proposed in this pull request?
      
      Array[Int] has been registered in KryoSerializer.
      The following file has been changed
      core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
      
      ## How was this patch tested?
      
      First, the issue was reproduced by new unit test.
      Then, the issue was fixed to pass the failed test.
      
      Author: Denis Bolshakov <denis.bolshakov@onefactor.com>
      
      Closes #17482 from dbolshak/SPARK-9002.
      fb5869f2
  19. Mar 31, 2017
    • Ryan Blue's avatar
      [SPARK-20084][CORE] Remove internal.metrics.updatedBlockStatuses from history files. · c4c03eed
      Ryan Blue authored
      ## What changes were proposed in this pull request?
      
      Remove accumulator updates for internal.metrics.updatedBlockStatuses from SparkListenerTaskEnd entries in the history file. These can cause history files to grow to hundreds of GB because the value of the accumulator contains all tracked blocks.
      
      ## How was this patch tested?
      
      Current History UI tests cover use of the history file.
      
      Author: Ryan Blue <blue@apache.org>
      
      Closes #17412 from rdblue/SPARK-20084-remove-block-accumulator-info.
      c4c03eed
  20. Mar 30, 2017
Loading