Skip to content

Commit 7e1a98f

Browse files
committed
remove println and unwrap
1 parent dddb7d7 commit 7e1a98f

File tree

3 files changed

+43
-42
lines changed

3 files changed

+43
-42
lines changed

rust/datafusion/src/datasource/parquet.rs

+34-41
Original file line numberDiff line numberDiff line change
@@ -44,14 +44,14 @@ pub struct ParquetTable {
4444
}
4545

4646
impl ParquetTable {
47-
pub fn new(filename: &str) -> Self {
48-
let file = File::open(filename).unwrap();
49-
let parquet_file = ParquetFile::open(file, None).unwrap();
47+
pub fn try_new(filename: &str) -> Result<Self> {
48+
let file = File::open(filename)?;
49+
let parquet_file = ParquetFile::open(file, None)?;
5050
let schema = parquet_file.schema.clone();
51-
Self {
51+
Ok(Self {
5252
filename: filename.to_string(),
5353
schema,
54-
}
54+
})
5555
}
5656
}
5757

@@ -65,8 +65,8 @@ impl Table for ParquetTable {
6565
projection: &Option<Vec<usize>>,
6666
_batch_size: usize,
6767
) -> Result<Vec<ScanResult>> {
68-
let file = File::open(self.filename.clone()).unwrap();
69-
let parquet_file = ParquetFile::open(file, projection.clone()).unwrap();
68+
let file = File::open(self.filename.clone())?;
69+
let parquet_file = ParquetFile::open(file, projection.clone())?;
7070
Ok(vec![Arc::new(Mutex::new(parquet_file))])
7171
}
7272
}
@@ -84,17 +84,14 @@ pub struct ParquetFile {
8484

8585
impl ParquetFile {
8686
pub fn open(file: File, projection: Option<Vec<usize>>) -> Result<Self> {
87-
println!("open()");
88-
89-
let reader = SerializedFileReader::new(file).unwrap();
87+
let reader = SerializedFileReader::new(file)?;
9088

9189
let metadata = reader.metadata();
9290
let file_type = to_arrow(metadata.file_metadata().schema())?;
9391

9492
match file_type.data_type() {
9593
DataType::Struct(fields) => {
9694
let schema = Schema::new(fields.clone());
97-
//println!("Parquet schema: {:?}", schema);
9895

9996
let projection = match projection {
10097
Some(p) => p,
@@ -130,21 +127,24 @@ impl ParquetFile {
130127
}
131128
}
132129

133-
fn load_next_row_group(&mut self) {
130+
fn load_next_row_group(&mut self) -> Result<()> {
134131
if self.row_group_index < self.reader.num_row_groups() {
135-
let reader = self.reader.get_row_group(self.row_group_index).unwrap();
132+
let reader = self.reader.get_row_group(self.row_group_index)?;
136133

137-
self.column_readers = vec![];
134+
self.column_readers = Vec::with_capacity(self.projection.len());
138135

139136
for i in &self.projection {
140-
self.column_readers
141-
.push(reader.get_column_reader(*i).unwrap());
137+
self.column_readers.push(reader.get_column_reader(*i)?);
142138
}
143139

144140
self.current_row_group = Some(reader);
145141
self.row_group_index += 1;
142+
143+
Ok(())
146144
} else {
147-
panic!()
145+
Err(ExecutionError::General(
146+
"Attempt to read past final row group".to_string(),
147+
))
148148
}
149149
}
150150

@@ -171,7 +171,7 @@ impl ParquetFile {
171171
) {
172172
Ok((count, _)) => {
173173
let mut builder = BooleanBuilder::new(count);
174-
builder.append_slice(&read_buffer[0..count]).unwrap();
174+
builder.append_slice(&read_buffer[0..count])?;
175175
row_count = count;
176176
Arc::new(builder.finish())
177177
}
@@ -199,7 +199,7 @@ impl ParquetFile {
199199
) {
200200
Ok((count, _)) => {
201201
let mut builder = Int32Builder::new(count);
202-
builder.append_slice(&read_buffer[0..count]).unwrap();
202+
builder.append_slice(&read_buffer[0..count])?;
203203
row_count = count;
204204
Arc::new(builder.finish())
205205
}
@@ -227,7 +227,7 @@ impl ParquetFile {
227227
) {
228228
Ok((count, _)) => {
229229
let mut builder = Int64Builder::new(count);
230-
builder.append_slice(&read_buffer[0..count]).unwrap();
230+
builder.append_slice(&read_buffer[0..count])?;
231231
row_count = count;
232232
Arc::new(builder.finish())
233233
}
@@ -262,7 +262,7 @@ impl ParquetFile {
262262
| (v[1] as u128) << 32
263263
| (v[2] as u128);
264264
let ms: i64 = (value / 1000000) as i64;
265-
builder.append_value(ms).unwrap();
265+
builder.append_value(ms)?;
266266
}
267267
row_count = count;
268268
Arc::new(builder.finish())
@@ -289,7 +289,7 @@ impl ParquetFile {
289289
&mut read_buffer,
290290
) {
291291
Ok((count, _)) => {
292-
builder.append_slice(&read_buffer[0..count]).unwrap();
292+
builder.append_slice(&read_buffer[0..count])?;
293293
row_count = count;
294294
Arc::new(builder.finish())
295295
}
@@ -315,7 +315,7 @@ impl ParquetFile {
315315
&mut read_buffer,
316316
) {
317317
Ok((count, _)) => {
318-
builder.append_slice(&read_buffer[0..count]).unwrap();
318+
builder.append_slice(&read_buffer[0..count])?;
319319
row_count = count;
320320
Arc::new(builder.finish())
321321
}
@@ -339,12 +339,10 @@ impl ParquetFile {
339339
let mut builder = BinaryBuilder::new(row_count);
340340
for j in 0..row_count {
341341
let slice = b[j].slice(0, b[j].len());
342-
builder
343-
.append_string(
344-
&String::from_utf8(slice.data().to_vec())
345-
.unwrap(),
346-
)
347-
.unwrap();
342+
builder.append_string(
343+
&String::from_utf8(slice.data().to_vec())
344+
.unwrap(),
345+
)?;
348346
}
349347
Arc::new(builder.finish())
350348
}
@@ -368,12 +366,10 @@ impl ParquetFile {
368366
let mut builder = BinaryBuilder::new(row_count);
369367
for j in 0..row_count {
370368
let slice = b[j].slice(0, b[j].len());
371-
builder
372-
.append_string(
373-
&String::from_utf8(slice.data().to_vec())
374-
.unwrap(),
375-
)
376-
.unwrap();
369+
builder.append_string(
370+
&String::from_utf8(slice.data().to_vec())
371+
.unwrap(),
372+
)?;
377373
}
378374
Arc::new(builder.finish())
379375
}
@@ -387,12 +383,9 @@ impl ParquetFile {
387383
}
388384
};
389385

390-
println!("Adding array to batch");
391386
batch.push(array);
392387
}
393388

394-
println!("Loaded batch of {} rows", row_count);
395-
396389
if row_count == 0 {
397390
Ok(None)
398391
} else {
@@ -445,14 +438,14 @@ impl RecordBatchIterator for ParquetFile {
445438
fn next(&mut self) -> Result<Option<RecordBatch>> {
446439
// advance the row group reader if necessary
447440
if self.current_row_group.is_none() {
448-
self.load_next_row_group();
441+
self.load_next_row_group()?;
449442
self.load_batch()
450443
} else {
451444
match self.load_batch() {
452445
Ok(Some(b)) => Ok(Some(b)),
453446
Ok(None) => {
454447
if self.row_group_index < self.reader.num_row_groups() {
455-
self.load_next_row_group();
448+
self.load_next_row_group()?;
456449
self.load_batch()
457450
} else {
458451
Ok(None)
@@ -678,7 +671,7 @@ mod tests {
678671
fn load_table(name: &str) -> Box<Table> {
679672
let testdata = env::var("PARQUET_TEST_DATA").unwrap();
680673
let filename = format!("{}/{}", testdata, name);
681-
let table = ParquetTable::new(&filename);
674+
let table = ParquetTable::try_new(&filename).unwrap();
682675
println!("Loading file {} with schema:", name);
683676
for field in table.schema().fields() {
684677
println!("\t{:?}", field);

rust/datafusion/src/execution/error.rs

+8
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ use std::io::Error;
2121
use std::result;
2222

2323
use arrow::error::ArrowError;
24+
use parquet::errors::ParquetError;
2425

2526
use sqlparser::sqlparser::ParserError;
2627

@@ -35,6 +36,7 @@ pub enum ExecutionError {
3536
NotImplemented(String),
3637
InternalError(String),
3738
ArrowError(ArrowError),
39+
ParquetError(ParquetError),
3840
ExecutionError(String),
3941
}
4042

@@ -62,6 +64,12 @@ impl From<ArrowError> for ExecutionError {
6264
}
6365
}
6466

67+
impl From<ParquetError> for ExecutionError {
68+
fn from(e: ParquetError) -> Self {
69+
ExecutionError::ParquetError(e)
70+
}
71+
}
72+
6573
impl From<ParserError> for ExecutionError {
6674
fn from(e: ParserError) -> Self {
6775
ExecutionError::ParserError(e)

rust/datafusion/tests/sql.rs

+1-1
Original file line numberDiff line numberDiff line change
@@ -182,7 +182,7 @@ fn register_csv(
182182
fn load_parquet_table(name: &str) -> Rc<Table> {
183183
let testdata = env::var("PARQUET_TEST_DATA").unwrap();
184184
let filename = format!("{}/{}", testdata, name);
185-
let table = ParquetTable::new(&filename);
185+
let table = ParquetTable::try_new(&filename).unwrap();
186186
println!("{:?}", table.schema());
187187
Rc::new(table)
188188
}

0 commit comments

Comments
 (0)