/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/spill.rs
Line | Count | Source (jump to first uncovered line) |
1 | | // Licensed to the Apache Software Foundation (ASF) under one |
2 | | // or more contributor license agreements. See the NOTICE file |
3 | | // distributed with this work for additional information |
4 | | // regarding copyright ownership. The ASF licenses this file |
5 | | // to you under the Apache License, Version 2.0 (the |
6 | | // "License"); you may not use this file except in compliance |
7 | | // with the License. You may obtain a copy of the License at |
8 | | // |
9 | | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | | // |
11 | | // Unless required by applicable law or agreed to in writing, |
12 | | // software distributed under the License is distributed on an |
13 | | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | | // KIND, either express or implied. See the License for the |
15 | | // specific language governing permissions and limitations |
16 | | // under the License. |
17 | | |
18 | | //! Defines the spilling functions |
19 | | |
20 | | use std::fs::File; |
21 | | use std::io::BufReader; |
22 | | use std::path::{Path, PathBuf}; |
23 | | |
24 | | use arrow::datatypes::SchemaRef; |
25 | | use arrow::ipc::reader::FileReader; |
26 | | use arrow::record_batch::RecordBatch; |
27 | | use log::debug; |
28 | | use tokio::sync::mpsc::Sender; |
29 | | |
30 | | use datafusion_common::{exec_datafusion_err, Result}; |
31 | | use datafusion_execution::disk_manager::RefCountedTempFile; |
32 | | use datafusion_execution::memory_pool::human_readable_size; |
33 | | use datafusion_execution::SendableRecordBatchStream; |
34 | | |
35 | | use crate::common::IPCWriter; |
36 | | use crate::stream::RecordBatchReceiverStream; |
37 | | |
38 | | /// Read spilled batches from the disk |
39 | | /// |
40 | | /// `path` - temp file |
41 | | /// `schema` - batches schema, should be the same across batches |
42 | | /// `buffer` - internal buffer of capacity batches |
43 | 9 | pub(crate) fn read_spill_as_stream( |
44 | 9 | path: RefCountedTempFile, |
45 | 9 | schema: SchemaRef, |
46 | 9 | buffer: usize, |
47 | 9 | ) -> Result<SendableRecordBatchStream> { |
48 | 9 | let mut builder = RecordBatchReceiverStream::builder(schema, buffer); |
49 | 9 | let sender = builder.tx(); |
50 | 9 | |
51 | 9 | builder.spawn_blocking(move || read_spill(sender, path.path())); |
52 | 9 | |
53 | 9 | Ok(builder.build()) |
54 | 9 | } |
55 | | |
56 | | /// Spills in-memory `batches` to disk. |
57 | | /// |
58 | | /// Returns total number of the rows spilled to disk. |
59 | 42 | pub(crate) fn spill_record_batches( |
60 | 42 | batches: Vec<RecordBatch>, |
61 | 42 | path: PathBuf, |
62 | 42 | schema: SchemaRef, |
63 | 42 | ) -> Result<usize> { |
64 | 42 | let mut writer = IPCWriter::new(path.as_ref(), schema.as_ref())?0 ; |
65 | 85 | for batch43 in batches { |
66 | 43 | writer.write(&batch)?0 ; |
67 | | } |
68 | 42 | writer.finish()?0 ; |
69 | 42 | debug!( |
70 | 0 | "Spilled {} batches of total {} rows to disk, memory released {}", |
71 | 0 | writer.num_batches, |
72 | 0 | writer.num_rows, |
73 | 0 | human_readable_size(writer.num_bytes), |
74 | | ); |
75 | 42 | Ok(writer.num_rows) |
76 | 42 | } |
77 | | |
78 | 9 | fn read_spill(sender: Sender<Result<RecordBatch>>, path: &Path) -> Result<()> { |
79 | 9 | let file = BufReader::new(File::open(path)?0 ); |
80 | 9 | let reader = FileReader::try_new(file, None)?0 ; |
81 | 18 | for batch9 in reader { |
82 | 9 | sender |
83 | 9 | .blocking_send(batch.map_err(Into::into)) |
84 | 9 | .map_err(|e| exec_datafusion_err!("{e}")0 )?0 ; |
85 | | } |
86 | 9 | Ok(()) |
87 | 9 | } |
88 | | |
89 | | /// Spill the `RecordBatch` to disk as smaller batches |
90 | | /// split by `batch_size_rows` |
91 | 5 | pub fn spill_record_batch_by_size( |
92 | 5 | batch: &RecordBatch, |
93 | 5 | path: PathBuf, |
94 | 5 | schema: SchemaRef, |
95 | 5 | batch_size_rows: usize, |
96 | 5 | ) -> Result<()> { |
97 | 5 | let mut offset = 0; |
98 | 5 | let total_rows = batch.num_rows(); |
99 | 5 | let mut writer = IPCWriter::new(&path, schema.as_ref())?0 ; |
100 | | |
101 | 13 | while offset < total_rows { |
102 | 8 | let length = std::cmp::min(total_rows - offset, batch_size_rows); |
103 | 8 | let batch = batch.slice(offset, length); |
104 | 8 | offset += batch.num_rows(); |
105 | 8 | writer.write(&batch)?0 ; |
106 | | } |
107 | 5 | writer.finish()?0 ; |
108 | | |
109 | 5 | Ok(()) |
110 | 5 | } |
111 | | |
112 | | #[cfg(test)] |
113 | | mod tests { |
114 | | use crate::spill::{spill_record_batch_by_size, spill_record_batches}; |
115 | | use crate::test::build_table_i32; |
116 | | use datafusion_common::Result; |
117 | | use datafusion_execution::disk_manager::DiskManagerConfig; |
118 | | use datafusion_execution::DiskManager; |
119 | | use std::fs::File; |
120 | | use std::io::BufReader; |
121 | | use std::sync::Arc; |
122 | | |
123 | | #[test] |
124 | 1 | fn test_batch_spill_and_read() -> Result<()> { |
125 | 1 | let batch1 = build_table_i32( |
126 | 1 | ("a2", &vec![0, 1, 2]), |
127 | 1 | ("b2", &vec![3, 4, 5]), |
128 | 1 | ("c2", &vec![4, 5, 6]), |
129 | 1 | ); |
130 | 1 | |
131 | 1 | let batch2 = build_table_i32( |
132 | 1 | ("a2", &vec![10, 11, 12]), |
133 | 1 | ("b2", &vec![13, 14, 15]), |
134 | 1 | ("c2", &vec![14, 15, 16]), |
135 | 1 | ); |
136 | | |
137 | 1 | let disk_manager = DiskManager::try_new(DiskManagerConfig::NewOs)?0 ; |
138 | | |
139 | 1 | let spill_file = disk_manager.create_tmp_file("Test Spill")?0 ; |
140 | 1 | let schema = batch1.schema(); |
141 | 1 | let num_rows = batch1.num_rows() + batch2.num_rows(); |
142 | 1 | let cnt = spill_record_batches( |
143 | 1 | vec![batch1, batch2], |
144 | 1 | spill_file.path().into(), |
145 | 1 | Arc::clone(&schema), |
146 | 1 | ); |
147 | 1 | assert_eq!(cnt.unwrap(), num_rows); |
148 | | |
149 | 1 | let file = BufReader::new(File::open(spill_file.path())?0 ); |
150 | 1 | let reader = arrow::ipc::reader::FileReader::try_new(file, None)?0 ; |
151 | | |
152 | 1 | assert_eq!(reader.num_batches(), 2); |
153 | 1 | assert_eq!(reader.schema(), schema); |
154 | | |
155 | 1 | Ok(()) |
156 | 1 | } |
157 | | |
158 | | #[test] |
159 | 1 | fn test_batch_spill_by_size() -> Result<()> { |
160 | 1 | let batch1 = build_table_i32( |
161 | 1 | ("a2", &vec![0, 1, 2, 3]), |
162 | 1 | ("b2", &vec![3, 4, 5, 6]), |
163 | 1 | ("c2", &vec![4, 5, 6, 7]), |
164 | 1 | ); |
165 | | |
166 | 1 | let disk_manager = DiskManager::try_new(DiskManagerConfig::NewOs)?0 ; |
167 | | |
168 | 1 | let spill_file = disk_manager.create_tmp_file("Test Spill")?0 ; |
169 | 1 | let schema = batch1.schema(); |
170 | 1 | spill_record_batch_by_size( |
171 | 1 | &batch1, |
172 | 1 | spill_file.path().into(), |
173 | 1 | Arc::clone(&schema), |
174 | 1 | 1, |
175 | 1 | )?0 ; |
176 | | |
177 | 1 | let file = BufReader::new(File::open(spill_file.path())?0 ); |
178 | 1 | let reader = arrow::ipc::reader::FileReader::try_new(file, None)?0 ; |
179 | | |
180 | 1 | assert_eq!(reader.num_batches(), 4); |
181 | 1 | assert_eq!(reader.schema(), schema); |
182 | | |
183 | 1 | Ok(()) |
184 | 1 | } |
185 | | } |