You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/05/17 23:58:55 UTC

[GitHub] [iceberg] kbendick commented on a change in pull request #2577: Spark: Add read.locality.enabled to TableProperties to support disabl…

kbendick commented on a change in pull request #2577:
URL: https://github.com/apache/iceberg/pull/2577#discussion_r633937711



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -227,15 +233,58 @@ public StructType readSchema() {
     Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
 
     List<InputPartition<InternalRow>> readTasks = Lists.newArrayList();
-    for (CombinedScanTask task : tasks()) {
-      readTasks.add(new ReadTask<>(
-          task, tableBroadcast, expectedSchemaString, caseSensitive,
-          localityPreferred, InternalRowReaderFactory.INSTANCE));
-    }
+
+    initializeReadTasks(readTasks, tableBroadcast, expectedSchemaString, () -> InternalRowReaderFactory.INSTANCE);
 
     return readTasks;
   }
 
+  /**
+   * Initialize ReadTasks with multi threads as get block locations can be slow
+   *
+   * @param readTasks Result list to return
+   */
+  private <T> void initializeReadTasks(List<InputPartition<T>> readTasks,
+      Broadcast<Table> tableBroadcast, String expectedSchemaString, Supplier<ReaderFactory<T>> supplier) {
+    int taskInitThreads = Math.max(1, PropertyUtil.propertyAsInt(table.properties(), LOCALITY_TASK_INITIALIZE_THREADS,
+        LOCALITY_TASK_INITIALIZE_THREADS_DEFAULT));
+
+    if (!localityPreferred || taskInitThreads == 1) {
+      for (CombinedScanTask task : tasks()) {
+        readTasks.add(new ReadTask<>(
+            task, tableBroadcast, expectedSchemaString, caseSensitive,
+            localityPreferred, supplier.get()));
+      }
+      return;
+    }
+
+    List<Future<ReadTask<T>>> futures = new ArrayList<>();
+
+    final ExecutorService pool = Executors.newFixedThreadPool(
+        taskInitThreads,
+        new ThreadFactoryBuilder()
+            .setDaemon(true)
+            .setNameFormat("Init-ReadTask-%d")
+            .build());
+
+    List<CombinedScanTask> scanTasks = tasks();
+    for (int i = 0; i < scanTasks.size(); i++) {
+      final int curIndex = i;
+      futures.add(pool.submit(() -> new ReadTask<>(scanTasks.get(curIndex), tableBroadcast,
+          expectedSchemaString, caseSensitive, true, supplier.get())));
+    }

Review comment:
       You might be able to use `org.apache.iceberg.util.ParallelIterable` here to help simplify some of this code that takes care of submitting tasks. 

##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -227,15 +233,58 @@ public StructType readSchema() {
     Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
 
     List<InputPartition<InternalRow>> readTasks = Lists.newArrayList();
-    for (CombinedScanTask task : tasks()) {
-      readTasks.add(new ReadTask<>(
-          task, tableBroadcast, expectedSchemaString, caseSensitive,
-          localityPreferred, InternalRowReaderFactory.INSTANCE));
-    }
+
+    initializeReadTasks(readTasks, tableBroadcast, expectedSchemaString, () -> InternalRowReaderFactory.INSTANCE);
 
     return readTasks;
   }
 
+  /**
+   * Initialize ReadTasks with multi threads as get block locations can be slow
+   *
+   * @param readTasks Result list to return
+   */
+  private <T> void initializeReadTasks(List<InputPartition<T>> readTasks,
+      Broadcast<Table> tableBroadcast, String expectedSchemaString, Supplier<ReaderFactory<T>> supplier) {
+    int taskInitThreads = Math.max(1, PropertyUtil.propertyAsInt(table.properties(), LOCALITY_TASK_INITIALIZE_THREADS,
+        LOCALITY_TASK_INITIALIZE_THREADS_DEFAULT));
+
+    if (!localityPreferred || taskInitThreads == 1) {
+      for (CombinedScanTask task : tasks()) {
+        readTasks.add(new ReadTask<>(
+            task, tableBroadcast, expectedSchemaString, caseSensitive,
+            localityPreferred, supplier.get()));
+      }
+      return;
+    }
+
+    List<Future<ReadTask<T>>> futures = new ArrayList<>();
+
+    final ExecutorService pool = Executors.newFixedThreadPool(
+        taskInitThreads,
+        new ThreadFactoryBuilder()
+            .setDaemon(true)
+            .setNameFormat("Init-ReadTask-%d")
+            .build());

Review comment:
       Have a look at `org.apache.iceberg.util.ThreadPools`, which can help handle this for you. It would also allow for the pool size to be set as a system property (as well as providing space for a default).
   
   I'm not too sure how I feel about the default coming as a table level property though. Seems to me like how parallel and whether or not you want to wait for locality would be specific to the cluster that the job is running on. But I don't have a strong opinion on that.




-- 
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org