forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathcsv_json_opener.rs
More file actions
163 lines (144 loc) · 4.9 KB
/
csv_json_opener.rs
File metadata and controls
163 lines (144 loc) · 4.9 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
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! See `main.rs` for how to run it.
use std::sync::Arc;
use arrow::datatypes::{DataType, Field, Schema};
use datafusion::common::config::CsvOptions;
use datafusion::{
assert_batches_eq,
datasource::{
file_format::file_compression_type::FileCompressionType,
listing::PartitionedFile,
object_store::ObjectStoreUrl,
physical_plan::{CsvSource, FileSource, FileStream, JsonOpener, JsonSource},
},
error::Result,
physical_plan::metrics::ExecutionPlanMetricsSet,
};
use datafusion::datasource::physical_plan::FileScanConfigBuilder;
use datafusion_examples::utils::datasets::ExampleDataset;
use futures::StreamExt;
use object_store::{ObjectStore, local::LocalFileSystem, memory::InMemory};
/// This example demonstrates using the low level [`FileStream`] / [`FileOpener`] APIs to directly
/// read data from (CSV/JSON) into Arrow RecordBatches.
///
/// If you want to query data in CSV or JSON files, see the [`dataframe.rs`] and [`sql_query.rs`] examples
pub async fn csv_json_opener() -> Result<()> {
csv_opener().await?;
json_opener().await?;
Ok(())
}
async fn csv_opener() -> Result<()> {
let object_store = Arc::new(LocalFileSystem::new());
let dataset = ExampleDataset::Cars;
let csv_path = dataset.path();
let schema = dataset.schema();
let options = CsvOptions {
has_header: Some(true),
delimiter: b',',
quote: b'"',
..Default::default()
};
let source = CsvSource::new(Arc::clone(&schema))
.with_csv_options(options)
.with_comment(Some(b'#'))
.with_batch_size(8192);
let scan_config =
FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), source)
.with_projection_indices(Some(vec![0, 1]))?
.with_limit(Some(5))
.with_file(PartitionedFile::new(csv_path.display().to_string(), 10))
.build();
let opener =
scan_config
.file_source()
.create_file_opener(object_store, &scan_config, 0)?;
let mut result = vec![];
let mut stream =
FileStream::new(&scan_config, 0, opener, &ExecutionPlanMetricsSet::new())?;
while let Some(batch) = stream.next().await.transpose()? {
result.push(batch);
}
assert_batches_eq!(
&[
"+-----+-------+",
"| car | speed |",
"+-----+-------+",
"| red | 20.0 |",
"| red | 20.3 |",
"| red | 21.4 |",
"| red | 21.5 |",
"| red | 19.0 |",
"+-----+-------+",
],
&result
);
Ok(())
}
async fn json_opener() -> Result<()> {
let object_store = InMemory::new();
let path = object_store::path::Path::from("demo.json");
let data = bytes::Bytes::from(
r#"{"num":5,"str":"test"}
{"num":2,"str":"hello"}
{"num":4,"str":"foo"}"#,
);
object_store.put(&path, data.into()).await?;
let schema = Arc::new(Schema::new(vec![
Field::new("num", DataType::Int64, false),
Field::new("str", DataType::Utf8, false),
]));
let projected = Arc::new(schema.clone().project(&[1, 0])?);
let opener = JsonOpener::new(
8192,
projected,
FileCompressionType::UNCOMPRESSED,
Arc::new(object_store),
true,
);
let scan_config = FileScanConfigBuilder::new(
ObjectStoreUrl::local_filesystem(),
Arc::new(JsonSource::new(schema)),
)
.with_projection_indices(Some(vec![1, 0]))?
.with_limit(Some(5))
.with_file(PartitionedFile::new(path.to_string(), 10))
.build();
let mut stream = FileStream::new(
&scan_config,
0,
Arc::new(opener),
&ExecutionPlanMetricsSet::new(),
)?;
let mut result = vec![];
while let Some(batch) = stream.next().await.transpose()? {
result.push(batch);
}
assert_batches_eq!(
&[
"+-------+-----+",
"| str | num |",
"+-------+-----+",
"| test | 5 |",
"| hello | 2 |",
"| foo | 4 |",
"+-------+-----+",
],
&result
);
Ok(())
}