singhpk234 commented on code in PR #5143:
URL: https://github.com/apache/iceberg/pull/5143#discussion_r908119520


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java:
##########
@@ -65,18 +63,24 @@ public InputPartition[] planInputPartitions() {
     Broadcast<Table> tableBroadcast = 
sparkContext.broadcast(SerializableTable.copyOf(table));
     String expectedSchemaString = SchemaParser.toJson(expectedSchema);
 
-    InputPartition[] readTasks = new InputPartition[tasks.size()];
+    InputPartition[] readTasks = new InputPartition[tasks().size()];
 
     Tasks.range(readTasks.length)
         .stopOnFailure()
         .executeWith(localityEnabled ? ThreadPools.getWorkerPool() : null)
         .run(index -> readTasks[index] = new ReadTask(
-            tasks.get(index), tableBroadcast, expectedSchemaString,
+            tasks().get(index), tableBroadcast, expectedSchemaString,

Review Comment:
   [minor] task() is synchronized, and we might be accessing here with a pool, 
should remove synchronized from function def and make it like : 
   
   ```
   private volatile List<CombinedScanTask> tasks = null
   
   protected  List<CombinedScanTask> tasks() {
       if (tasks == null) {
         synchronized(this) {
           if (tasks == null) {
               task = // assign stuff to task
            }
          }
       }
       return tasks;
     }
   ```
   
   I think now we might also want to make `SparkFilesScan#task()` handle 
multi-threading.
   



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to