parthchandra commented on code in PR #8755:
URL: https://github.com/apache/iceberg/pull/8755#discussion_r1353486486


##########
core/src/main/java/org/apache/iceberg/SystemConfigs.java:
##########
@@ -42,6 +42,13 @@ private SystemConfigs() {}
           Math.max(2, Runtime.getRuntime().availableProcessors()),
           Integer::parseUnsignedInt);
 
+  public static final ConfigEntry<Integer> DELETE_WORKER_THREAD_POOL_SIZE =
+      new ConfigEntry<>(
+          "iceberg.worker.delete-num-threads",
+          "ICEBERG_WORKER_DELETE_NUM_THREADS",
+          4 * Runtime.getRuntime().availableProcessors(),

Review Comment:
   Is this end user configurable? If not then it probably needs to be.



##########
core/src/main/java/org/apache/iceberg/util/ThreadPools.java:
##########
@@ -59,6 +65,20 @@ public static ExecutorService getWorkerPool() {
     return WORKER_POOL;
   }
 
+  /**
+   * Return an {@link ExecutorService} for loading deletes.
+   *
+   * <p>The size of this worker pool limits the number of tasks concurrently 
reading delete files
+   * within a single JVM. In most cases, deletes are loaded whenever reading 
data on executors. The
+   * size of this pool has to be big enough to handle all data tasks running 
on the same executor.
+   *
+   * @return an {@link ExecutorService} that uses the delete worker pool
+   * @see SystemConfigs#DELETE_WORKER_THREAD_POOL_SIZE
+   */
+  public static ExecutorService getDeleteWorkerPool() {

Review Comment:
   You might have to. And maybe will need to pass in the configured thread pool 
size as an argument.



##########
data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.orc.GenericOrcReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.Deletes;
+import org.apache.iceberg.deletes.PositionDeleteIndex;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.orc.OrcRowReader;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.ParquetValueReader;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.CharSequenceMap;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.Tasks;
+import org.apache.orc.TypeDescription;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BaseDeleteLoader implements DeleteLoader {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(BaseDeleteLoader.class);
+  private static final long MAX_CACHE_FILE_SIZE = 32 * 1024 * 1024; // 32 MB

Review Comment:
   Probably, yes. Even if it is only to avoid the inevitable comment from a 
reviewer. 



##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -236,6 +236,8 @@ private TableProperties() {}
   public static final String DELETE_PLANNING_MODE = 
"read.delete-planning-mode";
   public static final String PLANNING_MODE_DEFAULT = 
PlanningMode.AUTO.modeName();
 
+  public static final String SPARK_EXECUTOR_CACHE_ENABLED = 
"read.spark.executor-cache.enabled";

Review Comment:
   Passed in as a hadoop conf property, or a catalog property since we want 
this to be end user configurable? So probably not as part of table properties? 



##########
core/src/main/java/org/apache/iceberg/util/ThreadPools.java:
##########
@@ -44,6 +44,12 @@ private ThreadPools() {}
 
   private static final ExecutorService WORKER_POOL = 
newWorkerPool("iceberg-worker-pool");
 
+  public static final int DELETE_WORKER_THREAD_POOL_SIZE =
+      SystemConfigs.DELETE_WORKER_THREAD_POOL_SIZE.value();
+
+  private static final ExecutorService DELETE_WORKER_POOL =

Review Comment:
   If the size of the thread pool is end user configurable this will not work. 
But you could initialize the thread pool lazily in `getDeleteWorkerPool()` and 
presumably there will be some way to read the end user configured value at that 
point.



##########
core/src/main/java/org/apache/iceberg/SystemConfigs.java:
##########
@@ -42,6 +42,13 @@ private SystemConfigs() {}
           Math.max(2, Runtime.getRuntime().availableProcessors()),
           Integer::parseUnsignedInt);
 
+  public static final ConfigEntry<Integer> DELETE_WORKER_THREAD_POOL_SIZE =
+      new ConfigEntry<>(
+          "iceberg.worker.delete-num-threads",
+          "ICEBERG_WORKER_DELETE_NUM_THREADS",
+          4 * Runtime.getRuntime().availableProcessors(),

Review Comment:
   There is really no good way to pick the default for this since
   `Thread_pool_size = fn(cores, io_wait_time/compute_time)`, and your guess is 
as good as anyone else's whether `4` is a good number for the environment the 
code is going to run on.
   Assuming `io_wait time/compute_time == 5`, a factor of 4 above would give 
you a utilization of 80% which sounds pretty good. 
   
   



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to