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/07/12 01:35:26 UTC

[GitHub] [iceberg] southernriver commented on a change in pull request #2803: Spark: Add Multi-thread to construct ReadTask

southernriver commented on a change in pull request #2803:
URL: https://github.com/apache/iceberg/pull/2803#discussion_r667572467



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -205,35 +215,62 @@ public StructType readSchema() {
     // broadcast the table metadata as input partitions will be sent to executors
     Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
 
-    List<InputPartition<ColumnarBatch>> readTasks = Lists.newArrayList();
-    for (CombinedScanTask task : tasks()) {
-      readTasks.add(new ReadTask<>(
-          task, tableBroadcast, expectedSchemaString, caseSensitive,
-          localityPreferred, new BatchReaderFactory(batchSize)));
+    int taskSize = tasks().size();
+    InputPartition<ColumnarBatch>[] readTasks = new InputPartition[taskSize];
+    Long startTime = System.currentTimeMillis();
+    try {
+      pool.submit(() -> IntStream.range(0, taskSize).parallel()
+              .mapToObj(taskId -> {
+                LOG.trace("The size of scanTasks is {},  current taskId is {}, current thread id is {}",
+                        taskSize, taskId, Thread.currentThread().getName());
+                readTasks[taskId] = new ReadTask<>(
+                        tasks().get(taskId), tableBroadcast, expectedSchemaString, caseSensitive,
+                        localityPreferred, new BatchReaderFactory(batchSize));
+                return true;
+              }).collect(Collectors.toList())).get();
+    } catch (Exception e) {
+      e.printStackTrace();
+      System.exit(-1);
     }
-    LOG.info("Batching input partitions with {} tasks.", readTasks.size());
-
-    return readTasks;
+    Long endTime = System.currentTimeMillis();
+    LOG.info("Batching input partitions with {} tasks.", readTasks.length);
+    LOG.info("It took {} s to construct {} readTasks with localityPreferred = {}.", (endTime - startTime) / 1000,
+            taskSize, localityPreferred);
+    return Arrays.asList(readTasks.clone());
   }
 
   /**
    * This is called in the Spark Driver when data is to be materialized into {@link InternalRow}
    */
+  @SuppressWarnings({"checkstyle:LocalVariableName", "checkstyle:RegexpSinglelineJava"})
   @Override
   public List<InputPartition<InternalRow>> planInputPartitions() {
     String expectedSchemaString = SchemaParser.toJson(lazySchema());
 
     // broadcast the table metadata as input partitions will be sent to executors
     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));
+    int taskSize = tasks().size();
+    InputPartition<InternalRow>[] readTasks = new InputPartition[taskSize];
+    Long start_time = System.currentTimeMillis();
+    try {
+      pool.submit(() -> IntStream.range(0, taskSize).parallel()
+              .mapToObj(taskId -> {
+                LOG.trace("The size of scanTasks is {},  current taskId is {}, current thread id is {}",
+                        taskSize, taskId, Thread.currentThread().getName());
+                readTasks[taskId] = new ReadTask<>(
+                        tasks().get(taskId), tableBroadcast, expectedSchemaString, caseSensitive,
+                        localityPreferred, InternalRowReaderFactory.INSTANCE);
+                return true;
+              }).collect(Collectors.toList())).get();
+    } catch (Exception e) {
+      e.printStackTrace();
+      System.exit(-1);

Review comment:
       Thank you for reviewing this! I wanted to throw exception out at first,  and if I did this, I also need to change code of 
   `org.apache.spark.sql.connector.read.Batch` and so on  which is outside of iceberg. Here is the err msg:
   
   > planInputPartitions()' in 'org.apache.iceberg.spark.source.SparkBatchScan' clashes with 'planInputPartitions()' in 'org.apache.spark.sql.connector.read.Batch'; overridden method does not throw 'java.lang.Exception'
   




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