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

 ##########
 File path: 
common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java
 ##########
 @@ -126,64 +134,149 @@ 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> {
+
+    /**
+     * A BiConsumer to control multi-entity removal.  We use this in a forEach 
rather than an
+     * iterator because there is a bug in jdk8 which affects remove() on all 
concurrent map
+     * iterators.  https://bugs.openjdk.java.net/browse/JDK-8078645
+     */
+    private static class CountingRemoveIfForEach<T> implements 
BiConsumer<Comparable<Object>, T> {
+      private final ConcurrentMap<Comparable<Object>, T> data;
+      private final Predicate<? super T> filter;
+
+      /**
+       * Keeps a count of the number of elements removed.  This count is not 
currently surfaced
+       * to clients of KVStore as Java's generic removeAll() construct returns 
only a boolean,
+       * but I found it handy to have the count of elements removed while 
debugging; a count being
+       * no more complicated than a boolean, I've retained that behavior here, 
even though there
+       * is no current requirement.
+       */
+      private 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) {
+        if (filter.test(value)) {
+          if (data.remove(key, value)) {
+            count++;
+          }
+        }
+      }
+
+      public int count() { return count; }
+    }
 
     private final KVTypeInfo ti;
     private final KVTypeInfo.Accessor naturalKey;
-    private final ConcurrentMap<Comparable<Object>, Object> data;
+    private final ConcurrentMap<Comparable<Object>, T> data;
 
-    private int size;
-
-    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) {
+    int countingRemoveAllByIndexValues(String index, Collection<?> 
indexValues) {
+      Predicate<? super T> filter = getPredicate(ti.getAccessor(index), 
indexValues);
+      CountingRemoveIfForEach<T> callback = new 
CountingRemoveIfForEach<>(data, filter);
+
+      data.forEach(callback);
+      return callback.count();
+    }
+
+    public T get(Object key) {
       return data.get(asKey(key));
     }
 
-    public void put(Object value) throws Exception {
-      Preconditions.checkArgument(ti.type().equals(value.getClass()),
-        "Unexpected type: %s", value.getClass());
-      if (data.put(asKey(naturalKey.get(value)), value) == null) {
-        size++;
-      }
+    public void put(T value) throws Exception {
+      data.put(asKey(naturalKey.get(value)), value);
     }
 
     public void delete(Object key) {
-      if (data.remove(asKey(key)) != null) {
-        size--;
-      }
+      data.remove(asKey(key));
     }
 
     public int size() {
-      return size;
+      return data.size();
     }
 
-    @SuppressWarnings("unchecked")
-    public <T> InMemoryView<T> view(Class<T> type) {
-      Preconditions.checkArgument(ti.type().equals(type), "Unexpected type: 
%s", type);
-      Collection<T> all = (Collection<T>) data.values();
-      return new InMemoryView<>(type, all, ti);
+    public InMemoryView<T> view() {
+      return new InMemoryView<>(data.values(), ti);
+    }
+
+    private static <T> Predicate<? super T> getPredicate(
+        KVTypeInfo.Accessor getter,
+        Collection<?> keys) {
 
 Review comment:
   s/keys/values

----------------------------------------------------------------
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:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to