You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/02/26 22:59:17 UTC

[GitHub] [spark] gengliangwang opened a new pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

gengliangwang opened a new pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716
 
 
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   Spark uses the class `InMemoryStore` as the KV storage for live UI and history server(by default if no LevelDB file path is provided).
   In `InMemoryStore`, all the task data in one application is stored in a hashmap, which key is the task ID and the value is the task data. This fine for getting or deleting with a provided task ID.
   However, Spark stage UI always shows all the task data in one stage and the current implementation is to look up all the values in the hashmap. The time complexity is O(numOfTasks).
   Also, when there are too many stages (>spark.ui.retainedStages), Spark will linearly try to look up all the task data of the stages to be deleted as well.
   
   This can be very bad for a large application with many stages and tasks. We can improve it by allowing the natural key of an entity to have a real parent index. So that on each lookup with parent node provided, Spark can look up all the natural keys(in our case, the task IDs) first, and then find the data with the natural keys in the hashmap.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   The in-memory KV store becomes really slow for large applications. We can improve it with a new index. The performance can be 10 times, 100 times, even 1000 times faster.
   
   ### Does this PR introduce any user-facing change?
   <!--
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If no, write 'No'.
   -->
   No
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Existing unit tests.
   Also, I run a benchmark with the following code
   ```
     val store = new InMemoryStore()
     val numberOfTasksPerStage = 10000
      (0 until 1000).map { sId =>
        (0 until numberOfTasksPerStage).map { taskId =>
          val task = newTaskData(sId * numberOfTasksPerStage + taskId, "SUCCESS", sId)
          store.write(task)
        }
      }
     val appStatusStore = new AppStatusStore(store)
     var start = System.nanoTime()
     appStatusStore.taskSummary(2, attemptId, Array(0, 0.25, 0.5, 0.75, 1))
     println("task summary run time: " + ((System.nanoTime() - start) / 1000000))
     val stageIds = Seq(1, 11, 66, 88)
     val stageKeys = stageIds.map(Array(_, attemptId))
     start = System.nanoTime()
     store.removeAllByIndexValues(classOf[TaskDataWrapper], TaskIndexNames.STAGE,
       stageKeys.asJavaCollection)
      println("clean up tasks run time: " + ((System.nanoTime() - start) / 1000000))
   ```
   
   Before the changes: task summary takes 98642ms, cleaning up tasks 4900ms
   After the changes: task summary takes 120ms, cleaning up tasks 4ms
   It's 800x faster after the changes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592324415
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23813/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592923045
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23861/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592923045
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23861/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592871462
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan closed pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592871469
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23858/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592659479
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23843/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592416373
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23826/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592397821
 
 
   **[Test build #119068 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119068/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591788436
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385990510
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -230,18 +261,31 @@ public T get(Object key) {
 
     public void put(T value) throws Exception {
       data.put(asKey(naturalKey.get(value)), value);
+      if (!naturalParentIndexName.isEmpty()) {
+        Comparable<Object> parentKey = asKey(getIndexAccessor(naturalParentIndexName).get(value));
+        NaturalKeys children =
+          parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+        children.put(asKey(naturalKey.get(value)), true);
+      }
     }
 
     public void delete(Object key) {
       data.remove(asKey(key));
+      if (!naturalParentIndexName.isEmpty()) {
+        for (NaturalKeys v : parentToChildrenMap.values()) {
+          if (v.remove(asKey(key))) {
+            break;
+          }
+        }
 
 Review comment:
   Oh, I meant after `v.remove(asKey(key))`, if `v` is empty, can  we remove the  (parent key, empty NaturalKeys) from `parentToChildrenMap`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592237424
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119051/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385179549
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
 
 Review comment:
   can `naturalParentIndexName` be null?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592398351
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119068/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592472046
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385366184
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
+        int count = 0;
+        for (Object indexValue : indexValues) {
+          Comparable<Object> parentKey = asKey(indexValue);
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.remove(naturalKey);
 
 Review comment:
    `data.forEach(callback)` is O(n) scan and there is a filter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591725101
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591697403
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23742/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591701759
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23743/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591765542
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118996/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067901
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385194333
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -163,6 +163,12 @@ public void clear() {
     }
   }
 
+  /**
+   * An alias class for the type "ConcurrentHashMap<Comparable<Object>, Boolean>", which is used
+   * as a concurrent hashset for storing natural keys.
 
 Review comment:
   `... storing natural keys, and the boolean value doesn't matter.`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591776629
 
 
   **[Test build #118999 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118999/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591693884
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593055184
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385986029
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -230,18 +261,31 @@ public T get(Object key) {
 
     public void put(T value) throws Exception {
       data.put(asKey(naturalKey.get(value)), value);
+      if (!naturalParentIndexName.isEmpty()) {
+        Comparable<Object> parentKey = asKey(getIndexAccessor(naturalParentIndexName).get(value));
+        NaturalKeys children =
+          parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+        children.put(asKey(naturalKey.get(value)), true);
+      }
     }
 
     public void delete(Object key) {
       data.remove(asKey(key));
+      if (!naturalParentIndexName.isEmpty()) {
+        for (NaturalKeys v : parentToChildrenMap.values()) {
+          if (v.remove(asKey(key))) {
+            break;
+          }
+        }
 
 Review comment:
   Yes, nothing will change if the NaturalKeys `v` doesn't contain `key`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592958956
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591744577
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118997/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591827239
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591743564
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593040372
 
 
   **[Test build #119134 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119134/testReport)** for PR 27716 at commit [`c0d3755`](https://github.com/apache/spark/commit/c0d375535d4cc20f8fd18a011fc3165da1ab2b7e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591752428
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592917825
 
 
   **[Test build #119116 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119116/testReport)** for PR 27716 at commit [`e02acc0`](https://github.com/apache/spark/commit/e02acc058d78eb1e074a018bda45539941540632).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591693890
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118993/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591699413
 
 
   **[Test build #118994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118994/testReport)** for PR 27716 at commit [`c6c2c82`](https://github.com/apache/spark/commit/c6c2c8240393da7723a5ca5c009e0e64fd74ac2f).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591836410
 
 
   **[Test build #119015 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119015/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591752192
 
 
   **[Test build #119005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119005/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385497150
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
+        int count = 0;
+        for (Object indexValue : indexValues) {
+          Comparable<Object> parentKey = asKey(indexValue);
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.remove(naturalKey);
 
 Review comment:
   then why the else branch can't use `data.remove`? Can we add some comments to highlight the difference between the if and else branches?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593055185
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119134/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993332
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23872/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592179859
 
 
   **[Test build #119051 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119051/testReport)** for PR 27716 at commit [`2ec82df`](https://github.com/apache/spark/commit/2ec82df4fa5365e6b41e80c8719062a90af6c66e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592180392
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592236784
 
 
   **[Test build #119051 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119051/testReport)** for PR 27716 at commit [`2ec82df`](https://github.com/apache/spark/commit/2ec82df4fa5365e6b41e80c8719062a90af6c66e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593055185
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119134/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385497409
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +370,28 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          ArrayList<T> elements = new ArrayList<>();
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.computeIfPresent(naturalKey, (k, v) -> {
 
 Review comment:
   ditto, let's add comment to explain why we need to do linear scan in the else branch.
   
   So linear scan can happen if an index has a parent but the parent is not indexed?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385989150
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +374,32 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
 
 Review comment:
   Is it possible that `naturalParentIndexName` doesn't equal to `ti.getParentIndexName(index)`? Isn't `String index = KVIndex.NATURAL_INDEX_NAME`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591744164
 
 
   **[Test build #118997 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118997/testReport)** for PR 27716 at commit [`d50c801`](https://github.com/apache/spark/commit/d50c801b6ca7106a49aa9f6819836af4174d86ef).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591708044
 
 
   **[Test build #118997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118997/testReport)** for PR 27716 at commit [`d50c801`](https://github.com/apache/spark/commit/d50c801b6ca7106a49aa9f6819836af4174d86ef).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591777152
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118999/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591931088
 
 
   **[Test build #119018 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119018/testReport)** for PR 27716 at commit [`753a14a`](https://github.com/apache/spark/commit/753a14a699761b41bc2c8059f101c1c68fc54100).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591690003
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993852
 
 
   **[Test build #119130 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119130/testReport)** for PR 27716 at commit [`491e9eb`](https://github.com/apache/spark/commit/491e9eba32dfe6d9075115af5010691855c82b24).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592738402
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592416369
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591752428
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067903
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23882/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592737576
 
 
   **[Test build #119099 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119099/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592659479
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23843/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593049000
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119130/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591744570
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591725111
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23746/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591689534
 
 
   **[Test build #118993 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118993/testReport)** for PR 27716 at commit [`4f93ffc`](https://github.com/apache/spark/commit/4f93ffcb55a2e79211a5d600b110bf652f95d0c5).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591705625
 
 
   **[Test build #118996 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118996/testReport)** for PR 27716 at commit [`8969a4d`](https://github.com/apache/spark/commit/8969a4dffc6fe9f54d7de34bee1e72fbf5c6f16d).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592738412
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119099/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591758835
 
 
   **[Test build #118994 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118994/testReport)** for PR 27716 at commit [`c6c2c82`](https://github.com/apache/spark/commit/c6c2c8240393da7723a5ca5c009e0e64fd74ac2f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591724726
 
 
   **[Test build #118998 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118998/testReport)** for PR 27716 at commit [`9cb44c3`](https://github.com/apache/spark/commit/9cb44c32c49ef7ea782e47426ce0b45bbf8238d7).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591690014
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23741/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591836479
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119015/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591932067
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119018/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385999609
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -230,18 +261,31 @@ public T get(Object key) {
 
     public void put(T value) throws Exception {
       data.put(asKey(naturalKey.get(value)), value);
+      if (!naturalParentIndexName.isEmpty()) {
+        Comparable<Object> parentKey = asKey(getIndexAccessor(naturalParentIndexName).get(value));
+        NaturalKeys children =
+          parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+        children.put(asKey(naturalKey.get(value)), true);
+      }
     }
 
     public void delete(Object key) {
       data.remove(asKey(key));
+      if (!naturalParentIndexName.isEmpty()) {
+        for (NaturalKeys v : parentToChildrenMap.values()) {
+          if (v.remove(asKey(key))) {
+            break;
+          }
+        }
 
 Review comment:
   Well,  `parentToChildrenMap` is a concurrent map and checking emptiness costs time. 
   The method here is to delete one entry. I think we can make it simple and keep it this way.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591690014
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23741/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593048827
 
 
   **[Test build #119130 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119130/testReport)** for PR 27716 at commit [`491e9eb`](https://github.com/apache/spark/commit/491e9eba32dfe6d9075115af5010691855c82b24).
    * This patch **fails from timeout after a configured wait of `400m`**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592324203
 
 
   **[Test build #119068 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119068/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592398342
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591846924
 
 
   **[Test build #119018 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119018/testReport)** for PR 27716 at commit [`753a14a`](https://github.com/apache/spark/commit/753a14a699761b41bc2c8059f101c1c68fc54100).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591705988
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23744/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591765542
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118996/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593055184
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591765538
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591788161
 
 
   **[Test build #119005 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119005/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591708337
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592415850
 
 
   **[Test build #119082 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119082/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592870997
 
 
   **[Test build #119116 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119116/testReport)** for PR 27716 at commit [`e02acc0`](https://github.com/apache/spark/commit/e02acc058d78eb1e074a018bda45539941540632).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591701420
 
 
   **[Test build #118995 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118995/testReport)** for PR 27716 at commit [`b0bb448`](https://github.com/apache/spark/commit/b0bb4484b0b84bfb4027472795acf430f6cf3757).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593048998
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591729213
 
 
   **[Test build #118999 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118999/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591826830
 
 
   **[Test build #119015 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119015/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591836479
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119015/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592871469
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23858/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591697403
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23742/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591788440
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119005/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591844205
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592398342
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591693890
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118993/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592324415
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23813/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591742923
 
 
   **[Test build #118995 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118995/testReport)** for PR 27716 at commit [`b0bb448`](https://github.com/apache/spark/commit/b0bb4484b0b84bfb4027472795acf430f6cf3757).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591764366
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118998/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592180392
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592237418
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591764366
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118998/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591764356
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591699413
 
 
   **[Test build #118994 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118994/testReport)** for PR 27716 at commit [`c6c2c82`](https://github.com/apache/spark/commit/c6c2c8240393da7723a5ca5c009e0e64fd74ac2f).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591701756
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067501
 
 
   retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592180399
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23797/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591752434
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23752/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592738402
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592472052
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119082/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591690003
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591729597
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23747/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592658871
 
 
   **[Test build #119099 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119099/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591743572
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118995/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591844205
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592658871
 
 
   **[Test build #119099 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119099/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593266155
 
 
   linear scan all the tasks data to look up only one stage looks like a performance issue to me, and we should fix it in 3.0 as well.
   
   Thanks, merging to master/3.0!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592416369
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385988906
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +374,32 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          // If there is a parent index for the natural index and the parent of`index` happens to be
 
 Review comment:
   ``` of`index` ```  -> ``` of `index` ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591725101
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591788440
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119005/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593041315
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591697393
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591759545
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592324410
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385187715
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
+        int count = 0;
+        for (Object indexValue : indexValues) {
+          Comparable<Object> parentKey = asKey(indexValue);
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
 
 Review comment:
   why do we need to new a `NaturalKeys` here? If `parentKey` doesn't exist we can just do `parentToChildrenMap.remove(parentKey)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591827249
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23762/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592917931
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119116/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592917926
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385356791
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
 
 Review comment:
   no, the default value is `""`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591932067
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119018/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385512118
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
+        int count = 0;
+        for (Object indexValue : indexValues) {
+          Comparable<Object> parentKey = asKey(indexValue);
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.remove(naturalKey);
 
 Review comment:
   ok, let me add comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593041327
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23876/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592922624
 
 
   retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592871462
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592917926
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591705978
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592472046
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591701756
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592922884
 
 
   **[Test build #119119 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119119/testReport)** for PR 27716 at commit [`491e9eb`](https://github.com/apache/spark/commit/491e9eba32dfe6d9075115af5010691855c82b24).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591729587
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592398351
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119068/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592179859
 
 
   **[Test build #119051 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119051/testReport)** for PR 27716 at commit [`2ec82df`](https://github.com/apache/spark/commit/2ec82df4fa5365e6b41e80c8719062a90af6c66e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591693884
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593080918
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591729587
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592958960
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119119/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591752434
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23752/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593080921
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119140/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993332
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23872/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591764096
 
 
   **[Test build #118998 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118998/testReport)** for PR 27716 at commit [`9cb44c3`](https://github.com/apache/spark/commit/9cb44c32c49ef7ea782e47426ce0b45bbf8238d7).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385199591
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala
 ##########
 @@ -78,6 +82,26 @@ class AppStatusStoreSuite extends SparkFunSuite {
     assert(store.count(classOf[CachedQuantile]) === 2)
   }
 
+  test("InMemoryStore should build index from Stage ID to Task data") {
+    val store = new InMemoryStore()
+    (0 until 1000).map { sId =>
+       (0 until 1000).map { taskId =>
+         val task = newTaskData(sId * 1000 + taskId, "SUCCESS", sId)
+         store.write(task)
+       }
+    }
+    val appStatusStore = new AppStatusStore(store)
+    failAfter(Span(200, Millis)) {
+      appStatusStore.taskSummary(1, attemptId, Array(0, 0.25, 0.5, 0.75, 1))
+    }
+    val stageIds = Seq(1, 11, 66, 88)
+    val stageKeys = stageIds.map(Array(_, attemptId))
+    failAfter(Span(10, Millis)) {
 
 Review comment:
   up to my experience, this can be flaky as it assumes some code should be executed within some time. I think the benchmark result in PR description is good enough.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592413718
 
 
   retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591825428
 
 
   retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591697393
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592917931
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119116/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593080918
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593040372
 
 
   **[Test build #119134 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119134/testReport)** for PR 27716 at commit [`c0d3755`](https://github.com/apache/spark/commit/c0d375535d4cc20f8fd18a011fc3165da1ab2b7e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385982006
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -230,18 +261,31 @@ public T get(Object key) {
 
     public void put(T value) throws Exception {
       data.put(asKey(naturalKey.get(value)), value);
+      if (!naturalParentIndexName.isEmpty()) {
+        Comparable<Object> parentKey = asKey(getIndexAccessor(naturalParentIndexName).get(value));
+        NaturalKeys children =
+          parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+        children.put(asKey(naturalKey.get(value)), true);
+      }
     }
 
     public void delete(Object key) {
       data.remove(asKey(key));
+      if (!naturalParentIndexName.isEmpty()) {
+        for (NaturalKeys v : parentToChildrenMap.values()) {
+          if (v.remove(asKey(key))) {
+            break;
+          }
+        }
 
 Review comment:
   When a parent key in `parentToChildrenMap` points to empty `NaturalKeys`, we can also remove it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591708344
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23745/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591844216
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23765/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591836476
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592870997
 
 
   **[Test build #119116 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119116/testReport)** for PR 27716 at commit [`e02acc0`](https://github.com/apache/spark/commit/e02acc058d78eb1e074a018bda45539941540632).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592472052
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119082/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591932061
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067901
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592180399
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23797/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591708337
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593049000
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119130/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593041315
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592738412
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119099/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385497409
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +370,28 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          ArrayList<T> elements = new ArrayList<>();
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.computeIfPresent(naturalKey, (k, v) -> {
 
 Review comment:
   ditto, let's highlight why we need to do linear scan in the else branch.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592471676
 
 
   **[Test build #119082 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119082/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592324203
 
 
   **[Test build #119068 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119068/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385613040
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,48 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
 
 Review comment:
   since there is no document, I still have problems understanding it.
   
   The basic idea is: if the `index` is natural index, we can just look it up in O(1). For other index, we have to do linear scan, extract keys and find matches.
   
   The extension here: if the `index` is parent of natural index, get the children natural indexes and do O(1) lookup.
   
   However, seems the basic idea is missing? Shall we avoid linear scan if `index` is natural index?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592415850
 
 
   **[Test build #119082 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119082/testReport)** for PR 27716 at commit [`091fb7e`](https://github.com/apache/spark/commit/091fb7e536ed67ee715acbec925fb21acbe9e450).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385999668
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +374,32 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
 
 Review comment:
   It is possible.  I have explained in https://github.com/apache/spark/pull/27716/files#r385846069. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592958960
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119119/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591826830
 
 
   **[Test build #119015 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119015/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385988906
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +374,32 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          // If there is a parent index for the natural index and the parent of`index` happens to be
 
 Review comment:
   of`index`  -> of `index` 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591743564
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593080739
 
 
   **[Test build #119140 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119140/testReport)** for PR 27716 at commit [`c0d3755`](https://github.com/apache/spark/commit/c0d375535d4cc20f8fd18a011fc3165da1ab2b7e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591708044
 
 
   **[Test build #118997 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118997/testReport)** for PR 27716 at commit [`d50c801`](https://github.com/apache/spark/commit/d50c801b6ca7106a49aa9f6819836af4174d86ef).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591689534
 
 
   **[Test build #118993 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118993/testReport)** for PR 27716 at commit [`4f93ffc`](https://github.com/apache/spark/commit/4f93ffcb55a2e79211a5d600b110bf652f95d0c5).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591846924
 
 
   **[Test build #119018 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119018/testReport)** for PR 27716 at commit [`753a14a`](https://github.com/apache/spark/commit/753a14a699761b41bc2c8059f101c1c68fc54100).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593041327
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23876/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591788436
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593315869
 
 
   @cloud-fan @viirya  Thanks for the review !

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591744570
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591701420
 
 
   **[Test build #118995 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118995/testReport)** for PR 27716 at commit [`b0bb448`](https://github.com/apache/spark/commit/b0bb4484b0b84bfb4027472795acf430f6cf3757).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r386289180
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,48 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
 
 Review comment:
   Yes,  I will do it in another PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591759552
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118994/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591708344
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23745/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593080921
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119140/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r386233234
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,48 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
 
 Review comment:
   we should probably unify the `delete` method, but it's not related to this PR.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385197298
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +370,28 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
 
 Review comment:
   if `parentKey` doesn't exist, we can just return an empty list?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592958778
 
 
   **[Test build #119119 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119119/testReport)** for PR 27716 at commit [`491e9eb`](https://github.com/apache/spark/commit/491e9eba32dfe6d9075115af5010691855c82b24).
    * This patch **fails from timeout after a configured wait of `400m`**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591743572
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118995/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] kiszk commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
kiszk commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591750512
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591777150
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993852
 
 
   **[Test build #119130 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119130/testReport)** for PR 27716 at commit [`491e9eb`](https://github.com/apache/spark/commit/491e9eba32dfe6d9075115af5010691855c82b24).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385198544
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +370,28 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          ArrayList<T> elements = new ArrayList<>();
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.computeIfPresent(naturalKey, (k, v) -> {
 
 Review comment:
   is this faster than `data.values().stream()...`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592657623
 
 
   retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592324410
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593055141
 
 
   **[Test build #119134 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119134/testReport)** for PR 27716 at commit [`c0d3755`](https://github.com/apache/spark/commit/c0d375535d4cc20f8fd18a011fc3165da1ab2b7e).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591705625
 
 
   **[Test build #118996 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118996/testReport)** for PR 27716 at commit [`8969a4d`](https://github.com/apache/spark/commit/8969a4dffc6fe9f54d7de34bee1e72fbf5c6f16d).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592923043
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592659469
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591764356
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067795
 
 
   **[Test build #119140 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119140/testReport)** for PR 27716 at commit [`c0d3755`](https://github.com/apache/spark/commit/c0d375535d4cc20f8fd18a011fc3165da1ab2b7e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591827239
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385372097
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -322,15 +370,28 @@ private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object ent
      */
     private List<T> copyElements() {
       if (parent != null) {
-        KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index);
-        Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index.");
-        Comparable<?> parentKey = asKey(parent);
-
-        return elements.stream()
-          .filter(e -> compare(e, parentGetter, parentKey) == 0)
-          .collect(Collectors.toList());
+        Comparable<Object> parentKey = asKey(parent);
+        if (!naturalParentIndexName.isEmpty() &&
+          naturalParentIndexName.equals(ti.getParentIndexName(index))) {
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          ArrayList<T> elements = new ArrayList<>();
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.computeIfPresent(naturalKey, (k, v) -> {
 
 Review comment:
   That's a linear scan, which we are trying to avoid. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591729597
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23747/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591777152
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118999/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591729213
 
 
   **[Test build #118999 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118999/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591701759
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23743/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591705988
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23744/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993329
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993329
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591744577
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118997/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591827249
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23762/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591705978
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593048998
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591759552
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/118994/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591765538
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591932061
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385192587
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,44 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
-      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), indexValues);
-      CountingRemoveIfForEach<T> callback = new CountingRemoveIfForEach<>(data, filter);
+      if (!naturalParentIndexName.isEmpty() && naturalParentIndexName.equals(index)) {
+        int count = 0;
+        for (Object indexValue : indexValues) {
+          Comparable<Object> parentKey = asKey(indexValue);
+          NaturalKeys children =
+            parentToChildrenMap.computeIfAbsent(parentKey, k -> new NaturalKeys());
+          for (Comparable<Object> naturalKey : children.keySet()) {
+            data.remove(naturalKey);
 
 Review comment:
   what's the difference between just call `data.remove` and `data.forEach(callback)`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592237424
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119051/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591752192
 
 
   **[Test build #119005 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119005/testReport)** for PR 27716 at commit [`da463e9`](https://github.com/apache/spark/commit/da463e95097f6173a3a9989d09bf054307a8e098).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591836476
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591844216
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23765/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592958956
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592416373
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23826/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592922884
 
 
   **[Test build #119119 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119119/testReport)** for PR 27716 at commit [`491e9eb`](https://github.com/apache/spark/commit/491e9eba32dfe6d9075115af5010691855c82b24).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591725111
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23746/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592659469
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#discussion_r385846069
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -205,23 +211,48 @@ public void accept(Comparable<Object> key, T value) {
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
     private final ConcurrentMap<Comparable<Object>, T> data;
+    private final String naturalParentIndexName;
+    // A mapping from parent to the natural keys of its children.
+    // For example, a mapping from a stage ID to all the task IDs in the stage.
+    private final ConcurrentMap<Comparable<Object>, NaturalKeys> parentToChildrenMap;
 
     private InstanceList(Class<?> klass) {
       this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
+      this.naturalParentIndexName = ti.getParentIndexName(KVIndex.NATURAL_INDEX_NAME);
+      this.parentToChildrenMap = new ConcurrentHashMap<>();
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
     int countingRemoveAllByIndexValues(String index, Collection<?> indexValues) {
 
 Review comment:
   Yes and no.
   I am aware of the natural index is not handled properly here. But all the method calls of this method doesn't pass the natural index in. 
   There is a simpler API to use in `KVStore`
   ```
   void delete(Class<?> type, Object naturalKey)
   ```
   
   So I think this is minor and I prefer to do it in another PR, since this one is complicated.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591765007
 
 
   **[Test build #118996 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118996/testReport)** for PR 27716 at commit [`8969a4d`](https://github.com/apache/spark/commit/8969a4dffc6fe9f54d7de34bee1e72fbf5c6f16d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591777150
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592237418
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592993262
 
 
   retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067903
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23882/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591724726
 
 
   **[Test build #118998 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118998/testReport)** for PR 27716 at commit [`9cb44c3`](https://github.com/apache/spark/commit/9cb44c32c49ef7ea782e47426ce0b45bbf8238d7).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591759545
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-592923043
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-591693847
 
 
   **[Test build #118993 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/118993/testReport)** for PR 27716 at commit [`4f93ffc`](https://github.com/apache/spark/commit/4f93ffcb55a2e79211a5d600b110bf652f95d0c5).
    * This patch **fails Java style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27716: [SPARK-30964][Core][WebUI] Accelerate InMemoryStore with a new index
URL: https://github.com/apache/spark/pull/27716#issuecomment-593067795
 
 
   **[Test build #119140 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119140/testReport)** for PR 27716 at commit [`c0d3755`](https://github.com/apache/spark/commit/c0d375535d4cc20f8fd18a011fc3165da1ab2b7e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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