Commit aad6c51
[SPARK-54182][SQL][PYTHON] Optimize non-arrow conversion of
### What changes were proposed in this pull request?
Following up with #52680, this PR optimizes the non-Arrow path of `toPandas()` to eliminate intermediate DataFrame creation.
**Key optimizations:**
1. **Avoid intermediate DataFrame copy**
- `pd.DataFrame.from_records(rows)` → Direct column extraction via `zip(*rows)`
- 2 DataFrame creations → 1 DataFrame creation
2. **Optimize column-by-column conversion** (especially for wide tables)
- Tuples → Lists for faster Series construction
- Implicit dtype inference → Explicit `dtype=object`
- `pd.concat(axis="columns")` + column rename → `pd.concat(axis=1, keys=columns)`
- Result: **43-67% speedup for 50-100 columns**
### Why are the changes needed?
**Problem:** Current flow creates DataFrame twice:
- `rows` → `pd.DataFrame.from_records()` → temporary DataFrame → `pd.concat()` → final DataFrame
The intermediate DataFrame is immediately discarded, wasting memory. This is especially inefficient for wide tables where column-by-column overhead is significant.
### Does this PR introduce _any_ user-facing change?
No. This is a pure performance optimization with no API or behavior changes.
### How was this patch tested?
- Existing unit tests.
- Benchmark
**Benchmark setup:**
- **Hardware**: Driver memory 4GB, Executor memory 4GB
- **Configuration**: `spark.sql.execution.arrow.pyspark.enabled=false` (testing non-Arrow path)
- **Iterations**: 10 iterations per test case for statistical reliability
- **Test cases**:
- Simple (numeric columns)
- Mixed (int, string, double, boolean)
- Timestamp (date and timestamp types)
- Nested (struct and array types)
- Wide (5, 10, 50, 100 column counts)
### Performance Results
**General Benchmark** (10 iterations):
| Test Case | Rows | OLD → NEW | Speedup |
|------------|------|-----------|---------|
| simple | 1M | 1.376s → 1.383s | ≈ Tied |
| mixed | 1M | 2.396s → 2.553s | 6% slower |
| timestamp | 500K | 4.323s → 4.392s | ≈ Tied |
| nested | 100K | 0.558s → 0.580s | 4% slower |
| wide (50) | 100K | 1.458s → **1.141s** | **28% faster** 🚀 |
**Column Width Benchmark** (100K rows, 10 iterations):
| Columns | OLD → NEW | Speedup |
|---------|-----------|---------|
| 5 | 0.188s → 0.179s | 5% faster |
| 10 | 0.262s → 0.270s | ≈ Tied |
| 50 | 1.430s → **0.998s** | **43% faster** 🚀 |
| 100 | 3.320s → **1.988s** | **67% faster** 🚀 |
### Was this patch authored or co-authored using generative AI tooling?
Yes. Co-Generated-by Cursor
Closes #52897 from Yicong-Huang/SPARK-54182/refactor/avoid-intermedia-df-in-non-arrow-toPandas.
Authored-by: Yicong-Huang <[email protected]>
Signed-off-by: Ruifeng Zheng <[email protected]>df.toPandas
1 parent 674ed48 commit aad6c51
1 file changed
+17
-12
lines changed| Original file line number | Diff line number | Diff line change | |
|---|---|---|---|
| |||
18 | 18 | | |
19 | 19 | | |
20 | 20 | | |
| 21 | + | |
21 | 22 | | |
22 | 23 | | |
23 | 24 | | |
| |||
208 | 209 | | |
209 | 210 | | |
210 | 211 | | |
211 | | - | |
212 | | - | |
213 | | - | |
214 | | - | |
215 | | - | |
216 | | - | |
217 | 212 | | |
218 | | - | |
| 213 | + | |
219 | 214 | | |
220 | 215 | | |
221 | 216 | | |
222 | | - | |
| 217 | + | |
| 218 | + | |
| 219 | + | |
| 220 | + | |
| 221 | + | |
| 222 | + | |
| 223 | + | |
| 224 | + | |
| 225 | + | |
223 | 226 | | |
224 | 227 | | |
225 | 228 | | |
| |||
230 | 233 | | |
231 | 234 | | |
232 | 235 | | |
233 | | - | |
234 | | - | |
| 236 | + | |
| 237 | + | |
235 | 238 | | |
236 | | - | |
| 239 | + | |
| 240 | + | |
237 | 241 | | |
238 | | - | |
239 | 242 | | |
| 243 | + | |
| 244 | + | |
240 | 245 | | |
241 | 246 | | |
242 | 247 | | |
| |||
0 commit comments