@@ -105,13 +105,29 @@ def toPandas(self):
105
105
import pyarrow
106
106
# Rename columns to avoid duplicated column names.
107
107
tmp_column_names = ['col_{}' .format (i ) for i in range (len (self .columns ))]
108
- batches = self .toDF (* tmp_column_names )._collect_as_arrow ()
108
+ self_destruct = self .sql_ctx ._conf .arrowPySparkSelfDestructEnabled ()
109
+ batches = self .toDF (* tmp_column_names )._collect_as_arrow (
110
+ split_batches = self_destruct )
109
111
if len (batches ) > 0 :
110
112
table = pyarrow .Table .from_batches (batches )
113
+ # Ensure only the table has a reference to the batches, so that
114
+ # self_destruct (if enabled) is effective
115
+ del batches
111
116
# Pandas DataFrame created from PyArrow uses datetime64[ns] for date type
112
117
# values, but we should use datetime.date to match the behavior with when
113
118
# Arrow optimization is disabled.
114
- pdf = table .to_pandas (date_as_object = True )
119
+ pandas_options = {'date_as_object' : True }
120
+ if self_destruct :
121
+ # Configure PyArrow to use as little memory as possible:
122
+ # self_destruct - free columns as they are converted
123
+ # split_blocks - create a separate Pandas block for each column
124
+ # use_threads - convert one column at a time
125
+ pandas_options .update ({
126
+ 'self_destruct' : True ,
127
+ 'split_blocks' : True ,
128
+ 'use_threads' : False ,
129
+ })
130
+ pdf = table .to_pandas (** pandas_options )
115
131
# Rename back to the original column names.
116
132
pdf .columns = self .columns
117
133
for field in self .schema :
@@ -225,11 +241,16 @@ def _to_corrected_pandas_type(dt):
225
241
else :
226
242
return None
227
243
228
- def _collect_as_arrow (self ):
244
+ def _collect_as_arrow (self , split_batches = False ):
229
245
"""
230
246
Returns all records as a list of ArrowRecordBatches, pyarrow must be installed
231
247
and available on driver and worker Python environments.
232
248
This is an experimental feature.
249
+
250
+ :param split_batches: split batches such that each column is in its own allocation, so
251
+ that the selfDestruct optimization is effective; default False.
252
+
253
+ .. note:: Experimental.
233
254
"""
234
255
from pyspark .sql .dataframe import DataFrame
235
256
@@ -240,7 +261,26 @@ def _collect_as_arrow(self):
240
261
241
262
# Collect list of un-ordered batches where last element is a list of correct order indices
242
263
try :
243
- results = list (_load_from_socket ((port , auth_secret ), ArrowCollectSerializer ()))
264
+ batch_stream = _load_from_socket ((port , auth_secret ), ArrowCollectSerializer ())
265
+ if split_batches :
266
+ # When spark.sql.execution.arrow.pyspark.selfDestruct.enabled, ensure
267
+ # each column in each record batch is contained in its own allocation.
268
+ # Otherwise, selfDestruct does nothing; it frees each column as its
269
+ # converted, but each column will actually be a list of slices of record
270
+ # batches, and so no memory is actually freed until all columns are
271
+ # converted.
272
+ import pyarrow as pa
273
+ results = []
274
+ for batch_or_indices in batch_stream :
275
+ if isinstance (batch_or_indices , pa .RecordBatch ):
276
+ batch_or_indices = pa .RecordBatch .from_arrays ([
277
+ # This call actually reallocates the array
278
+ pa .concat_arrays ([array ])
279
+ for array in batch_or_indices
280
+ ], schema = batch_or_indices .schema )
281
+ results .append (batch_or_indices )
282
+ else :
283
+ results = list (batch_stream )
244
284
finally :
245
285
# Join serving thread and raise any exceptions from collectAsArrowToPython
246
286
jsocket_auth_server .getResult ()
0 commit comments