-
Notifications
You must be signed in to change notification settings - Fork 72
Expand file tree
/
Copy patharrow.rs
More file actions
125 lines (109 loc) · 4.07 KB
/
arrow.rs
File metadata and controls
125 lines (109 loc) · 4.07 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
use std::sync::Arc;
use arrow::array::RecordBatch;
use arrow::compute::cast;
use datafusion::arrow::datatypes::SchemaRef;
use datafusion::error::DataFusionError;
/// Cast a `RecordBatch` to match the target schema, casting columns whose types differ.
/// Columns that already match are passed through unchanged.
pub fn cast_batch_to_schema(
batch: &RecordBatch,
target_schema: &SchemaRef,
) -> Result<RecordBatch, DataFusionError> {
let columns: Vec<_> = batch
.columns()
.iter()
.zip(target_schema.fields())
.map(|(col, target_field)| {
if col.data_type() == target_field.data_type() {
Ok(Arc::clone(col))
} else {
cast(col, target_field.data_type()).map_err(|e| {
DataFusionError::Execution(format!(
"Failed to cast column '{}' from {:?} to {:?}: {e}",
target_field.name(),
col.data_type(),
target_field.data_type(),
))
})
}
})
.collect::<Result<_, _>>()?;
RecordBatch::try_new(Arc::clone(target_schema), columns).map_err(|e| {
DataFusionError::Execution(format!("Failed to create RecordBatch after cast: {e}"))
})
}
#[cfg(test)]
mod tests {
use super::*;
use arrow::array::{Int32Array, TimestampMicrosecondArray, TimestampNanosecondArray};
use arrow::datatypes::{DataType, Field, Schema, TimeUnit};
#[test]
fn test_cast_timestamp_us_to_ns() {
let source_schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, false),
Field::new(
"ts",
DataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())),
false,
),
]));
let target_schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, false),
Field::new(
"ts",
DataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into())),
false,
),
]));
let batch = RecordBatch::try_new(
source_schema,
vec![
Arc::new(Int32Array::from(vec![1, 2, 3])),
Arc::new(
TimestampMicrosecondArray::from(vec![1_000_000, 2_000_000, 3_000_000])
.with_timezone("UTC"),
),
],
)
.unwrap();
let result = cast_batch_to_schema(&batch, &target_schema).unwrap();
assert_eq!(result.schema(), target_schema);
assert_eq!(
result.column(1).data_type(),
&DataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into()))
);
// Values should be multiplied by 1000
let ts_col = result
.column(1)
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.unwrap();
assert_eq!(ts_col.value(0), 1_000_000_000);
assert_eq!(ts_col.value(1), 2_000_000_000);
assert_eq!(ts_col.value(2), 3_000_000_000);
}
#[test]
fn test_no_cast_when_types_match() {
let schema = Arc::new(Schema::new(vec![Field::new("id", DataType::Int32, false)]));
let batch = RecordBatch::try_new(
Arc::clone(&schema),
vec![Arc::new(Int32Array::from(vec![1, 2, 3]))],
)
.unwrap();
let result = cast_batch_to_schema(&batch, &schema).unwrap();
assert_eq!(result.schema(), schema);
}
#[test]
fn test_cast_incompatible_types_returns_error() {
use arrow::array::StringArray;
let source_schema = Arc::new(Schema::new(vec![Field::new("val", DataType::Utf8, false)]));
let target_schema = Arc::new(Schema::new(vec![Field::new("val", DataType::Int32, false)]));
let batch = RecordBatch::try_new(
source_schema,
vec![Arc::new(StringArray::from(vec!["not_a_number"]))],
)
.unwrap();
let result = cast_batch_to_schema(&batch, &target_schema);
assert!(result.is_err());
}
}