You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by lianhuiwang <gi...@git.apache.org> on 2015/12/12 16:04:45 UTC

[GitHub] spark pull request: [SPARK-4621][shuffle]Shuffle index can cached ...

GitHub user lianhuiwang opened a pull request:

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

    [SPARK-4621][shuffle]Shuffle index can cached for SortShuffleManager in ExternalShuffle in order to  reduce indexFile's io

    In ExternalShuffle shuffle index can cached for SortShuffleManager at first. Next time another reduce's fetching can read index from cache. Because reduce tasks fetch map output from ExternalShuffle at same time, shuffle index cache is effective.
    In my test that it has 1000 partitions and 100 map tasks, it spends 3882ms for no cache index and it spends 608ms for cache index. So it is six times faster than index file.
    @rxin @JoshRosen @andrewor14 

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

    $ git pull https://github.com/lianhuiwang/spark SPARK-4621

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

    https://github.com/apache/spark/pull/10277.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 #10277
    
----
commit da3e330411c602f1d1816333cc7574083f5737f0
Author: Lianhui Wang <li...@gmail.com>
Date:   2015-12-12T14:42:01Z

    init commit

----


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165132326
  
    **[Test build #47813 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47813/consoleFull)** for PR 10277 at commit [`636f080`](https://github.com/apache/spark/commit/636f080823b67890586aabf9384eae0463859a99).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165124020
  
    **[Test build #47815 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47815/consoleFull)** for PR 10277 at commit [`eb52c89`](https://github.com/apache/spark/commit/eb52c89eb16a1ede44291cd47a43ceab689f63fb).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165123474
  
    **[Test build #47815 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47815/consoleFull)** for PR 10277 at commit [`eb52c89`](https://github.com/apache/spark/commit/eb52c89eb16a1ede44291cd47a43ceab689f63fb).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165024307
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165044198
  
    **[Test build #47804 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47804/consoleFull)** for PR 10277 at commit [`b48f1b1`](https://github.com/apache/spark/commit/b48f1b1eeed7609cd5a355aa7b4c53c305ad2d58).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `trait JobSubmitter `\n  * `class ComplexFutureAction[T](run : JobSubmitter => Future[T])`\n  * `  case class AttachCompletedRebuildUI(appId: String)`\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n  * `case class WrapOption(child: Expression, optType: DataType)`\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165221923
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/47827/
    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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164160691
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164576639
  
    @lianhuiwang can you make this pass tests first?


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165132422
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165221559
  
    **[Test build #47827 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47827/consoleFull)** for PR 10277 at commit [`3485e0f`](https://github.com/apache/spark/commit/3485e0f6680732bd055789d68f15fc3189ab54ad).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164577982
  
    Also why not do it for the non-external case as well?


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165221920
  
    Merged build finished. 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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165017467
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165017265
  
    **[Test build #47796 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47796/consoleFull)** for PR 10277 at commit [`a647fc9`](https://github.com/apache/spark/commit/a647fc9a427fd0211479b5b1864eae5dd5836a6a).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165172256
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165172249
  
    **[Test build #47818 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47818/consoleFull)** for PR 10277 at commit [`b30943a`](https://github.com/apache/spark/commit/b30943a828599a3016ab4d4c69bf53aa0562a22b).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165044304
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#discussion_r47566753
  
    --- Diff: network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexRecord.java ---
    @@ -0,0 +1,29 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.network.shuffle;
    +
    +/** The offsets of a single map output file for a reduceId */
    +public class ShuffleIndexRecord {
    +  public long offset;
    +  public long nextOffset;
    +
    +  public ShuffleIndexRecord(long offset, long nextOffset) {
    +    this.offset = offset;
    +    this.nextOffset = nextOffset;
    +  }
    +}
    --- End diff --
    
    need new line after


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165124025
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#discussion_r47566839
  
    --- Diff: network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexCache.java ---
    @@ -0,0 +1,251 @@
    +package org.apache.spark.network.shuffle;
    +
    +import java.io.DataInputStream;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +import java.nio.ByteBuffer;
    +import java.nio.LongBuffer;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Objects;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
    +import org.apache.spark.network.util.JavaUtils;
    +
    +/**
    + * Store the offsets of the data blocks in cache.
    + * When index cache is not enough, remove firstly used index information.
    + */
    +public class ShuffleIndexCache {
    +  private static final Logger logger = LoggerFactory.getLogger(ShuffleIndexCache.class);
    +
    +  private final ConcurrentMap<ShuffleMapId, IndexInformation> indexCache;
    +  private final LinkedBlockingQueue<ShuffleMapId> queue = new LinkedBlockingQueue<ShuffleMapId>();
    +  private final int totalMemoryAllowed;
    +  private AtomicInteger totalMemoryUsed = new AtomicInteger();
    +
    +  public ShuffleIndexCache(int totalMemoryAllowed) {
    +    this.indexCache = new ConcurrentHashMap<ShuffleMapId, IndexInformation>();
    +    this.totalMemoryAllowed = totalMemoryAllowed;
    +    logger.info("IndexCache created with max memory = {}", totalMemoryAllowed);
    +  }
    +
    +  /**
    +   * Get the index information for the given shuffleId, mapId and reduceId.
    +   * It reads the index file into cache if it is not already present.
    +   */
    +  public ShuffleIndexRecord getIndexInformation(
    +    ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) throws IOException {
    +    if (totalMemoryAllowed > 0) {
    +      ShuffleMapId shuffleMapId = new ShuffleMapId(shuffleId, mapId);
    +      IndexInformation info = indexCache.get(shuffleMapId);
    +
    +      if (info == null) {
    +        info = readIndexFileToCache(executor, shuffleMapId);
    +      } else {
    +        synchronized(info) {
    +          while (isUnderConstruction(info)) {
    +            try {
    +              info.wait();
    +            } catch (InterruptedException e) {
    +              throw new IOException("Interrupted waiting for construction", e);
    +            }
    +          }
    +        }
    +      }
    +
    +      if(info.getLength() == 0 || info.getLength() <= reduceId + 1) {
    +        throw new IOException("Invalid request " + " shuffleMapId = " + shuffleMapId +
    +          " reduceId = " + reduceId + " Index Info Length = " + info.getLength() +
    +            " index file = " + getIndexFile(executor, mapId, reduceId));
    +      }
    +
    +      return info.getIndex(reduceId);
    +    } else {
    +      return this.readIndexFile(executor, shuffleId, mapId, reduceId);
    +    }
    +  }
    +
    +  public ShuffleIndexRecord readIndexFile(
    +    ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) throws IOException {
    --- End diff --
    
    style:
    ```
    public ShuffleIndexRecord readIndexFile(
        ExecutorShuffleInfo executor,
        int shuffleId,
        int mapId,
        int reduceId) throws IOException {
      ...
    }
    ```


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164160666
  
    **[Test build #47612 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47612/consoleFull)** for PR 10277 at commit [`da3e330`](https://github.com/apache/spark/commit/da3e330411c602f1d1816333cc7574083f5737f0).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165017464
  
    **[Test build #47796 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47796/consoleFull)** for PR 10277 at commit [`a647fc9`](https://github.com/apache/spark/commit/a647fc9a427fd0211479b5b1864eae5dd5836a6a).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164947540
  
    actually, I would do it in a separate PR. In general it's good to keep these small.


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165171843
  
    **[Test build #47818 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47818/consoleFull)** for PR 10277 at commit [`b30943a`](https://github.com/apache/spark/commit/b30943a828599a3016ab4d4c69bf53aa0562a22b).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#discussion_r47566732
  
    --- Diff: network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexCache.java ---
    @@ -0,0 +1,251 @@
    +package org.apache.spark.network.shuffle;
    --- End diff --
    
    need license header


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165014864
  
    Merged build finished. Test FAILed.


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165024082
  
    **[Test build #47800 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47800/consoleFull)** for PR 10277 at commit [`76e46ea`](https://github.com/apache/spark/commit/76e46ead682a7a2aa8e53228083746f7c13c0469).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165024305
  
    **[Test build #47800 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47800/consoleFull)** for PR 10277 at commit [`76e46ea`](https://github.com/apache/spark/commit/76e46ead682a7a2aa8e53228083746f7c13c0469).
     * This patch **fails Java style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165185707
  
    **[Test build #47827 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47827/consoleFull)** for PR 10277 at commit [`3485e0f`](https://github.com/apache/spark/commit/3485e0f6680732bd055789d68f15fc3189ab54ad).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165016535
  
    @andrewor14 I have update this PR with non-external shuffle.
    Now use two configurations: 
    spark.shuffle.index.cacheSize is for non-external, spark.shuffle.service.indexCacheSize is for external. 
    I am not sure they can be the same one configuration, because their size is different. So now i use two different configuration.


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165026785
  
    **[Test build #47804 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47804/consoleFull)** for PR 10277 at commit [`b48f1b1`](https://github.com/apache/spark/commit/b48f1b1eeed7609cd5a355aa7b4c53c305ad2d58).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164160688
  
    **[Test build #47612 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47612/consoleFull)** for PR 10277 at commit [`da3e330`](https://github.com/apache/spark/commit/da3e330411c602f1d1816333cc7574083f5737f0).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `public class ShuffleIndexCache `\n  * `public class ShuffleIndexRecord `\n


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165182463
  
    Now i set cacheSize=0 in ExternalShuffleServiceSuite. because every unit test use the same shuffleId at the same time with multi-threads.


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-164631642
  
    @andrewor14 thanks. because now we only use external shuffle. but it does not matter. latter i will add non-external in this PR. 


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

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


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165097715
  
    **[Test build #47813 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/47813/consoleFull)** for PR 10277 at commit [`636f080`](https://github.com/apache/spark/commit/636f080823b67890586aabf9384eae0463859a99).


---
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-4621][shuffle]Shuffle index can be cach...

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

    https://github.com/apache/spark/pull/10277#issuecomment-165015427
  
    Merged build finished. Test FAILed.


---
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