You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by zsxwing <gi...@git.apache.org> on 2014/12/17 10:56:30 UTC

[GitHub] spark pull request: WriteAheadLogBasedBlockHandler improvement

GitHub user zsxwing opened a pull request:

    https://github.com/apache/spark/pull/3721

    WriteAheadLogBasedBlockHandler improvement

    1. Move the block store work to a separate thread pool. IO work usually runs longer than CPU work. Running IO work in `Implicits.global` may block other work that want to run in `Implicits.global`.
    2. Use `Future.zip` instead of `Future.flatMap`(for-loop). `zip` implies these two Futures will run concurrently, while `flatMap` usually means one Future depends on the other one.
    3. Fix a unit test in `ReceivedBlockTrackerSuite` which may fail occasionally.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/zsxwing/spark SPARK-4873

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/3721.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3721
    
----
commit 860f3a2b1d30f4640e1be16cae9609bb5e394c40
Author: zsxwing <zs...@gmail.com>
Date:   2014-12-17T09:54:53Z

    WriteAheadLogBasedBlockHandler improvement

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67311555
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24543/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67434030
  
      [Test build #24566 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24566/consoleFull) for   PR 3721 at commit [`d3d8a51`](https://github.com/apache/spark/commit/d3d8a5157cbded6a39b39a0b65d6fff90c0968d9).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22263897
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala ---
    @@ -114,6 +114,8 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
         clock: Clock = new SystemClock
       ) extends ReceivedBlockHandler with Logging {
     
    +  import WriteAheadLogBasedBlockHandler.storeBlockExecutor
    --- End diff --
    
    There is already an `executionContext` implicitly available in this class (line 134-136). Why another context?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67433896
  
    Used `Future` to fix the test and override my previous commits.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22063405
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala ---
    @@ -125,7 +125,7 @@ private[streaming] class WriteAheadLogManager(
        * between the node calculating the threshTime (say, driver node), and the local system time
        * (say, worker node), the caller has to take account of possible time skew.
        */
    -  def cleanupOldLogs(threshTime: Long): Unit = {
    +  def cleanupOldLogs(threshTime: Long): Future[Unit] = {
    --- End diff --
    
    The thing is that deleting asynchronously is an implementation detail - which we choose to do now, which can be changed later, don't you think? If we change it later, we'd have to change this method's signature - which can cause pain if there is code that uses Await.result(*) on this future. 
    
    If we expose it via a parameter, we can choose to ignore the param and still the calling code will not have to change. Since it is unlikely that the calling code will actually depend on the async nature, it is unlikely to see any difference in functionality and no change in code is required.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22027430
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala ---
    @@ -125,7 +125,7 @@ private[streaming] class WriteAheadLogManager(
        * between the node calculating the threshTime (say, driver node), and the local system time
        * (say, worker node), the caller has to take account of possible time skew.
        */
    -  def cleanupOldLogs(threshTime: Long): Unit = {
    +  def cleanupOldLogs(threshTime: Long): Future[Unit] = {
    --- End diff --
    
    IMO, I prefer to also return `Future` for an asynchronous action. Returning `Unit` hides the asynchronous feature and such method will be misused easily.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22265855
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala ---
    @@ -114,6 +114,8 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
         clock: Clock = new SystemClock
       ) extends ReceivedBlockHandler with Logging {
     
    +  import WriteAheadLogBasedBlockHandler.storeBlockExecutor
    --- End diff --
    
    Sorry, I didn't see it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22243660
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala ---
    @@ -169,10 +171,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
         }
     
         // Combine the futures, wait for both to complete, and return the write ahead log segment
    -    val combinedFuture = for {
    --- End diff --
    
    Very good catch. I wasnt aware that `for-yield` is `flatMap` which is obviously ordered. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68082183
  
    LGTM. Will merge if tests pass. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: WriteAheadLogBasedBlockHandler improvement

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67303062
  
      [Test build #24543 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24543/consoleFull) for   PR 3721 at commit [`8b6d095`](https://github.com/apache/spark/commit/8b6d0952c1333d4d57fee2777e68f4f72c755d2e).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67400254
  
    Hmm, looking at it again - this would fix the test as well, though I think the approach in #3726 is cleaner.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67309187
  
      [Test build #24542 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24542/consoleFull) for   PR 3721 at commit [`860f3a2`](https://github.com/apache/spark/commit/860f3a2b1d30f4640e1be16cae9609bb5e394c40).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68084624
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24799/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22027214
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala ---
    @@ -125,7 +125,7 @@ private[streaming] class WriteAheadLogManager(
        * between the node calculating the threshTime (say, driver node), and the local system time
        * (say, worker node), the caller has to take account of possible time skew.
        */
    -  def cleanupOldLogs(threshTime: Long): Unit = {
    +  def cleanupOldLogs(threshTime: Long): Future[Unit] = {
    --- End diff --
    
    I don't think we should be exposing this future. That is internal implementation detail and we'd have to stick with this implementation. At some point, we might want to delete the files synchronously - at which point returning this Future might not make sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68076185
  
    High level comment, I think I completely agree on the Future.zip, but not sure if I find the other changed related to other two valid. 
    - There is already a separate execution context implicitly available. 
    - The flaky test, I think the solution in #3726 is closer to the ideal as changing the signature to return `Future`  exposes unnecessary internal details. 
    
    So it would be great if you can reduce the scope of this PR to the `Future.zip` and accordingly update the title or the PR + JIRA to a more specific than "improvements".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/3721


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67311551
  
      [Test build #24543 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24543/consoleFull) for   PR 3721 at commit [`8b6d095`](https://github.com/apache/spark/commit/8b6d0952c1333d4d57fee2777e68f4f72c755d2e).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `trait ParquetTest `
      * `protected class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, String] `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: WriteAheadLogBasedBlockHandler improvement

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67300181
  
      [Test build #24542 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24542/consoleFull) for   PR 3721 at commit [`860f3a2`](https://github.com/apache/spark/commit/860f3a2b1d30f4640e1be16cae9609bb5e394c40).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67439083
  
      [Test build #24566 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24566/consoleFull) for   PR 3721 at commit [`d3d8a51`](https://github.com/apache/spark/commit/d3d8a5157cbded6a39b39a0b65d6fff90c0968d9).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68085654
  
    Merged this, thanks very much!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22091804
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala ---
    @@ -125,7 +125,7 @@ private[streaming] class WriteAheadLogManager(
        * between the node calculating the threshTime (say, driver node), and the local system time
        * (say, worker node), the caller has to take account of possible time skew.
        */
    -  def cleanupOldLogs(threshTime: Long): Unit = {
    +  def cleanupOldLogs(threshTime: Long): Future[Unit] = {
    --- End diff --
    
    > The thing is that deleting asynchronously is an implementation detail - which we choose to do now, which can be changed later, don't you think? If we change it later, we'd have to change this method's signature - which can cause pain if there is code that uses Await.result(*) on this future.
    
    I doubt if it will be changed. However, `asynchronously` is an important implementation detail that the caller should know it, or they may misuse it.
    
    > If we expose it via a parameter, we can choose to ignore the param and still the calling code will not have to change. Since it is unlikely that the calling code will actually depend on the async nature, it is unlikely to see any difference in functionality and no change in code is required.
    
    I don't think a parameter is enough. At least, it needs a more parameter, a `timeout` parameter. In your PR, you used `1 second` which may not be enough.
    
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67309192
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24542/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67430734
  
    My first trial was making `cleanupOldBatches` return a Future so that the caller can use it to wait. But I looked other places and found they used `eventually`. So I followed their approach. But IMO it'


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68082098
  
    Now this PR only contains `Future.zip` change. I also updated the PR & JIRA.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67392489
  
    The flaky test fix here looks like it addresses https://issues.apache.org/jira/browse/SPARK-4790.
    
    /cc @harishreedharan, who is investigating that flaky test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68084622
  
      [Test build #24799 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24799/consoleFull) for   PR 3721 at commit [`46a2cd9`](https://github.com/apache/spark/commit/46a2cd927eacfc9367e7a01a7189a7d3de80737f).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/3721#discussion_r22091865
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala ---
    @@ -125,7 +125,7 @@ private[streaming] class WriteAheadLogManager(
        * between the node calculating the threshTime (say, driver node), and the local system time
        * (say, worker node), the caller has to take account of possible time skew.
        */
    -  def cleanupOldLogs(threshTime: Long): Unit = {
    +  def cleanupOldLogs(threshTime: Long): Future[Unit] = {
    --- End diff --
    
    What's more, if we really want to change it to a asynchronously deleting. Returning Future does still work. Just simply writing something like:
    ```scala
    deleteFiles()
    return Promise[Unit]().success(null).future
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67398023
  
    The eventually ensures that the block is retried - but the failure is happening before the eventually, so the test would still throw. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] Use `Future.zip` inste...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-68082135
  
      [Test build #24799 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/24799/consoleFull) for   PR 3721 at commit [`46a2cd9`](https://github.com/apache/spark/commit/46a2cd927eacfc9367e7a01a7189a7d3de80737f).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67439090
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/24566/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4873][Streaming] WriteAheadLogBasedBloc...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/3721#issuecomment-67397850
  
    This would not address the test issue as we are still returning from cleanupOldLogs before the deletion is completed. This `tracker3.cleanupOldBatches(batchTime2)` will still end up hitting an exception if the deletion is not done in between the listFiles and getFileStatus (for the child file) calls in the getFileStatus method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org