-
Notifications
You must be signed in to change notification settings - Fork 1.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Major performance regression in reading partitioned Parquet data on master #1363
Comments
Hi @Dandandan, sorry if its #1010 that created this regression. I think I only ran the TPCH on CSV for performance testing. Did you do your tests on the latest of master? after #1347 was merged? |
No problem - just want to figure out what could be the reason :)! So far I tested:
For TPCH I remember collecting stats doesn't have a big effect now, as data is very evenly distributed, and I think also doesn't take a long time to collect those. To reproduce:
|
Off-topic: how about introducing bench results in every PR? |
EDIT: I tried reverting just that part back to the original |
@rdettai It seems changing |
Hm - maybe too early. it seems that for query 6 this brings it down to the roughly the earlier performance, but for other queries it's doing even worse 🤔 |
FTR - this is the entire diff: diff --git a/datafusion/src/datasource/file_format/parquet.rs b/datafusion/src/datasource/file_format/parquet.rs
index 7976be791..1d493c274 100644
--- a/datafusion/src/datasource/file_format/parquet.rs
+++ b/datafusion/src/datasource/file_format/parquet.rs
@@ -18,6 +18,7 @@
//! Parquet format abstractions
use std::any::Any;
+use std::io::BufRead;
use std::io::Read;
use std::sync::Arc;
@@ -321,7 +322,7 @@ impl Length for ChunkObjectReader {
}
impl ChunkReader for ChunkObjectReader {
- type T = Box<dyn Read + Send + Sync>;
+ type T = Box<dyn BufRead + Send + Sync>;
fn get_read(&self, start: u64, length: usize) -> ParquetResult<Self::T> {
self.0
diff --git a/datafusion/src/datasource/object_store/local.rs b/datafusion/src/datasource/object_store/local.rs
index b2a2ddfa9..5b87bee1a 100644
--- a/datafusion/src/datasource/object_store/local.rs
+++ b/datafusion/src/datasource/object_store/local.rs
@@ -18,7 +18,7 @@
//! Object store that represents the Local File System.
use std::fs::{self, File, Metadata};
-use std::io::{Read, Seek, SeekFrom};
+use std::io::{BufRead, BufReader, Read, Seek, SeekFrom};
use std::sync::Arc;
use async_trait::async_trait;
@@ -82,12 +82,15 @@ impl ObjectReader for LocalFileReader {
&self,
start: u64,
length: usize,
- ) -> Result<Box<dyn Read + Send + Sync>> {
+ ) -> Result<Box<dyn BufRead + Send + Sync>> {
// A new file descriptor is opened for each chunk reader.
// This okay because chunks are usually fairly large.
let mut file = File::open(&self.file.path)?;
file.seek(SeekFrom::Start(start))?;
- Ok(Box::new(file.take(length as u64)))
+
+ let file = BufReader::new(file.take(length as u64));
+
+ Ok(Box::new(file))
}
fn length(&self) -> u64 {
diff --git a/datafusion/src/datasource/object_store/mod.rs b/datafusion/src/datasource/object_store/mod.rs
index 59e184103..97085c5df 100644
--- a/datafusion/src/datasource/object_store/mod.rs
+++ b/datafusion/src/datasource/object_store/mod.rs
@@ -21,7 +21,7 @@ pub mod local;
use std::collections::HashMap;
use std::fmt::{self, Debug};
-use std::io::Read;
+use std::io::{BufRead, Read};
use std::pin::Pin;
use std::sync::{Arc, RwLock};
@@ -48,10 +48,10 @@ pub trait ObjectReader: Send + Sync {
&self,
start: u64,
length: usize,
- ) -> Result<Box<dyn Read + Send + Sync>>;
+ ) -> Result<Box<dyn BufRead + Send + Sync>>;
/// Get reader for the entire file
- fn sync_reader(&self) -> Result<Box<dyn Read + Send + Sync>> {
+ fn sync_reader(&self) -> Result<Box<dyn BufRead + Send + Sync>> {
self.sync_chunk_reader(0, self.length() as usize)
}
diff --git a/datafusion/src/physical_plan/file_format/file_stream.rs b/datafusion/src/physical_plan/file_format/file_stream.rs
index 958b1721b..ca3a69a6e 100644
--- a/datafusion/src/physical_plan/file_format/file_stream.rs
+++ b/datafusion/src/physical_plan/file_format/file_stream.rs
@@ -32,13 +32,7 @@ use arrow::{
record_batch::RecordBatch,
};
use futures::Stream;
-use std::{
- io::Read,
- iter,
- pin::Pin,
- sync::Arc,
- task::{Context, Poll},
-};
+use std::{io::{BufRead, Read}, iter, pin::Pin, sync::Arc, task::{Context, Poll}};
use super::PartitionColumnProjector;
@@ -48,12 +42,12 @@ pub type BatchIter = Box<dyn Iterator<Item = ArrowResult<RecordBatch>> + Send +
/// A closure that creates a file format reader (iterator over `RecordBatch`) from a `Read` object
/// and an optional number of required records.
pub trait FormatReaderOpener:
- FnMut(Box<dyn Read + Send + Sync>, &Option<usize>) -> BatchIter + Send + Unpin + 'static
+ FnMut(Box<dyn BufRead + Send + Sync>, &Option<usize>) -> BatchIter + Send + Unpin + 'static
{
}
impl<T> FormatReaderOpener for T where
- T: FnMut(Box<dyn Read + Send + Sync>, &Option<usize>) -> BatchIter
+ T: FnMut(Box<dyn BufRead + Send + Sync>, &Option<usize>) -> BatchIter
+ Send
+ Unpin
+ 'static |
Interesting fact:
After #1010, this is not the case anymore, but
So everything is slowed down, both on the thread where the reading occurs and in the Tokio threads... WeEEeeIiiiIrd 😅 |
I opened a PR here https://github.com/apache/arrow-datafusion/pull/1366/files so you can have a look too. It seems the slowdown of query 10 might be unrelated to the reading performance. |
Yes - some continuous benchmarking would definitely be something valuable and could avoid regressions. |
Why the issue was closed? Was the issue resolved and performance come back ? |
Yes, according to my benchmarks performance was around the same as before introducing the regression. |
I'm not sure whether it is relevant or not. In the current implementation, the sync_chunk_reader() method was FilePageIterator.next() -> FileReader.get_row_group().get_column_page_reader() -> SerializedRowGroupReader.get_column_page_reader() -> ChunkObjectReader.get_read() -> LocalFileReader.sync_chunk_reader()
TPCH Q1: Read parquet file lineitem.parquet time spent: 590639777 ns, row group count 60, skipped row group 0 Query 1 iteration 0 took 679.9 ms |
I think that might be relevant. How about opening a new issue to track improving on that (reusing the file descriptor)? |
Sure, I can open a new issue to track it. But please help to take a look and confirm my understanding is correct. |
Reading (partitioned) Parquet data got slower, when executing parquet data.
It seems to have started with #1010 - the commit before doesn't have the performance regression.
This is visible when running TPCH benchmarks, for example, executing query 6 became much slower.
I am not sure what the cause is - it would help to find the commit where the "slowness" was introduced.
Originally posted by @Dandandan in #68 (comment)
The text was updated successfully, but these errors were encountered: