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/09/01 23:00:44 UTC

[GitHub] [spark] baohe-zhang commented on a change in pull request #29425: [SPARK-32350][FOLLOW-UP] Fix count update issue and partition the value list to a set of small batches for LevelDB writeAll

baohe-zhang commented on a change in pull request #29425:
URL: https://github.com/apache/spark/pull/29425#discussion_r481478177



##########
File path: common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java
##########
@@ -164,35 +165,39 @@ public void writeAll(List<?> values) throws Exception {
     Preconditions.checkArgument(values != null && !values.isEmpty(),
       "Non-empty values required.");
 
-    // Group by class, in case there are values from different classes in the values
+    // Group by class, in case there are values from different classes in the values.
     // Typical usecase is for this to be a single class.
     // A NullPointerException will be thrown if values contain null object.
     for (Map.Entry<? extends Class<?>, ? extends List<?>> entry :
         values.stream().collect(Collectors.groupingBy(Object::getClass)).entrySet()) {
-
-      final Iterator<?> valueIter = entry.getValue().iterator();
-      final Iterator<byte[]> serializedValueIter;
-
-      // Deserialize outside synchronized block
-      List<byte[]> list = new ArrayList<>(entry.getValue().size());
-      for (Object value : values) {
-        list.add(serializer.serialize(value));
-      }
-      serializedValueIter = list.iterator();
-
       final Class<?> klass = entry.getKey();
-      final LevelDBTypeInfo ti = getTypeInfo(klass);
 
-      synchronized (ti) {
-        final LevelDBTypeInfo.Index naturalIndex = ti.naturalIndex();
-        final Collection<LevelDBTypeInfo.Index> indices = ti.indices();
+      // Partition the value list to a set of the 128-values batches. It can reduce the
+      // memory pressure caused by serialization and give fairness to other writing threads
+      // when writing a very large list.
+      for (List<?> batchList : Iterables.partition(entry.getValue(), 128)) {
+        final Iterator<?> valueIter = batchList.iterator();
+        final Iterator<byte[]> serializedValueIter;
+
+        // Deserialize outside synchronized block
+        List<byte[]> serializedValueList = new ArrayList<>(batchList.size());
+        for (Object value : batchList) {
+          serializedValueList.add(serializer.serialize(value));
+        }
+        serializedValueIter = serializedValueList.iterator();
+
+        final LevelDBTypeInfo ti = getTypeInfo(klass);
+        synchronized (ti) {
+          final LevelDBTypeInfo.Index naturalIndex = ti.naturalIndex();
+          final Collection<LevelDBTypeInfo.Index> indices = ti.indices();
 
-        try (WriteBatch batch = db().createWriteBatch()) {
           while (valueIter.hasNext()) {
-            updateBatch(batch, valueIter.next(), serializedValueIter.next(), klass,
-              naturalIndex, indices);
+            try (WriteBatch batch = db().createWriteBatch()) {
+              updateBatch(batch, valueIter.next(), serializedValueIter.next(), klass,
+                naturalIndex, indices);
+              db().write(batch);

Review comment:
       Hi @HeartSaVioR , I fixed the bug by using a hashmap to store and update the <countKey, delta> pair in a batch and use the information in that hashmap to update the count right before calling db.write(batch). I updated the test case to cover the count update issue. I also ran benchmark tests for writeAll and put the result in the description section of 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



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