parquet 0.5.12 → 0.6.0
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- checksums.yaml +4 -4
- data/Cargo.lock +295 -98
- data/Cargo.toml +1 -1
- data/Gemfile +1 -0
- data/README.md +94 -3
- data/ext/parquet/Cargo.toml +8 -5
- data/ext/parquet/src/adapter_ffi.rs +156 -0
- data/ext/parquet/src/lib.rs +13 -21
- data/ext/parquet-core/Cargo.toml +23 -0
- data/ext/parquet-core/src/arrow_conversion.rs +1133 -0
- data/ext/parquet-core/src/error.rs +163 -0
- data/ext/parquet-core/src/lib.rs +60 -0
- data/ext/parquet-core/src/reader.rs +263 -0
- data/ext/parquet-core/src/schema.rs +283 -0
- data/ext/parquet-core/src/test_utils.rs +308 -0
- data/ext/parquet-core/src/traits/mod.rs +5 -0
- data/ext/parquet-core/src/traits/schema.rs +151 -0
- data/ext/parquet-core/src/value.rs +209 -0
- data/ext/parquet-core/src/writer.rs +839 -0
- data/ext/parquet-core/tests/arrow_conversion_tests.rs +423 -0
- data/ext/parquet-core/tests/binary_data.rs +437 -0
- data/ext/parquet-core/tests/column_projection.rs +557 -0
- data/ext/parquet-core/tests/complex_types.rs +821 -0
- data/ext/parquet-core/tests/compression_tests.rs +434 -0
- data/ext/parquet-core/tests/concurrent_access.rs +430 -0
- data/ext/parquet-core/tests/decimal_tests.rs +488 -0
- data/ext/parquet-core/tests/edge_cases_corner_cases.rs +322 -0
- data/ext/parquet-core/tests/error_handling_comprehensive_tests.rs +547 -0
- data/ext/parquet-core/tests/null_handling_tests.rs +430 -0
- data/ext/parquet-core/tests/performance_memory.rs +181 -0
- data/ext/parquet-core/tests/primitive_types.rs +547 -0
- data/ext/parquet-core/tests/real_world_patterns.rs +777 -0
- data/ext/parquet-core/tests/roundtrip_correctness.rs +279 -0
- data/ext/parquet-core/tests/schema_comprehensive_tests.rs +534 -0
- data/ext/parquet-core/tests/temporal_tests.rs +518 -0
- data/ext/parquet-core/tests/test_helpers.rs +132 -0
- data/ext/parquet-core/tests/writer_tests.rs +545 -0
- data/ext/parquet-ruby-adapter/Cargo.toml +22 -0
- data/ext/parquet-ruby-adapter/build.rs +5 -0
- data/ext/parquet-ruby-adapter/examples/try_into_value_demo.rs +98 -0
- data/ext/parquet-ruby-adapter/src/batch_manager.rs +116 -0
- data/ext/parquet-ruby-adapter/src/chunk_reader.rs +237 -0
- data/ext/parquet-ruby-adapter/src/converter.rs +1685 -0
- data/ext/parquet-ruby-adapter/src/error.rs +148 -0
- data/ext/{parquet/src/ruby_reader.rs → parquet-ruby-adapter/src/io.rs} +190 -56
- data/ext/parquet-ruby-adapter/src/lib.rs +90 -0
- data/ext/parquet-ruby-adapter/src/logger.rs +64 -0
- data/ext/parquet-ruby-adapter/src/metadata.rs +427 -0
- data/ext/parquet-ruby-adapter/src/reader.rs +317 -0
- data/ext/parquet-ruby-adapter/src/schema.rs +810 -0
- data/ext/parquet-ruby-adapter/src/string_cache.rs +106 -0
- data/ext/parquet-ruby-adapter/src/try_into_value.rs +91 -0
- data/ext/parquet-ruby-adapter/src/types.rs +94 -0
- data/ext/parquet-ruby-adapter/src/utils.rs +186 -0
- data/ext/parquet-ruby-adapter/src/writer.rs +435 -0
- data/lib/parquet/schema.rb +19 -0
- data/lib/parquet/version.rb +1 -1
- metadata +50 -24
- data/ext/parquet/src/enumerator.rs +0 -68
- data/ext/parquet/src/header_cache.rs +0 -99
- data/ext/parquet/src/logger.rs +0 -171
- data/ext/parquet/src/reader/common.rs +0 -111
- data/ext/parquet/src/reader/mod.rs +0 -211
- data/ext/parquet/src/reader/parquet_column_reader.rs +0 -44
- data/ext/parquet/src/reader/parquet_row_reader.rs +0 -43
- data/ext/parquet/src/reader/unified/mod.rs +0 -363
- data/ext/parquet/src/types/core_types.rs +0 -120
- data/ext/parquet/src/types/mod.rs +0 -100
- data/ext/parquet/src/types/parquet_value.rs +0 -1275
- data/ext/parquet/src/types/record_types.rs +0 -603
- data/ext/parquet/src/types/schema_converter.rs +0 -290
- data/ext/parquet/src/types/schema_node.rs +0 -424
- data/ext/parquet/src/types/timestamp.rs +0 -285
- data/ext/parquet/src/types/type_conversion.rs +0 -1949
- data/ext/parquet/src/types/writer_types.rs +0 -329
- data/ext/parquet/src/utils.rs +0 -184
- data/ext/parquet/src/writer/mod.rs +0 -505
- data/ext/parquet/src/writer/write_columns.rs +0 -238
- data/ext/parquet/src/writer/write_rows.rs +0 -488
@@ -0,0 +1,839 @@
|
|
1
|
+
//! Core Parquet writing functionality
|
2
|
+
|
3
|
+
use crate::{
|
4
|
+
arrow_conversion::parquet_values_to_arrow_array, ParquetError, ParquetValue, Result, Schema,
|
5
|
+
SchemaNode,
|
6
|
+
};
|
7
|
+
use arrow::record_batch::RecordBatch;
|
8
|
+
use arrow_schema::{DataType, Field};
|
9
|
+
use parquet::arrow::ArrowWriter;
|
10
|
+
use parquet::basic::Compression;
|
11
|
+
use parquet::file::properties::WriterProperties;
|
12
|
+
use rand::Rng;
|
13
|
+
use std::sync::Arc;
|
14
|
+
|
15
|
+
// Default configuration constants
|
16
|
+
const DEFAULT_BATCH_SIZE: usize = 1000;
|
17
|
+
const DEFAULT_MEMORY_THRESHOLD: usize = 100 * 1024 * 1024; // 100MB
|
18
|
+
const DEFAULT_SAMPLE_SIZE: usize = 100;
|
19
|
+
const MIN_BATCH_SIZE: usize = 10;
|
20
|
+
const MIN_SAMPLES_FOR_ESTIMATE: usize = 10;
|
21
|
+
|
22
|
+
/// Builder for creating a configured Writer
|
23
|
+
pub struct WriterBuilder {
|
24
|
+
compression: Compression,
|
25
|
+
batch_size: Option<usize>,
|
26
|
+
memory_threshold: usize,
|
27
|
+
sample_size: usize,
|
28
|
+
}
|
29
|
+
|
30
|
+
impl Default for WriterBuilder {
|
31
|
+
fn default() -> Self {
|
32
|
+
Self {
|
33
|
+
compression: Compression::SNAPPY,
|
34
|
+
batch_size: None,
|
35
|
+
memory_threshold: DEFAULT_MEMORY_THRESHOLD,
|
36
|
+
sample_size: DEFAULT_SAMPLE_SIZE,
|
37
|
+
}
|
38
|
+
}
|
39
|
+
}
|
40
|
+
|
41
|
+
impl WriterBuilder {
|
42
|
+
/// Create a new WriterBuilder with default settings
|
43
|
+
pub fn new() -> Self {
|
44
|
+
Self::default()
|
45
|
+
}
|
46
|
+
|
47
|
+
/// Set the compression algorithm
|
48
|
+
pub fn with_compression(mut self, compression: Compression) -> Self {
|
49
|
+
self.compression = compression;
|
50
|
+
self
|
51
|
+
}
|
52
|
+
|
53
|
+
/// Set a fixed batch size (disables dynamic sizing)
|
54
|
+
pub fn with_batch_size(mut self, size: usize) -> Self {
|
55
|
+
self.batch_size = Some(size);
|
56
|
+
self
|
57
|
+
}
|
58
|
+
|
59
|
+
/// Set the memory threshold for flushing
|
60
|
+
pub fn with_memory_threshold(mut self, threshold: usize) -> Self {
|
61
|
+
self.memory_threshold = threshold;
|
62
|
+
self
|
63
|
+
}
|
64
|
+
|
65
|
+
/// Set the sample size for row size estimation
|
66
|
+
pub fn with_sample_size(mut self, size: usize) -> Self {
|
67
|
+
self.sample_size = size;
|
68
|
+
self
|
69
|
+
}
|
70
|
+
|
71
|
+
/// Build a Writer with the configured settings
|
72
|
+
pub fn build<W: std::io::Write + Send>(self, writer: W, schema: Schema) -> Result<Writer<W>> {
|
73
|
+
let arrow_schema = schema_to_arrow(&schema)?;
|
74
|
+
|
75
|
+
let props = WriterProperties::builder()
|
76
|
+
.set_compression(self.compression)
|
77
|
+
.build();
|
78
|
+
|
79
|
+
let arrow_writer = ArrowWriter::try_new(writer, arrow_schema.clone(), Some(props))?;
|
80
|
+
|
81
|
+
Ok(Writer {
|
82
|
+
arrow_writer: Some(arrow_writer),
|
83
|
+
arrow_schema,
|
84
|
+
buffered_rows: Vec::new(),
|
85
|
+
current_batch_size: self.batch_size.unwrap_or(DEFAULT_BATCH_SIZE),
|
86
|
+
memory_threshold: self.memory_threshold,
|
87
|
+
sample_size: self.sample_size,
|
88
|
+
size_samples: Vec::with_capacity(self.sample_size),
|
89
|
+
total_rows_written: 0,
|
90
|
+
fixed_batch_size: self.batch_size,
|
91
|
+
})
|
92
|
+
}
|
93
|
+
}
|
94
|
+
|
95
|
+
/// Core Parquet writer that works with any type implementing Write
|
96
|
+
pub struct Writer<W: std::io::Write> {
|
97
|
+
arrow_writer: Option<ArrowWriter<W>>,
|
98
|
+
arrow_schema: Arc<arrow_schema::Schema>,
|
99
|
+
buffered_rows: Vec<Vec<ParquetValue>>,
|
100
|
+
current_batch_size: usize,
|
101
|
+
memory_threshold: usize,
|
102
|
+
sample_size: usize,
|
103
|
+
size_samples: Vec<usize>,
|
104
|
+
total_rows_written: usize,
|
105
|
+
fixed_batch_size: Option<usize>,
|
106
|
+
}
|
107
|
+
|
108
|
+
impl<W> Writer<W>
|
109
|
+
where
|
110
|
+
W: std::io::Write + Send,
|
111
|
+
{
|
112
|
+
/// Create a new writer with default settings
|
113
|
+
pub fn new(writer: W, schema: Schema) -> Result<Self> {
|
114
|
+
WriterBuilder::new().build(writer, schema)
|
115
|
+
}
|
116
|
+
|
117
|
+
/// Create a new writer with custom properties
|
118
|
+
pub fn new_with_properties(writer: W, schema: Schema, props: WriterProperties) -> Result<Self> {
|
119
|
+
let arrow_schema = schema_to_arrow(&schema)?;
|
120
|
+
|
121
|
+
let arrow_writer = ArrowWriter::try_new(writer, arrow_schema.clone(), Some(props))?;
|
122
|
+
|
123
|
+
Ok(Self {
|
124
|
+
arrow_writer: Some(arrow_writer),
|
125
|
+
arrow_schema,
|
126
|
+
buffered_rows: Vec::new(),
|
127
|
+
current_batch_size: DEFAULT_BATCH_SIZE,
|
128
|
+
memory_threshold: DEFAULT_MEMORY_THRESHOLD,
|
129
|
+
sample_size: DEFAULT_SAMPLE_SIZE,
|
130
|
+
size_samples: Vec::with_capacity(DEFAULT_SAMPLE_SIZE),
|
131
|
+
total_rows_written: 0,
|
132
|
+
fixed_batch_size: None,
|
133
|
+
})
|
134
|
+
}
|
135
|
+
|
136
|
+
/// Write a batch of rows to the Parquet file
|
137
|
+
///
|
138
|
+
/// Each row is a vector of values corresponding to the schema fields
|
139
|
+
pub fn write_rows(&mut self, rows: Vec<Vec<ParquetValue>>) -> Result<()> {
|
140
|
+
for row in rows {
|
141
|
+
self.write_row(row)?;
|
142
|
+
}
|
143
|
+
Ok(())
|
144
|
+
}
|
145
|
+
|
146
|
+
/// Write a single row to the Parquet file
|
147
|
+
///
|
148
|
+
/// Rows are buffered internally and written in batches to optimize memory usage
|
149
|
+
pub fn write_row(&mut self, row: Vec<ParquetValue>) -> Result<()> {
|
150
|
+
// Validate row length
|
151
|
+
let num_cols = self.arrow_schema.fields().len();
|
152
|
+
if row.len() != num_cols {
|
153
|
+
return Err(ParquetError::Schema(format!(
|
154
|
+
"Row has {} values but schema has {} fields",
|
155
|
+
row.len(),
|
156
|
+
num_cols
|
157
|
+
)));
|
158
|
+
}
|
159
|
+
|
160
|
+
// Validate each value matches its schema
|
161
|
+
for (idx, (value, field)) in row.iter().zip(self.arrow_schema.fields()).enumerate() {
|
162
|
+
validate_value_against_field(value, field, &format!("row[{}]", idx))?;
|
163
|
+
}
|
164
|
+
|
165
|
+
// Sample row size for dynamic batch sizing
|
166
|
+
if self.fixed_batch_size.is_none() {
|
167
|
+
self.sample_row_size(&row)?;
|
168
|
+
}
|
169
|
+
|
170
|
+
// Add row to buffer
|
171
|
+
self.buffered_rows.push(row);
|
172
|
+
|
173
|
+
// Check if we need to flush
|
174
|
+
if self.buffered_rows.len() >= self.current_batch_size {
|
175
|
+
self.flush_buffered_rows()?;
|
176
|
+
}
|
177
|
+
|
178
|
+
Ok(())
|
179
|
+
}
|
180
|
+
|
181
|
+
/// Sample row size for dynamic batch sizing using reservoir sampling
|
182
|
+
fn sample_row_size(&mut self, row: &[ParquetValue]) -> Result<()> {
|
183
|
+
let row_size = self.estimate_row_size(row)?;
|
184
|
+
|
185
|
+
if self.size_samples.len() < self.sample_size {
|
186
|
+
self.size_samples.push(row_size);
|
187
|
+
} else {
|
188
|
+
// Reservoir sampling
|
189
|
+
let mut rng = rand::rng();
|
190
|
+
let idx = rng.random_range(0..=self.total_rows_written);
|
191
|
+
if idx < self.sample_size {
|
192
|
+
self.size_samples[idx] = row_size;
|
193
|
+
}
|
194
|
+
}
|
195
|
+
|
196
|
+
// Update batch size if we have enough samples
|
197
|
+
if self.size_samples.len() >= MIN_SAMPLES_FOR_ESTIMATE {
|
198
|
+
self.update_batch_size();
|
199
|
+
}
|
200
|
+
|
201
|
+
Ok(())
|
202
|
+
}
|
203
|
+
|
204
|
+
/// Estimate the memory size of a single row
|
205
|
+
fn estimate_row_size(&self, row: &[ParquetValue]) -> Result<usize> {
|
206
|
+
let mut size = 0;
|
207
|
+
for (idx, value) in row.iter().enumerate() {
|
208
|
+
let field = &self.arrow_schema.fields()[idx];
|
209
|
+
size += self.estimate_value_size(value, field.data_type())?;
|
210
|
+
}
|
211
|
+
Ok(size)
|
212
|
+
}
|
213
|
+
|
214
|
+
/// Estimate the memory footprint of a single value
|
215
|
+
#[allow(clippy::only_used_in_recursion)]
|
216
|
+
fn estimate_value_size(&self, value: &ParquetValue, data_type: &DataType) -> Result<usize> {
|
217
|
+
use ParquetValue::*;
|
218
|
+
|
219
|
+
Ok(match (value, data_type) {
|
220
|
+
(Null, _) => 0,
|
221
|
+
|
222
|
+
// Fixed size types
|
223
|
+
(Boolean(_), DataType::Boolean) => 1,
|
224
|
+
(Int8(_), DataType::Int8) => 1,
|
225
|
+
(UInt8(_), DataType::UInt8) => 1,
|
226
|
+
(Int16(_), DataType::Int16) => 2,
|
227
|
+
(UInt16(_), DataType::UInt16) => 2,
|
228
|
+
(Int32(_), DataType::Int32) => 4,
|
229
|
+
(UInt32(_), DataType::UInt32) => 4,
|
230
|
+
(Float32(_), DataType::Float32) => 4,
|
231
|
+
(Int64(_), DataType::Int64) => 8,
|
232
|
+
(UInt64(_), DataType::UInt64) => 8,
|
233
|
+
(Float64(_), DataType::Float64) => 8,
|
234
|
+
(Date32(_), DataType::Date32) => 4,
|
235
|
+
(Date64(_), DataType::Date64) => 8,
|
236
|
+
(TimeMillis(_), DataType::Time32(_)) => 4,
|
237
|
+
(TimeMicros(_), DataType::Time64(_)) => 8,
|
238
|
+
(TimestampSecond(_, _), DataType::Timestamp(_, _)) => 8,
|
239
|
+
(TimestampMillis(_, _), DataType::Timestamp(_, _)) => 8,
|
240
|
+
(TimestampMicros(_, _), DataType::Timestamp(_, _)) => 8,
|
241
|
+
(TimestampNanos(_, _), DataType::Timestamp(_, _)) => 8,
|
242
|
+
(Decimal128(_, _), DataType::Decimal128(_, _)) => 16,
|
243
|
+
|
244
|
+
// Variable size types
|
245
|
+
(String(s), DataType::Utf8) => s.len() + std::mem::size_of::<usize>() * 3,
|
246
|
+
(Bytes(b), DataType::Binary) => b.len() + std::mem::size_of::<usize>() * 3,
|
247
|
+
(Bytes(_), DataType::FixedSizeBinary(len)) => *len as usize,
|
248
|
+
|
249
|
+
(Decimal256(v, _), DataType::Decimal256(_, _)) => {
|
250
|
+
let bytes = v.to_signed_bytes_le();
|
251
|
+
32 + bytes.len()
|
252
|
+
}
|
253
|
+
|
254
|
+
// Complex types
|
255
|
+
(List(items), DataType::List(field)) => {
|
256
|
+
let base_size = std::mem::size_of::<usize>() * 3;
|
257
|
+
if items.is_empty() {
|
258
|
+
base_size
|
259
|
+
} else {
|
260
|
+
// Sample up to 5 elements
|
261
|
+
let sample_count = items.len().min(5);
|
262
|
+
let sample_size: usize = items
|
263
|
+
.iter()
|
264
|
+
.take(sample_count)
|
265
|
+
.map(|item| {
|
266
|
+
self.estimate_value_size(item, field.data_type())
|
267
|
+
.unwrap_or(0)
|
268
|
+
})
|
269
|
+
.sum();
|
270
|
+
let avg_size = sample_size / sample_count;
|
271
|
+
base_size + (avg_size * items.len())
|
272
|
+
}
|
273
|
+
}
|
274
|
+
|
275
|
+
(Map(entries), DataType::Map(entries_field, _)) => {
|
276
|
+
if let DataType::Struct(fields) = entries_field.data_type() {
|
277
|
+
let base_size = std::mem::size_of::<usize>() * 4;
|
278
|
+
if entries.is_empty() || fields.len() < 2 {
|
279
|
+
base_size
|
280
|
+
} else {
|
281
|
+
// Sample up to 5 entries
|
282
|
+
let sample_count = entries.len().min(5);
|
283
|
+
let mut total_size = base_size;
|
284
|
+
|
285
|
+
for (key, val) in entries.iter().take(sample_count) {
|
286
|
+
total_size += self
|
287
|
+
.estimate_value_size(key, fields[0].data_type())
|
288
|
+
.unwrap_or(0);
|
289
|
+
total_size += self
|
290
|
+
.estimate_value_size(val, fields[1].data_type())
|
291
|
+
.unwrap_or(0);
|
292
|
+
}
|
293
|
+
|
294
|
+
let avg_entry_size = (total_size - base_size) / sample_count;
|
295
|
+
base_size + (avg_entry_size * entries.len())
|
296
|
+
}
|
297
|
+
} else {
|
298
|
+
100 // Default estimate
|
299
|
+
}
|
300
|
+
}
|
301
|
+
|
302
|
+
(Record(fields), DataType::Struct(schema_fields)) => {
|
303
|
+
let base_size = std::mem::size_of::<usize>() * 3;
|
304
|
+
let field_sizes: usize = fields
|
305
|
+
.iter()
|
306
|
+
.zip(schema_fields.iter())
|
307
|
+
.map(|((_, val), field)| {
|
308
|
+
self.estimate_value_size(val, field.data_type())
|
309
|
+
.unwrap_or(0)
|
310
|
+
})
|
311
|
+
.sum();
|
312
|
+
base_size + field_sizes
|
313
|
+
}
|
314
|
+
|
315
|
+
_ => 100, // Default estimate for mismatched types
|
316
|
+
})
|
317
|
+
}
|
318
|
+
|
319
|
+
/// Update dynamic batch size based on current samples
|
320
|
+
fn update_batch_size(&mut self) {
|
321
|
+
if self.size_samples.is_empty() {
|
322
|
+
return;
|
323
|
+
}
|
324
|
+
|
325
|
+
let total_size: usize = self.size_samples.iter().sum();
|
326
|
+
let avg_row_size = (total_size as f64 / self.size_samples.len() as f64).max(1.0);
|
327
|
+
let suggested_batch_size = (self.memory_threshold as f64 / avg_row_size).floor() as usize;
|
328
|
+
self.current_batch_size = suggested_batch_size.max(MIN_BATCH_SIZE);
|
329
|
+
}
|
330
|
+
|
331
|
+
/// Flush buffered rows to the Parquet file
|
332
|
+
fn flush_buffered_rows(&mut self) -> Result<()> {
|
333
|
+
if self.buffered_rows.is_empty() {
|
334
|
+
return Ok(());
|
335
|
+
}
|
336
|
+
|
337
|
+
let rows = std::mem::take(&mut self.buffered_rows);
|
338
|
+
let num_rows = rows.len();
|
339
|
+
self.total_rows_written += num_rows;
|
340
|
+
|
341
|
+
// Convert rows to columnar format
|
342
|
+
let num_cols = self.arrow_schema.fields().len();
|
343
|
+
let mut columns: Vec<Vec<ParquetValue>> = vec![Vec::with_capacity(num_rows); num_cols];
|
344
|
+
|
345
|
+
// Transpose rows to columns
|
346
|
+
for row in rows {
|
347
|
+
for (col_idx, value) in row.into_iter().enumerate() {
|
348
|
+
columns[col_idx].push(value);
|
349
|
+
}
|
350
|
+
}
|
351
|
+
|
352
|
+
// Convert columns to Arrow arrays
|
353
|
+
let arrow_columns = columns
|
354
|
+
.into_iter()
|
355
|
+
.zip(self.arrow_schema.fields())
|
356
|
+
.map(|(values, field)| parquet_values_to_arrow_array(values, field))
|
357
|
+
.collect::<Result<Vec<_>>>()?;
|
358
|
+
|
359
|
+
// Create RecordBatch
|
360
|
+
let batch = RecordBatch::try_new(self.arrow_schema.clone(), arrow_columns)?;
|
361
|
+
|
362
|
+
// Write the batch
|
363
|
+
if let Some(writer) = &mut self.arrow_writer {
|
364
|
+
writer.write(&batch)?;
|
365
|
+
|
366
|
+
// Check if we need to flush based on memory usage
|
367
|
+
if writer.in_progress_size() >= self.memory_threshold {
|
368
|
+
writer.flush()?;
|
369
|
+
}
|
370
|
+
} else {
|
371
|
+
return Err(ParquetError::Io(std::io::Error::new(
|
372
|
+
std::io::ErrorKind::Other,
|
373
|
+
"Writer has been closed",
|
374
|
+
)));
|
375
|
+
}
|
376
|
+
|
377
|
+
Ok(())
|
378
|
+
}
|
379
|
+
|
380
|
+
/// Write columns to the Parquet file
|
381
|
+
///
|
382
|
+
/// Each element is a tuple of (column_name, values)
|
383
|
+
pub fn write_columns(&mut self, columns: Vec<(String, Vec<ParquetValue>)>) -> Result<()> {
|
384
|
+
if columns.is_empty() {
|
385
|
+
return Ok(());
|
386
|
+
}
|
387
|
+
|
388
|
+
// Verify column names match schema
|
389
|
+
let schema_fields = self.arrow_schema.fields();
|
390
|
+
if columns.len() != schema_fields.len() {
|
391
|
+
return Err(ParquetError::Schema(format!(
|
392
|
+
"Provided {} columns but schema has {} fields",
|
393
|
+
columns.len(),
|
394
|
+
schema_fields.len()
|
395
|
+
)));
|
396
|
+
}
|
397
|
+
|
398
|
+
// Sort columns to match schema order and convert to arrays
|
399
|
+
let mut arrow_columns = Vec::with_capacity(columns.len());
|
400
|
+
|
401
|
+
for field in schema_fields {
|
402
|
+
let column_data = columns
|
403
|
+
.iter()
|
404
|
+
.find(|(name, _)| name == field.name())
|
405
|
+
.ok_or_else(|| ParquetError::Schema(format!("Missing column: {}", field.name())))?;
|
406
|
+
|
407
|
+
let array = parquet_values_to_arrow_array(column_data.1.clone(), field)?;
|
408
|
+
arrow_columns.push(array);
|
409
|
+
}
|
410
|
+
|
411
|
+
// Create RecordBatch
|
412
|
+
let batch = RecordBatch::try_new(self.arrow_schema.clone(), arrow_columns)?;
|
413
|
+
|
414
|
+
// Write the batch
|
415
|
+
if let Some(writer) = &mut self.arrow_writer {
|
416
|
+
writer.write(&batch)?;
|
417
|
+
} else {
|
418
|
+
return Err(ParquetError::Io(std::io::Error::new(
|
419
|
+
std::io::ErrorKind::Other,
|
420
|
+
"Writer has been closed",
|
421
|
+
)));
|
422
|
+
}
|
423
|
+
|
424
|
+
Ok(())
|
425
|
+
}
|
426
|
+
|
427
|
+
/// Flush any buffered data
|
428
|
+
pub fn flush(&mut self) -> Result<()> {
|
429
|
+
// First flush any buffered rows
|
430
|
+
self.flush_buffered_rows()?;
|
431
|
+
|
432
|
+
// Then flush the arrow writer
|
433
|
+
if let Some(writer) = &mut self.arrow_writer {
|
434
|
+
writer.flush()?;
|
435
|
+
}
|
436
|
+
Ok(())
|
437
|
+
}
|
438
|
+
|
439
|
+
/// Close the writer and write the file footer
|
440
|
+
///
|
441
|
+
/// This must be called to finalize the Parquet file
|
442
|
+
pub fn close(mut self) -> Result<()> {
|
443
|
+
// Flush any remaining buffered rows
|
444
|
+
self.flush_buffered_rows()?;
|
445
|
+
|
446
|
+
// Close the arrow writer
|
447
|
+
if let Some(writer) = self.arrow_writer.take() {
|
448
|
+
writer.close()?;
|
449
|
+
}
|
450
|
+
Ok(())
|
451
|
+
}
|
452
|
+
}
|
453
|
+
|
454
|
+
/// Validate a value against its field schema
|
455
|
+
fn validate_value_against_field(value: &ParquetValue, field: &Field, path: &str) -> Result<()> {
|
456
|
+
use ParquetValue::*;
|
457
|
+
|
458
|
+
// Null handling
|
459
|
+
if matches!(value, Null) {
|
460
|
+
if !field.is_nullable() {
|
461
|
+
return Err(ParquetError::Schema(format!(
|
462
|
+
"Found null value for non-nullable field at {}",
|
463
|
+
path
|
464
|
+
)));
|
465
|
+
}
|
466
|
+
return Ok(());
|
467
|
+
}
|
468
|
+
|
469
|
+
// Type validation
|
470
|
+
match (value, field.data_type()) {
|
471
|
+
// Boolean
|
472
|
+
(Boolean(_), DataType::Boolean) => Ok(()),
|
473
|
+
|
474
|
+
// Integer types
|
475
|
+
(Int8(_), DataType::Int8) => Ok(()),
|
476
|
+
(Int16(_), DataType::Int16) => Ok(()),
|
477
|
+
(Int32(_), DataType::Int32) => Ok(()),
|
478
|
+
(Int64(_), DataType::Int64) => Ok(()),
|
479
|
+
(UInt8(_), DataType::UInt8) => Ok(()),
|
480
|
+
(UInt16(_), DataType::UInt16) => Ok(()),
|
481
|
+
(UInt32(_), DataType::UInt32) => Ok(()),
|
482
|
+
(UInt64(_), DataType::UInt64) => Ok(()),
|
483
|
+
|
484
|
+
// Float types
|
485
|
+
(Float16(_), DataType::Float16) => Ok(()),
|
486
|
+
(Float32(_), DataType::Float32) => Ok(()),
|
487
|
+
(Float64(_), DataType::Float64) => Ok(()),
|
488
|
+
|
489
|
+
// String and binary
|
490
|
+
(String(_), DataType::Utf8) => Ok(()),
|
491
|
+
(Bytes(_), DataType::Binary) => Ok(()),
|
492
|
+
(Bytes(_), DataType::FixedSizeBinary(_)) => Ok(()), // Size check done during conversion
|
493
|
+
|
494
|
+
// Date/time types
|
495
|
+
(Date32(_), DataType::Date32) => Ok(()),
|
496
|
+
(Date64(_), DataType::Date64) => Ok(()),
|
497
|
+
(TimeMillis(_), DataType::Time32(_)) => Ok(()),
|
498
|
+
(TimeMicros(_), DataType::Time64(_)) => Ok(()),
|
499
|
+
(TimestampSecond(_, _), DataType::Timestamp(_, _)) => Ok(()),
|
500
|
+
(TimestampMillis(_, _), DataType::Timestamp(_, _)) => Ok(()),
|
501
|
+
(TimestampMicros(_, _), DataType::Timestamp(_, _)) => Ok(()),
|
502
|
+
(TimestampNanos(_, _), DataType::Timestamp(_, _)) => Ok(()),
|
503
|
+
|
504
|
+
// Decimal types
|
505
|
+
(Decimal128(_, _), DataType::Decimal128(_, _)) => Ok(()),
|
506
|
+
(Decimal256(_, _), DataType::Decimal256(_, _)) => Ok(()),
|
507
|
+
|
508
|
+
// List type
|
509
|
+
(List(items), DataType::List(item_field)) => {
|
510
|
+
for (idx, item) in items.iter().enumerate() {
|
511
|
+
validate_value_against_field(item, item_field, &format!("{}[{}]", path, idx))?;
|
512
|
+
}
|
513
|
+
Ok(())
|
514
|
+
}
|
515
|
+
|
516
|
+
// Map type
|
517
|
+
(Map(entries), DataType::Map(entries_field, _)) => {
|
518
|
+
if let DataType::Struct(fields) = entries_field.data_type() {
|
519
|
+
if fields.len() >= 2 {
|
520
|
+
let key_field = &fields[0];
|
521
|
+
let value_field = &fields[1];
|
522
|
+
|
523
|
+
for (idx, (key, val)) in entries.iter().enumerate() {
|
524
|
+
validate_value_against_field(
|
525
|
+
key,
|
526
|
+
key_field,
|
527
|
+
&format!("{}.key[{}]", path, idx),
|
528
|
+
)?;
|
529
|
+
validate_value_against_field(
|
530
|
+
val,
|
531
|
+
value_field,
|
532
|
+
&format!("{}.value[{}]", path, idx),
|
533
|
+
)?;
|
534
|
+
}
|
535
|
+
}
|
536
|
+
}
|
537
|
+
Ok(())
|
538
|
+
}
|
539
|
+
|
540
|
+
// Struct type
|
541
|
+
(Record(record_fields), DataType::Struct(schema_fields)) => {
|
542
|
+
for field in schema_fields {
|
543
|
+
let field_name = field.name();
|
544
|
+
if let Some(value) = record_fields.get(field_name.as_str()) {
|
545
|
+
validate_value_against_field(
|
546
|
+
value,
|
547
|
+
field,
|
548
|
+
&format!("{}.{}", path, field_name),
|
549
|
+
)?;
|
550
|
+
} else if !field.is_nullable() {
|
551
|
+
return Err(ParquetError::Schema(format!(
|
552
|
+
"Required field '{}' is missing in struct at {}",
|
553
|
+
field_name, path
|
554
|
+
)));
|
555
|
+
}
|
556
|
+
}
|
557
|
+
Ok(())
|
558
|
+
}
|
559
|
+
|
560
|
+
// Type mismatch
|
561
|
+
(value, expected_type) => Err(ParquetError::Schema(format!(
|
562
|
+
"Type mismatch at {}: expected {:?}, got {:?}",
|
563
|
+
path,
|
564
|
+
expected_type,
|
565
|
+
value.type_name()
|
566
|
+
))),
|
567
|
+
}
|
568
|
+
}
|
569
|
+
|
570
|
+
/// Convert our Schema to Arrow Schema
|
571
|
+
fn schema_to_arrow(schema: &Schema) -> Result<Arc<arrow_schema::Schema>> {
|
572
|
+
match &schema.root {
|
573
|
+
SchemaNode::Struct { fields, .. } => {
|
574
|
+
let arrow_fields = fields
|
575
|
+
.iter()
|
576
|
+
.map(schema_node_to_arrow_field)
|
577
|
+
.collect::<Result<Vec<_>>>()?;
|
578
|
+
|
579
|
+
Ok(Arc::new(arrow_schema::Schema::new(arrow_fields)))
|
580
|
+
}
|
581
|
+
_ => Err(ParquetError::Schema(
|
582
|
+
"Root schema node must be a struct".to_string(),
|
583
|
+
)),
|
584
|
+
}
|
585
|
+
}
|
586
|
+
|
587
|
+
/// Convert a SchemaNode to an Arrow Field
|
588
|
+
fn schema_node_to_arrow_field(node: &SchemaNode) -> Result<Field> {
|
589
|
+
match node {
|
590
|
+
SchemaNode::Primitive {
|
591
|
+
name,
|
592
|
+
primitive_type,
|
593
|
+
nullable,
|
594
|
+
..
|
595
|
+
} => {
|
596
|
+
let data_type = primitive_type_to_arrow(primitive_type)?;
|
597
|
+
Ok(Field::new(name, data_type, *nullable))
|
598
|
+
}
|
599
|
+
SchemaNode::List {
|
600
|
+
name,
|
601
|
+
item,
|
602
|
+
nullable,
|
603
|
+
} => {
|
604
|
+
let item_field = schema_node_to_arrow_field(item)?;
|
605
|
+
let list_type = DataType::List(Arc::new(Field::new(
|
606
|
+
"item",
|
607
|
+
item_field.data_type().clone(),
|
608
|
+
true,
|
609
|
+
)));
|
610
|
+
Ok(Field::new(name, list_type, *nullable))
|
611
|
+
}
|
612
|
+
SchemaNode::Map {
|
613
|
+
name,
|
614
|
+
key,
|
615
|
+
value,
|
616
|
+
nullable,
|
617
|
+
} => {
|
618
|
+
let key_field = schema_node_to_arrow_field(key)?;
|
619
|
+
let value_field = schema_node_to_arrow_field(value)?;
|
620
|
+
|
621
|
+
let struct_fields = vec![
|
622
|
+
Field::new("key", key_field.data_type().clone(), false),
|
623
|
+
Field::new("value", value_field.data_type().clone(), true),
|
624
|
+
];
|
625
|
+
|
626
|
+
let map_type = DataType::Map(
|
627
|
+
Arc::new(Field::new(
|
628
|
+
"entries",
|
629
|
+
DataType::Struct(struct_fields.into()),
|
630
|
+
false,
|
631
|
+
)),
|
632
|
+
false, // keys_sorted
|
633
|
+
);
|
634
|
+
|
635
|
+
Ok(Field::new(name, map_type, *nullable))
|
636
|
+
}
|
637
|
+
SchemaNode::Struct {
|
638
|
+
name,
|
639
|
+
fields,
|
640
|
+
nullable,
|
641
|
+
} => {
|
642
|
+
let struct_fields = fields
|
643
|
+
.iter()
|
644
|
+
.map(schema_node_to_arrow_field)
|
645
|
+
.collect::<Result<Vec<_>>>()?;
|
646
|
+
|
647
|
+
let struct_type = DataType::Struct(struct_fields.into());
|
648
|
+
Ok(Field::new(name, struct_type, *nullable))
|
649
|
+
}
|
650
|
+
}
|
651
|
+
}
|
652
|
+
|
653
|
+
/// Convert PrimitiveType to Arrow DataType
|
654
|
+
fn primitive_type_to_arrow(ptype: &crate::PrimitiveType) -> Result<DataType> {
|
655
|
+
use crate::PrimitiveType::*;
|
656
|
+
|
657
|
+
Ok(match ptype {
|
658
|
+
Boolean => DataType::Boolean,
|
659
|
+
Int8 => DataType::Int8,
|
660
|
+
Int16 => DataType::Int16,
|
661
|
+
Int32 => DataType::Int32,
|
662
|
+
Int64 => DataType::Int64,
|
663
|
+
UInt8 => DataType::UInt8,
|
664
|
+
UInt16 => DataType::UInt16,
|
665
|
+
UInt32 => DataType::UInt32,
|
666
|
+
UInt64 => DataType::UInt64,
|
667
|
+
Float32 => DataType::Float32,
|
668
|
+
Float64 => DataType::Float64,
|
669
|
+
String => DataType::Utf8,
|
670
|
+
Binary => DataType::Binary,
|
671
|
+
Date32 => DataType::Date32,
|
672
|
+
TimeMillis => DataType::Time32(arrow_schema::TimeUnit::Millisecond),
|
673
|
+
TimeMicros => DataType::Time64(arrow_schema::TimeUnit::Microsecond),
|
674
|
+
TimestampMillis(tz) => DataType::Timestamp(
|
675
|
+
arrow_schema::TimeUnit::Millisecond,
|
676
|
+
// PARQUET SPEC: ANY timezone (e.g., "+09:00", "America/New_York") means
|
677
|
+
// UTC-normalized storage (isAdjustedToUTC = true). Original timezone is lost.
|
678
|
+
tz.as_ref().map(|_| Arc::from("UTC")),
|
679
|
+
),
|
680
|
+
TimestampMicros(tz) => DataType::Timestamp(
|
681
|
+
arrow_schema::TimeUnit::Microsecond,
|
682
|
+
// PARQUET SPEC: ANY timezone (e.g., "+09:00", "America/New_York") means
|
683
|
+
// UTC-normalized storage (isAdjustedToUTC = true). Original timezone is lost.
|
684
|
+
tz.as_ref().map(|_| Arc::from("UTC")),
|
685
|
+
),
|
686
|
+
Decimal128(precision, scale) => DataType::Decimal128(*precision, *scale),
|
687
|
+
Decimal256(precision, scale) => DataType::Decimal256(*precision, *scale),
|
688
|
+
Date64 => DataType::Date64,
|
689
|
+
TimestampSecond(tz) => DataType::Timestamp(
|
690
|
+
arrow_schema::TimeUnit::Second,
|
691
|
+
// PARQUET SPEC: ANY timezone (e.g., "+09:00", "America/New_York") means
|
692
|
+
// UTC-normalized storage (isAdjustedToUTC = true). Original timezone is lost.
|
693
|
+
tz.as_ref().map(|_| Arc::from("UTC")),
|
694
|
+
),
|
695
|
+
TimestampNanos(tz) => DataType::Timestamp(
|
696
|
+
arrow_schema::TimeUnit::Nanosecond,
|
697
|
+
// PARQUET SPEC: ANY timezone (e.g., "+09:00", "America/New_York") means
|
698
|
+
// UTC-normalized storage (isAdjustedToUTC = true). Original timezone is lost.
|
699
|
+
tz.as_ref().map(|_| Arc::from("UTC")),
|
700
|
+
),
|
701
|
+
FixedLenByteArray(len) => DataType::FixedSizeBinary(*len),
|
702
|
+
})
|
703
|
+
}
|
704
|
+
|
705
|
+
#[cfg(test)]
|
706
|
+
mod tests {
|
707
|
+
use super::*;
|
708
|
+
use crate::SchemaBuilder;
|
709
|
+
|
710
|
+
#[test]
|
711
|
+
fn test_writer_creation() {
|
712
|
+
let schema = SchemaBuilder::new()
|
713
|
+
.with_root(SchemaNode::Struct {
|
714
|
+
name: "root".to_string(),
|
715
|
+
nullable: false,
|
716
|
+
fields: vec![SchemaNode::Primitive {
|
717
|
+
name: "id".to_string(),
|
718
|
+
primitive_type: crate::PrimitiveType::Int64,
|
719
|
+
nullable: false,
|
720
|
+
format: None,
|
721
|
+
}],
|
722
|
+
})
|
723
|
+
.build()
|
724
|
+
.unwrap();
|
725
|
+
|
726
|
+
let buffer = Vec::new();
|
727
|
+
let _writer = Writer::new(buffer, schema).unwrap();
|
728
|
+
}
|
729
|
+
|
730
|
+
#[test]
|
731
|
+
fn test_writer_builder() {
|
732
|
+
let schema = SchemaBuilder::new()
|
733
|
+
.with_root(SchemaNode::Struct {
|
734
|
+
name: "root".to_string(),
|
735
|
+
nullable: false,
|
736
|
+
fields: vec![SchemaNode::Primitive {
|
737
|
+
name: "id".to_string(),
|
738
|
+
primitive_type: crate::PrimitiveType::Int64,
|
739
|
+
nullable: false,
|
740
|
+
format: None,
|
741
|
+
}],
|
742
|
+
})
|
743
|
+
.build()
|
744
|
+
.unwrap();
|
745
|
+
|
746
|
+
let buffer = Vec::new();
|
747
|
+
let _writer = WriterBuilder::new()
|
748
|
+
.with_compression(Compression::ZSTD(parquet::basic::ZstdLevel::default()))
|
749
|
+
.with_batch_size(500)
|
750
|
+
.with_memory_threshold(50 * 1024 * 1024)
|
751
|
+
.with_sample_size(50)
|
752
|
+
.build(buffer, schema)
|
753
|
+
.unwrap();
|
754
|
+
}
|
755
|
+
|
756
|
+
#[test]
|
757
|
+
fn test_buffered_writing() {
|
758
|
+
let schema = SchemaBuilder::new()
|
759
|
+
.with_root(SchemaNode::Struct {
|
760
|
+
name: "root".to_string(),
|
761
|
+
nullable: false,
|
762
|
+
fields: vec![
|
763
|
+
SchemaNode::Primitive {
|
764
|
+
name: "id".to_string(),
|
765
|
+
primitive_type: crate::PrimitiveType::Int64,
|
766
|
+
nullable: false,
|
767
|
+
format: None,
|
768
|
+
},
|
769
|
+
SchemaNode::Primitive {
|
770
|
+
name: "name".to_string(),
|
771
|
+
primitive_type: crate::PrimitiveType::String,
|
772
|
+
nullable: true,
|
773
|
+
format: None,
|
774
|
+
},
|
775
|
+
],
|
776
|
+
})
|
777
|
+
.build()
|
778
|
+
.unwrap();
|
779
|
+
|
780
|
+
let buffer = Vec::new();
|
781
|
+
let mut writer = WriterBuilder::new()
|
782
|
+
.with_batch_size(10) // Small batch for testing
|
783
|
+
.build(buffer, schema)
|
784
|
+
.unwrap();
|
785
|
+
|
786
|
+
// Write 25 rows - should trigger 2 flushes with batch size 10
|
787
|
+
for i in 0..25 {
|
788
|
+
writer
|
789
|
+
.write_row(vec![
|
790
|
+
ParquetValue::Int64(i),
|
791
|
+
ParquetValue::String(Arc::from(format!("row_{}", i))),
|
792
|
+
])
|
793
|
+
.unwrap();
|
794
|
+
}
|
795
|
+
|
796
|
+
// Close to flush remaining rows
|
797
|
+
writer.close().unwrap();
|
798
|
+
}
|
799
|
+
|
800
|
+
#[test]
|
801
|
+
fn test_row_size_estimation() {
|
802
|
+
let schema = SchemaBuilder::new()
|
803
|
+
.with_root(SchemaNode::Struct {
|
804
|
+
name: "root".to_string(),
|
805
|
+
nullable: false,
|
806
|
+
fields: vec![
|
807
|
+
SchemaNode::Primitive {
|
808
|
+
name: "id".to_string(),
|
809
|
+
primitive_type: crate::PrimitiveType::Int64,
|
810
|
+
nullable: false,
|
811
|
+
format: None,
|
812
|
+
},
|
813
|
+
SchemaNode::Primitive {
|
814
|
+
name: "data".to_string(),
|
815
|
+
primitive_type: crate::PrimitiveType::String,
|
816
|
+
nullable: false,
|
817
|
+
format: None,
|
818
|
+
},
|
819
|
+
],
|
820
|
+
})
|
821
|
+
.build()
|
822
|
+
.unwrap();
|
823
|
+
|
824
|
+
let buffer = Vec::new();
|
825
|
+
let writer = Writer::new(buffer, schema).unwrap();
|
826
|
+
|
827
|
+
// Test size estimation for different value types
|
828
|
+
let row = vec![
|
829
|
+
ParquetValue::Int64(12345),
|
830
|
+
ParquetValue::String(Arc::from("Hello, World!")),
|
831
|
+
];
|
832
|
+
|
833
|
+
let size = writer.estimate_row_size(&row).unwrap();
|
834
|
+
assert!(size > 0);
|
835
|
+
|
836
|
+
// Int64 = 8 bytes, String = 13 chars + overhead
|
837
|
+
assert!(size >= 8 + 13);
|
838
|
+
}
|
839
|
+
}
|