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 2019/05/16 02:40:38 UTC

[GitHub] [spark] davidnavas commented on a change in pull request #24616: [SPARK-27726] [Core] Fix performance of ElementTrackingStore deletes when using InMemoryStore under high loads

davidnavas commented on a change in pull request #24616: [SPARK-27726] [Core] Fix performance of ElementTrackingStore deletes when using InMemoryStore under high loads
URL: https://github.com/apache/spark/pull/24616#discussion_r284520604
 
 

 ##########
 File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -126,64 +132,144 @@ public void close() {
     return (Comparable<Object>) in;
   }
 
-  private static class InstanceList {
+  @SuppressWarnings("unchecked")
+  private static <T> KVStoreView<T> emptyView() {
+    return (InMemoryView<T>) InMemoryView.EMPTY_VIEW;
+  }
+
+  /**
+   * Encapsulates ConcurrentHashMap so that the typing in and out of the map strictly maps a
+   * class of type T to an InstanceList of type T.
+   */
+  private static class InMemoryLists {
+    private ConcurrentMap<Class<?>, InstanceList<?>> data = new ConcurrentHashMap<>();
+
+    @SuppressWarnings("unchecked")
+    public <T> InstanceList<T> get(Class<T> type) {
+      return (InstanceList<T>)data.get(type);
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> void write(T value) throws Exception {
+      InstanceList<T> list =
+        (InstanceList<T>) data.computeIfAbsent(value.getClass(), InstanceList::new);
+      list.put(value);
+    }
+
+    public void clear() {
+      data.clear();
+    }
+  }
+
+  private static class InstanceList<T> {
+
+    private static class CountingRemoveIfForEach<T> implements BiConsumer<Comparable<Object>, T> {
+      ConcurrentMap<Comparable<Object>, T> data;
+      Predicate<? super T> filter;
+      int count = 0;
+
+      CountingRemoveIfForEach(
+          ConcurrentMap<Comparable<Object>, T> data,
+          Predicate<? super T> filter) {
+        this.data = data;
+        this.filter = filter;
+      }
+
+      public void accept(Comparable<Object> key, T value) {
+        // To address https://bugs.openjdk.java.net/browse/JDK-8078645 which affects remove() on
+        // all iterators of concurrent maps, and specifically makes countingRemoveIf difficult to
+        // implement correctly against the values() iterator, we use forEach instead....
+        if (filter.test(value)) {
+          if (data.remove(key, value)) {
+            count++;
+          }
+        }
+      }
+    }
 
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
-    private final ConcurrentMap<Comparable<Object>, Object> data;
-
-    private int size;
+    private final ConcurrentMap<Comparable<Object>, T> data;
 
-    private InstanceList(Class<?> type) throws Exception {
-      this.ti = new KVTypeInfo(type);
+    private InstanceList(Class<?> klass) {
+      this.ti = new KVTypeInfo(klass);
       this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME);
       this.data = new ConcurrentHashMap<>();
-      this.size = 0;
     }
 
     KVTypeInfo.Accessor getIndexAccessor(String indexName) {
       return ti.getAccessor(indexName);
     }
 
-    public Object get(Object key) {
+    // Note: removeIf returns a boolean if any element has been removed.
+    // While debugging this code, it was handy to have the count of elements
+    // removed, rather than an indicator of whether something has been
+    // removed, and a count is no more complicated than a boolean so I've
+    // retained that behavior here, although there is no current requirement.
+    @SuppressWarnings("unchecked")
+    int countingRemoveAllByKeys(String index, Collection keys) {
 
 Review comment:
   I'm removing values whose key indicated by "index" matches (one of) the passed keys.
   So, I'm matching keys, hence the ByKeys and the keyFromValue which retrieves the key indicated by 'getter' of the passed value.
   BTW, I think I could add generic typing to Accessor<V, K> and ensure that the getter and value objects match, but I don't think the K type would turn out to be terribly useful, as ultimately there's no match between the index (a String) and the key type.  Let me know if you think that's useful.
   
   Yes, I'll try to use the <?> in a number of the SuppressWarnings and see what happens.  It might be possible to get rid of a number of them, which would be fabulous.

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