ConeyLiu commented on code in PR #1427:
URL: https://github.com/apache/iceberg-python/pull/1427#discussion_r1886464171


##########
pyiceberg/table/__init__.py:
##########
@@ -1423,6 +1451,66 @@ def plan_files(self) -> Iterable[FileScanTask]:
             for data_entry in data_entries
         ]
 
+    def _target_split_size(self) -> int:
+        table_value = property_as_int(
+            self.table_metadata.properties, TableProperties.READ_SPLIT_SIZE, 
TableProperties.READ_SPLIT_SIZE_DEFAULT
+        )
+        return property_as_int(self.options, TableProperties.READ_SPLIT_SIZE, 
table_value)  # type: ignore
+
+    def _loop_back(self) -> int:
+        table_value = property_as_int(
+            self.table_metadata.properties, 
TableProperties.READ_SPLIT_LOOKBACK, TableProperties.READ_SPLIT_LOOKBACK_DEFAULT
+        )
+        return property_as_int(self.options, 
TableProperties.READ_SPLIT_LOOKBACK, table_value)  # type: ignore
+
+    def _split_open_file_cost(self) -> int:
+        table_value = property_as_int(
+            self.table_metadata.properties,
+            TableProperties.READ_SPLIT_OPEN_FILE_COST,
+            TableProperties.READ_SPLIT_OPEN_FILE_COST_DEFAULT,
+        )
+        return property_as_int(self.options, 
TableProperties.READ_SPLIT_OPEN_FILE_COST, table_value)  # type: ignore
+
+    def plan_task(self) -> Iterable[CombinedFileScanTask]:

Review Comment:
   > I assume it's intentional that we're not actually calling this in any of 
the methods (like to_arrow, etc) that actually execute a scan?
   
   Yes, I am not sure of the initial intention why we read the full file. So 
keep it not changing.
   
   > If there is no plan to call this directly in pyiceberg, I wonder who the 
intended consumers of this API would be? I would expect most query engines -- 
distributed or otherwise -- to have their own notions for how to optimize scan 
planning.
   
   I met OOM errors recently when we read the iceberg table with ray in 
distributed. Also, the parallelism is limited by the number of files when 
planning with files(files are rewritten into larger one, eg 512MB or 1GB). Like 
Spark/Flink, I think the `plan_tasks` should be useful for distributed reading.
   
   > In the case that this is primarily intended to be for the benefit of 
pyiceberg's own scan execution I would consider making this a private API.
   
   The distributed engine such as ray/daft would have to implement themself if 
they want to do `plan_tasks`. It would be better to put it into pyiceberg?



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