BryanCutler commented on a change in pull request #29818:
URL: https://github.com/apache/spark/pull/29818#discussion_r505197033
##########
File path: python/pyspark/sql/pandas/conversion.py
##########
@@ -34,7 +34,7 @@ class PandasConversionMixin(object):
"""
@since(1.3)
- def toPandas(self):
+ def toPandas(self, selfDestruct=False):
Review comment:
Would this be better as an sql config? Since this class is a mixin, I'm
not sure the user would see this option from the public api?
##########
File path: python/pyspark/sql/pandas/conversion.py
##########
@@ -103,10 +103,22 @@ def toPandas(self):
batches = self.toDF(*tmp_column_names)._collect_as_arrow()
if len(batches) > 0:
table = pyarrow.Table.from_batches(batches)
+ del batches
Review comment:
does this have any bearing on the buffers self destructing? is it taking
into account how many reference counts there are before destructing?
##########
File path: python/pyspark/sql/pandas/conversion.py
##########
@@ -103,10 +103,18 @@ def toPandas(self):
batches = self.toDF(*tmp_column_names)._collect_as_arrow()
if len(batches) > 0:
table = pyarrow.Table.from_batches(batches)
+ del batches
# Pandas DataFrame created from PyArrow uses
datetime64[ns] for date type
# values, but we should use datetime.date to match the
behavior with when
# Arrow optimization is disabled.
- pdf = table.to_pandas(date_as_object=True)
+ pandas_options = {'date_as_object': True}
+ if self_destruct:
+ pandas_options.update({
+ 'self_destruct': True,
+ 'split_blocks': True,
+ 'use_threads': False,
+ })
+ pdf = table.to_pandas(**pandas_options)
Review comment:
yeah, I think this option can lead to other side effects, best to
disable by default I think.
##########
File path: python/pyspark/sql/pandas/serializers.py
##########
@@ -90,7 +90,10 @@ def load_stream(self, stream):
import pyarrow as pa
reader = pa.ipc.open_stream(stream)
for batch in reader:
- yield batch
+ split_batch = pa.RecordBatch.from_arrays([
+ pa.concat_arrays([array]) for array in batch
+ ], schema=batch.schema)
Review comment:
This looks a little strange to me too. Is `concat_arrays` actually doing
anything here, and if so, wouldn't it do it for the case that `selfDestruct` is
False too?
##########
File path: python/pyspark/sql/pandas/conversion.py
##########
@@ -103,10 +103,22 @@ def toPandas(self):
batches = self.toDF(*tmp_column_names)._collect_as_arrow()
if len(batches) > 0:
table = pyarrow.Table.from_batches(batches)
+ del batches
# Pandas DataFrame created from PyArrow uses
datetime64[ns] for date type
# values, but we should use datetime.date to match the
behavior with when
# Arrow optimization is disabled.
- pdf = table.to_pandas(date_as_object=True)
+ pandas_options = {'date_as_object': True}
+ if selfDestruct:
+ # Configure PyArrow to use as little memory as
possible:
+ # self_destruct - free columns as they are
converted
+ # split_blocks - create a separate Pandas block
for each column
+ # use_threads - convert one column at a time
+ pandas_options.update({
+ 'self_destruct': True,
+ 'split_blocks': True,
Review comment:
Is this necessary to set with `self_destruct`? It might lead to Pandas
doing more memory allocation later, I believe.
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]