You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "deniskuzZ (via GitHub)" <gi...@apache.org> on 2023/05/22 14:03:08 UTC

[GitHub] [iceberg] deniskuzZ commented on a diff in pull request #6527: Iceberg delete files are read multiple times during query processing causing delays

deniskuzZ commented on code in PR #6527:
URL: https://github.com/apache/iceberg/pull/6527#discussion_r1200563839


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -367,4 +389,86 @@ private static Schema fileProjection(
 
     return new Schema(columns);
   }
+
+  public static Map<String, PositionDeleteIndex> createPosIndexMap(
+      Iterable<FileScanTask> fileTasks, FileIO io) {
+    return createPosIndexMap(fileTasks, DEFAULT_SET_FILTER_THRESHOLD, io);
+  }
+
+  public static Map<String, PositionDeleteIndex> createPosIndexMap(
+      Iterable<FileScanTask> fileTasks, long filterThreshold, FileIO io) {
+    List<DeleteFile> posDeletes = distinctPosDeletes(fileTasks);
+    if (posDeletes.isEmpty()) {
+      return ImmutableMap.of();
+    }
+    // if there are fewer deletes than a reasonable number to keep in memory, use a set
+    if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < filterThreshold) {
+      CharSequenceSet filePaths =
+          CharSequenceSet.of(Iterables.transform(fileTasks, task -> task.file().path()));
+      List<CloseableIterable<Record>> deletes =
+          Lists.transform(posDeletes, row -> openPosDeletes(io, row));
+      return Deletes.toPositionIndexMap(filePaths, deletes);
+    }
+    return null;
+  }
+
+  public static Map<String, PositionDeleteIndex> createPosIndexMap(
+      Cache<CharSequence, Map<String, PositionDeleteIndex>> posIndexCache,
+      Iterable<FileScanTask> fileTasks,
+      FileIO io) {
+    List<DeleteFile> posDeletes = distinctPosDeletes(fileTasks);
+    if (posDeletes.isEmpty()) {
+      return ImmutableMap.of();
+    }
+    // if there are fewer deletes than a reasonable number to keep in memory, use a set
+    if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum()
+        < DEFAULT_SET_FILTER_THRESHOLD) {
+      CharSequenceSet filePaths =
+          CharSequenceSet.of(Iterables.transform(fileTasks, task -> task.file().path()));
+
+      Map<String, PositionDeleteIndex> posIndexMap = Maps.newConcurrentMap();
+
+      // open all of the delete files in parallel, use index to avoid reordering
+      Tasks.range(posDeletes.size())
+          .stopOnFailure()
+          .throwFailureWhenFinished()
+          .executeWith(ThreadPools.getDeleteWorkerPool())
+          .run(
+              index -> {
+                DeleteFile deleteFile = posDeletes.get(index);
+
+                Maps.filterKeys(
+                        posIndexCache.get(
+                            deleteFile.path(),
+                            func -> {
+                              LOG.debug("Cache miss: {}", deleteFile.path());
+                              Instant start = Instant.now();
+                              Map<String, PositionDeleteIndex> res =
+                                  Deletes.toPositionIndexMap(openPosDeletes(io, deleteFile));
+                              LOG.debug(
+                                  "Cache load: {}; Time taken: {} ms;",
+                                  deleteFile.path(),
+                                  Duration.between(start, Instant.now()).toMillis());
+                              return res;
+                            }),
+                        filePaths::contains)
+                    .forEach(
+                        (key, value) -> posIndexMap.merge(key, value, PositionDeleteIndex::or));
+              });
+      LOG.debug("Cache size: {}", posIndexCache.estimatedSize());
+
+      return posIndexMap;
+    }
+    return null;

Review Comment:
   it's not. there is a special handling logic in PositionalDeletes class:



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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org