From c16badc81f90e33af7cf20c651b508e0329f4ea0 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 30 Aug 2024 14:26:31 -0700 Subject: [PATCH 01/38] Prototype Rearchitect Disable feature Enable; Add inference and estimators Cleanup --- Cargo.lock | 4 + Makefile | 4 + src/arrow2/src/io/csv/read_async/reader.rs | 25 +- src/daft-csv/Cargo.toml | 4 + src/daft-csv/src/lib.rs | 5 + src/daft-csv/src/local.rs | 959 +++++++++++++++++++++ src/daft-csv/src/read.rs | 82 +- src/daft-decoding/src/deserialize.rs | 12 +- 8 files changed, 1069 insertions(+), 26 deletions(-) create mode 100644 src/daft-csv/src/local.rs diff --git a/Cargo.lock b/Cargo.lock index 0dc52e53bd..b78da50d76 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1762,6 +1762,7 @@ dependencies = [ "async-stream", "common-error", "common-py-serde", + "crossbeam-channel", "csv-async", "daft-compression", "daft-core", @@ -1770,6 +1771,9 @@ dependencies = [ "daft-io", "daft-table", "futures", + "indexmap 2.5.0", + "memchr", + "memmap2", "pyo3", "rayon", "rstest", diff --git a/Makefile b/Makefile index d1d96e8df6..327f00513c 100644 --- a/Makefile +++ b/Makefile @@ -52,6 +52,10 @@ build: check-toolchain .venv ## Compile and install Daft for development build-release: check-toolchain .venv ## Compile and install a faster Daft binary @unset CONDA_PREFIX && PYO3_PYTHON=$(VENV_BIN)/python $(VENV_BIN)/maturin develop --release +.PHONY: build-bench +build-bench: check-toolchain .venv ## Compile and install a faster Daft binary + @unset CONDA_PREFIX && PYO3_PYTHON=$(VENV_BIN)/python $(VENV_BIN)/maturin develop --profile dev-bench + .PHONY: test test: .venv build ## Run tests HYPOTHESIS_MAX_EXAMPLES=$(HYPOTHESIS_MAX_EXAMPLES) $(VENV_BIN)/pytest --hypothesis-seed=$(HYPOTHESIS_SEED) diff --git a/src/arrow2/src/io/csv/read_async/reader.rs b/src/arrow2/src/io/csv/read_async/reader.rs index 0db0f25268..a95b88e5ad 100644 --- a/src/arrow2/src/io/csv/read_async/reader.rs +++ b/src/arrow2/src/io/csv/read_async/reader.rs @@ -1,10 +1,11 @@ use futures::AsyncRead; use super::{AsyncReader, ByteRecord}; +use crate::io::csv::read; use crate::error::{Error, Result}; -/// Asynchronosly read `len` rows from `reader` into `row`, skipping the first `skip`. +/// Asynchronosly read `rows.len` rows from `reader` into `rows`, skipping the first `skip`. /// This operation has minimal CPU work and is thus the fastest way to read through a CSV /// without deserializing the contents to Arrow. pub async fn read_rows( @@ -37,3 +38,25 @@ where } Ok(row_number) } + +/// Synchronously read `rows.len` rows from `reader` into `rows`. This is used in the local i/o case. +pub fn local_read_rows( + reader: &mut read::Reader, + rows: &mut [read::ByteRecord], +) -> Result<(usize, bool)> +where + R: std::io::Read, +{ + let mut row_number = 0; + let mut has_more = true; + for row in rows.iter_mut() { + has_more = reader + .read_byte_record(row) + .map_err(|e| Error::External(format!(" at line {}", row_number), Box::new(e)))?; + if !has_more { + break; + } + row_number += 1; + } + Ok((row_number, has_more)) +} diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index dde511422b..27f1cc2275 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -4,6 +4,7 @@ async-compat = {workspace = true} async-stream = {workspace = true} common-error = {path = "../common/error", default-features = false} common-py-serde = {path = "../common/py-serde", default-features = false} +crossbeam-channel = "0.5.1" csv-async = "1.3.0" daft-compression = {path = "../daft-compression", default-features = false} daft-core = {path = "../daft-core", default-features = false} @@ -12,6 +13,9 @@ daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} +indexmap = {workspace = true, features = ["serde"]} +memchr = "2.7.2" +memmap2 = "0.9.4" pyo3 = {workspace = true, optional = true} rayon = {workspace = true} serde = {workspace = true} diff --git a/src/daft-csv/src/lib.rs b/src/daft-csv/src/lib.rs index b49245edab..1e90f2eb25 100644 --- a/src/daft-csv/src/lib.rs +++ b/src/daft-csv/src/lib.rs @@ -2,9 +2,12 @@ #![feature(let_chains)] #![feature(trait_alias)] #![feature(trait_upcasting)] +#![feature(test)] +extern crate test; use common_error::DaftError; use snafu::Snafu; +pub mod local; pub mod metadata; pub mod options; #[cfg(feature = "python")] @@ -23,6 +26,8 @@ pub enum Error { #[snafu(display("{source}"))] IOError { source: daft_io::Error }, #[snafu(display("{source}"))] + StdIOError { source: std::io::Error }, + #[snafu(display("{source}"))] CSVError { source: csv_async::Error }, #[snafu(display("Invalid char: {}", val))] WrongChar { diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs new file mode 100644 index 0000000000..dad5f7b280 --- /dev/null +++ b/src/daft-csv/src/local.rs @@ -0,0 +1,959 @@ +use core::str; +use std::io::{Chain, Cursor, Read}; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::{num::NonZeroUsize, sync::Arc, sync::Condvar, sync::Mutex}; + +use crate::metadata::read_csv_schema_single; +use crate::ArrowSnafu; +use crate::{CsvConvertOptions, CsvParseOptions, CsvReadOptions}; +use arrow2::{ + datatypes::Field, + io::csv::read, + io::csv::read::{Reader, ReaderBuilder}, + io::csv::read_async::local_read_rows, +}; +use common_error::{DaftError, DaftResult}; +use crossbeam_channel::Sender; +use daft_core::{schema::Schema, utils::arrow::cast_array_for_daft_if_needed, Series}; +use daft_decoding::deserialize::deserialize_column; +use daft_dsl::{optimization::get_required_columns, Expr}; +use daft_io::{IOClient, IOStatsRef}; +use daft_table::Table; +use futures::{stream::BoxStream, Stream, StreamExt}; +use rayon::{ + iter::IndexedParallelIterator, + prelude::{IntoParallelRefIterator, ParallelIterator}, +}; +use snafu::ResultExt; + +use crate::read::{fields_to_projection_indices, tables_concat}; + +#[allow(clippy::doc_lazy_continuation)] +/// Our local CSV reader takes the following approach to reading CSV files: +/// 1. Read the CSV file in 4MB chunks from a slab pool. +/// 2. Adjust the chunks so that chunks are contiguous and contain complete +/// CSV records. See `get_file_chunk` for more details. +/// 3. In parallel with the above, convert the adjusted chunks into byte records, +/// which are stored within pre-allocated CSV buffers. +/// 4. In parallel with the above, deserialize each CSV buffer into a Daft table +/// and stream the results. +/// +/// Slab Pool CSV Buffer Pool +/// ┌────────────────────┐ ┌────────────────────┐ +/// │ 4MB Chunks │ │ CSV Buffers │ +/// │┌───┐┌───┐┌───┐ │ │┌───┐┌───┐┌───┐ │ +/// ││ ││ ││ │ ... │ ││ ││ ││ │ ... │ +/// │└─┬─┘└─┬─┘└───┘ │ │└─┬─┘└─┬─┘└───┘ │ +/// └──┼────┼────────────┘ └──┼────┼────────────┘ +/// │ │ │ │ +/// ───────┐ │ │ │ │ +/// /│ │ │ │ │ │ +/// /─┘ │ │ │ │ │ +/// │ │ ▼ ▼ ▼ ▼ +/// │ | ┌───┐ ┌───┐ ┌────┐ ┬--─┐ ┌───┐ ┌───┐ ┌───┐ ┌───┐ +/// │ ─┼───►│ │ │ │ ──────► │ ┬┘┌─┘ ┬─┘ ───────► │ │ │ │ ──────────► │ │ │ │ +/// │ CSV File │ └───┘ └───┘ └───┴ └───┘ └───┘ └───┘ └───┘ └───┘ +/// │ │ Chain of buffers Adjusted chunks Vectors of ByteRecords Stream of Daft tables +/// │ │ +/// └──────────┘ + +/// A pool of ByteRecord slabs. Used for deserializing CSV. +#[derive(Debug)] +struct CsvBufferPool { + buffers: Mutex>>, + buffer_size: usize, + record_buffer_size: usize, + num_fields: usize, +} + +/// A slab of ByteRecords. Used for deserializing CSV. +struct CsvBuffer { + pool: Arc, + buffer: Vec, +} + +impl CsvBufferPool { + pub fn new( + record_buffer_size: usize, + num_fields: usize, + chunk_size_rows: usize, + initial_pool_size: usize, + ) -> Self { + let chunk_buffers = vec![ + vec![ + read::ByteRecord::with_capacity(record_buffer_size, num_fields); + chunk_size_rows + ]; + initial_pool_size + ]; + CsvBufferPool { + buffers: Mutex::new(chunk_buffers), + buffer_size: chunk_size_rows, + record_buffer_size, + num_fields, + } + } + + pub fn get_buffer(self: &Arc) -> CsvBuffer { + let mut buffers = self.buffers.lock().unwrap(); + let buffer = buffers.pop(); + let buffer = match buffer { + Some(buffer) => buffer, + None => vec![ + read::ByteRecord::with_capacity(self.record_buffer_size, self.num_fields); + self.buffer_size + ], + }; + + CsvBuffer { + pool: Arc::clone(self), + buffer, + } + } + + fn return_buffer(&self, buffer: Vec) { + let mut buffers = self.buffers.lock().unwrap(); + buffers.push(buffer); + } +} + +// The default size of a slab used for reading CSV files in chunks. Currently set to 4 MiB. This can be tuned. +const SLABSIZE: usize = 4 * 1024 * 1024; +// The default number of slabs in a slab pool. +const SLABPOOL_DEFAULT_SIZE: usize = 20; + +/// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. +#[derive(Debug)] +struct SlabPool { + buffers: Mutex>>, + condvar: Condvar, +} + +/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. +#[derive(Clone)] +struct Slab { + pool: Arc, + // We wrap the Arc in an Option so that when a Slab is being dropped, we can move the Slab's reference + // to the Arc back to the slab pool. + buffer: Option>, +} + +impl Drop for Slab { + fn drop(&mut self) { + // Move the buffer back to the slab pool. + if let Some(buffer) = self.buffer.take() { + self.pool.return_buffer(buffer); + } + } +} + +impl SlabPool { + pub fn new() -> Self { + let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + .map(|_| Arc::new([0; SLABSIZE])) + .collect(); + SlabPool { + buffers: Mutex::new(chunk_buffers), + condvar: Condvar::new(), + } + } + + pub fn get_buffer(self: &Arc) -> Arc<[u8; SLABSIZE]> { + let mut buffers = self.buffers.lock().unwrap(); + while buffers.is_empty() { + // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. + // This potentially allows us to rate limit the CSV reader until downstream consumers are ready for data. + buffers = self.condvar.wait(buffers).unwrap(); + } + buffers.pop().unwrap() + } + + fn return_buffer(&self, buffer: Arc<[u8; SLABSIZE]>) { + let mut buffers = self.buffers.lock().unwrap(); + buffers.push(buffer); + self.condvar.notify_one(); + } +} + +/// A data structure that holds either a single slice of bytes, or a chain of two slices of bytes. +/// See the description to `parse_json` for more details. +#[derive(Debug)] +enum BufferSource<'a> { + Single(Cursor<&'a [u8]>), + Chain(Chain, Cursor<&'a [u8]>>), +} + +/// Read implementation that allows BufferSource to be used by csv::read::Reader. +impl<'a> Read for BufferSource<'a> { + fn read(&mut self, buf: &mut [u8]) -> std::io::Result { + match self { + BufferSource::Single(cursor) => std::io::Read::read(cursor, buf), + BufferSource::Chain(chain) => chain.read(buf), + } + } +} + +pub async fn read_csv_local( + uri: &str, + convert_options: Option, + parse_options: CsvParseOptions, + read_options: Option, + io_client: Arc, + io_stats: Option, + max_chunks_in_flight: Option, +) -> DaftResult { + let stream = stream_csv_local( + uri, + convert_options, + parse_options, + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) + .await?; + tables_concat(tables_stream_collect(Box::pin(stream)).await) +} + +async fn tables_stream_collect(stream: BoxStream<'static, DaftResult
>) -> Vec
{ + stream + .filter_map(|result| async { + match result { + Ok(table) => Some(table), + Err(_) => None, // Skips errors; you could log them or handle differently + } + }) + .collect() + .await +} + +pub async fn stream_csv_local( + uri: &str, + convert_options: Option, + parse_options: CsvParseOptions, + read_options: Option, + io_client: Arc, + io_stats: Option, + max_chunks_in_flight: Option, +) -> DaftResult> + Send> { + let uri = uri.trim_start_matches("file://"); + let file = std::fs::File::open(uri)?; + + let predicate = convert_options + .as_ref() + .and_then(|opts| opts.predicate.clone()); + + let limit = convert_options.as_ref().and_then(|opts| opts.limit); + + let include_columns = convert_options + .as_ref() + .and_then(|opts| opts.include_columns.clone()); + + let convert_options = match (convert_options, &predicate) { + (None, _) => None, + (co, None) => co, + (Some(mut co), Some(predicate)) => { + if let Some(ref mut include_columns) = co.include_columns { + let required_columns_for_predicate = get_required_columns(predicate); + for rc in required_columns_for_predicate { + if include_columns.iter().all(|c| c.as_str() != rc.as_str()) { + include_columns.push(rc) + } + } + } + // If we have a limit and a predicate, remove limit for stream. + co.limit = None; + Some(co) + } + } + .unwrap_or_default(); + + let slabpool = Arc::new(SlabPool::new()); + let (schema, estimated_mean_row_size, estimated_std_row_size) = + get_schema_and_estimators(uri, &convert_options, &parse_options, io_client, io_stats) + .await?; + + let n_threads: usize = std::thread::available_parallelism() + .unwrap_or(NonZeroUsize::new(2).unwrap()) + .into(); + let chunk_size = read_options + .as_ref() + .and_then(|opt| opt.chunk_size.or_else(|| opt.buffer_size.map(|bs| bs / 8))) + .unwrap_or(DEFAULT_CHUNK_SIZE); + let projection_indices = fields_to_projection_indices( + &schema.clone().fields, + &convert_options.clone().include_columns, + ); + let fields = schema.clone().fields; + let fields_subset = projection_indices + .iter() + .map(|i| fields.get(*i).unwrap().into()) + .collect::>(); + let read_schema = Arc::new(daft_core::schema::Schema::new(fields_subset)?); + let read_daft_fields = Arc::new( + read_schema + .fields + .values() + .map(|f| Arc::new(f.clone())) + .collect::>(), + ); + let record_buffer_size = (estimated_mean_row_size + estimated_std_row_size).ceil() as usize; + let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; + let num_fields = schema.fields.len(); + // TODO(desmond): We might consider creating per-process buffer pools and slab pools. + let buffer_pool = Arc::new(CsvBufferPool::new( + record_buffer_size, + num_fields, + chunk_size_rows, + n_threads * 2, + )); + // We suppose that each slab of CSV data produces (chunk size / slab size) number of Daft tables. We + // then double this capacity to ensure that our channel is never full and our threads won't deadlock. + let (sender, receiver) = + crossbeam_channel::bounded(max_chunks_in_flight.unwrap_or(2 * chunk_size / SLABSIZE)); + rayon::spawn(move || { + consume_csv_file( + file, + buffer_pool, + slabpool, + parse_options, + projection_indices, + read_daft_fields, + read_schema, + fields, + num_fields, + &include_columns, + predicate, + limit, + sender, + ); + }); + + Ok(futures::stream::iter(receiver)) +} + +/// Helper function that reads up to 1 MiB of the CSV file to estimate stats and/or infer the schema of the file. +async fn get_schema_and_estimators( + uri: &str, + convert_options: &CsvConvertOptions, + parse_options: &CsvParseOptions, + io_client: Arc, + io_stats: Option, +) -> DaftResult<(arrow2::datatypes::Schema, f64, f64)> { + let (inferred_schema, read_stats) = read_csv_schema_single( + uri, + parse_options.clone(), + // Read at most 1 MiB to estimate stats. + Some(1024 * 1024), + io_client.clone(), + io_stats.clone(), + ) + .await?; + + let mut schema = if let Some(schema) = convert_options.schema.clone() { + schema.to_arrow()? + } else { + inferred_schema.to_arrow()? + }; + // Rename fields, if necessary. + if let Some(column_names) = convert_options.column_names.clone() { + schema = schema + .fields + .into_iter() + .zip(column_names.iter()) + .map(|(field, name)| { + Field::new(name, field.data_type, field.is_nullable).with_metadata(field.metadata) + }) + .collect::>() + .into(); + } + Ok(( + schema, + read_stats.mean_record_size_bytes, + read_stats.stddev_record_size_bytes, + )) +} + +/// Consumes the CSV file and sends the results to `sender`. +#[allow(clippy::too_many_arguments)] +fn consume_csv_file( + mut file: std::fs::File, + buffer_pool: Arc, + slabpool: Arc, + parse_options: CsvParseOptions, + projection_indices: Arc>, + read_daft_fields: Arc>>, + read_schema: Arc, + fields: Vec, + num_fields: usize, + include_columns: &Option>, + predicate: Option>, + limit: Option, + sender: Sender>, +) { + let rows_read = Arc::new(AtomicUsize::new(0)); + let mut has_header = parse_options.has_header; + let total_len = file.metadata().unwrap().len() as usize; + let field_delimiter = parse_options.delimiter; + let escape_char = parse_options.escape_char; + let quote_char = parse_options.quote; + let double_quote_escape_allowed = parse_options.double_quote; + let mut total_bytes_read = 0; + let mut next_slab = None; + let mut next_buffer_len = 0; + let mut first_buffer = true; + loop { + let limit_reached = limit.map_or(false, |limit| { + let current_rows_read = rows_read.load(Ordering::Relaxed); + current_rows_read >= limit + }); + if limit_reached { + break; + } + let (current_slab, current_buffer_len) = match next_slab.take() { + Some(next_slab) => { + total_bytes_read += next_buffer_len; + (next_slab, next_buffer_len) + } + None => { + let mut buffer = slabpool.get_buffer(); + match Arc::get_mut(&mut buffer) { + Some(inner_buffer) => { + let bytes_read = file.read(inner_buffer).unwrap(); + if bytes_read == 0 { + slabpool.return_buffer(buffer); + break; + } + total_bytes_read += bytes_read; + ( + Arc::new(Slab { + pool: Arc::clone(&slabpool), + buffer: Some(buffer), + }), + bytes_read, + ) + } + None => { + slabpool.return_buffer(buffer); + break; + } + } + } + }; + (next_slab, next_buffer_len) = if total_bytes_read < total_len { + let mut next_buffer = slabpool.get_buffer(); + match Arc::get_mut(&mut next_buffer) { + Some(inner_buffer) => { + let bytes_read = file.read(inner_buffer).unwrap(); + if bytes_read == 0 { + slabpool.return_buffer(next_buffer); + (None, 0) + } else { + ( + Some(Arc::new(Slab { + pool: Arc::clone(&slabpool), + buffer: Some(next_buffer), + })), + bytes_read, + ) + } + } + None => { + slabpool.return_buffer(next_buffer); + break; + } + } + } else { + (None, 0) + }; + let file_chunk = get_file_chunk( + unsafe_clone_buffer(¤t_slab.buffer), + current_buffer_len, + next_slab + .as_ref() + .map(|slab| unsafe_clone_buffer(&slab.buffer)), + next_buffer_len, + first_buffer, + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ); + first_buffer = false; + if let (None, _) = file_chunk { + // Return the buffer. It doesn't matter that we still have a reference to the slab. We're going to fallback + // and the slabs will be useless. + slabpool.return_buffer(unsafe_clone_buffer(¤t_slab.buffer)); + // Exit early before spawning a new thread. + break; + // TODO(desmond): we should fallback instead. + } + let current_slab_clone = Arc::clone(¤t_slab); + let next_slab_clone = next_slab.clone(); + let parse_options = parse_options.clone(); + let csv_buffer = buffer_pool.get_buffer(); + let projection_indices = projection_indices.clone(); + let fields = fields.clone(); + let read_daft_fields = read_daft_fields.clone(); + let read_schema = read_schema.clone(); + let include_columns = include_columns.clone(); + let predicate = predicate.clone(); + let sender = sender.clone(); + let rows_read = Arc::clone(&rows_read); + rayon::spawn(move || { + let limit_reached = limit.map_or(false, |limit| { + let current_rows_read = rows_read.load(Ordering::Relaxed); + current_rows_read >= limit + }); + if !limit_reached { + match file_chunk { + (Some(start), None) => { + if let Some(buffer) = ¤t_slab_clone.buffer { + let buffer_source = BufferSource::Single(Cursor::new( + &buffer[start..current_buffer_len], + )); + dispatch_to_parse_csv( + has_header, + parse_options, + buffer_source, + projection_indices, + fields, + read_daft_fields, + read_schema, + csv_buffer, + &include_columns, + predicate, + sender, + rows_read, + ); + } else { + panic!("Trying to read from a CSV buffer that doesn't exist. Please report this issue.") + } + } + (Some(start), Some(end)) => { + if let Some(next_slab_clone) = next_slab_clone + && let Some(current_buffer) = ¤t_slab_clone.buffer + && let Some(next_buffer) = &next_slab_clone.buffer + { + let buffer_source = BufferSource::Chain(std::io::Read::chain( + Cursor::new(¤t_buffer[start..current_buffer_len]), + Cursor::new(&next_buffer[..end]), + )); + dispatch_to_parse_csv( + has_header, + parse_options, + buffer_source, + projection_indices, + fields, + read_daft_fields, + read_schema, + csv_buffer, + &include_columns, + predicate, + sender, + rows_read, + ); + } else { + panic!("Trying to read from an overflow CSV buffer that doesn't exist. Please report this issue.") + } + } + _ => panic!( + "Something went wrong when parsing the CSV file. Please report this issue." + ), + }; + } + }); + has_header = false; + if total_bytes_read >= total_len { + break; + } + } +} + +/// Unsafe helper function that extracts the buffer from an &Option>. Users should +/// ensure that the buffer is Some, otherwise this function causes the process to panic. +fn unsafe_clone_buffer(buffer: &Option>) -> Arc<[u8; SLABSIZE]> { + match buffer { + Some(buffer) => Arc::clone(buffer), + None => panic!("Tried to clone a CSV slab that doesn't exist. Please report this error."), + } +} + +#[allow(clippy::doc_lazy_continuation)] +/// Helper function that determines what chunk of data to parse given a starting position within the +/// file, and the desired initial chunk size. +/// +/// Given a starting position, we use our chunk size to compute a preliminary start and stop +/// position. For example, we can visualize all preliminary chunks in a file as follows. +/// +/// Chunk 1 Chunk 2 Chunk 3 Chunk N +/// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ +/// │ │ │\n │ │ \n │ │ \n │ +/// │ │ │ │ │ │ │ │ +/// │ │ │ \n │ │ │ │ │ +/// │ \n │ │ │ │ \n │ │ │ +/// │ │ │ │ │ │ ... │ \n │ +/// │ │ │ \n │ │ │ │ │ +/// │ \n │ │ │ │ │ │ │ +/// │ │ │ │ │ \n │ │ \n │ +/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +/// +/// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary +/// chunk boundaries. So we adjust each preliminary chunk as follows: +/// - Find the first record terminator from the chunk's start. This is the new starting position. +/// - Find the first record terminator from the chunk's end. This is the new ending position. +/// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. +/// +/// For example: +/// +/// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N +/// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ +/// │ \n││ \n│ │ \n│ \n │ │ +/// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ +/// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ +/// │ \n │ │ │ │ \n │ │ │ +/// │ │ │ │ │ │ ... │ \n │ +/// │ │ │ \n │ │ │ │ │ +/// │ \n │ │ │ │ │ │ │ +/// │ │ │ │ │ \n │ │ \n │ +/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +/// +/// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do +/// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can +/// now happen in parallel. +/// +/// This is the same method as described in: +/// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). +/// +/// Another observation is that seeing a pure \n character is not necessarily indicative of a record +/// terminator. We need to consider whether the \n character was seen within a quoted field, since the +/// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following +/// algorithm: +/// 1. Find a \n character. +/// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse +/// as valid CSV, and does it produce the same number of fields as our schema. +/// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. +/// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next +/// \n character and go back to 2. +#[allow(clippy::too_many_arguments)] +fn get_file_chunk( + current_buffer: Arc<[u8; SLABSIZE]>, + current_buffer_len: usize, + next_buffer: Option>, + next_buffer_len: usize, + first_buffer: bool, + num_fields: usize, + quote_char: u8, + field_delimiter: u8, + escape_char: Option, + double_quote_escape_allowed: bool, +) -> (Option, Option) { + // TODO(desmond): There is a potential fast path here when `escape_char` is None: simply check for \n characters. + let start = if !first_buffer { + let start = next_line_position( + ¤t_buffer[..current_buffer_len], + 0, + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ); + match start { + Some(_) => start, + None => return (None, None), // If the record size is >= 4MB, return None and fallback. + } + } else { + Some(0) + }; + // If there is a next buffer, find the adjusted chunk in that buffer. If there's no next buffer, we're at the end of the file. + let end = if let Some(next_buffer) = next_buffer { + let end = next_line_position( + &next_buffer[..next_buffer_len], + 0, + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ); + match end { + Some(_) => end, + None => return (None, None), // If the record size is >= 4MB, return None and fallback. + } + } else { + None + }; + (start, end) +} + +/// Helper function that finds the first valid record terminator in a buffer. +fn next_line_position( + buffer: &[u8], + offset: usize, + num_fields: usize, + quote_char: u8, + field_delimiter: u8, + escape_char: Option, + double_quote_escape_allowed: bool, +) -> Option { + let mut start = offset; + loop { + start = match newline_position(&buffer[start..]) { + // Start reading after the first record terminator from the start of the chunk. + Some(pos) => start + pos + 1, + None => return None, + }; + if start >= buffer.len() { + return None; + } + if validate_csv_record( + &buffer[start..], + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ) { + return Some(start); + } + } +} + +// Daft does not currently support non-\n record terminators (e.g. carriage return \r, which only +// matters for pre-Mac OS X). +const NEWLINE: u8 = b'\n'; +const DOUBLE_QUOTE: u8 = b'"'; +const DEFAULT_CHUNK_SIZE: usize = SLABSIZE; // 4MiB. Like SLABSIZE, this can be tuned. + +/// Helper function that finds the first new line character (\n) in the given byte slice. +fn newline_position(buffer: &[u8]) -> Option { + // Assuming we are searching for the ASCII `\n` character, we don't need to do any special + // handling for UTF-8, since a `\n` value always corresponds to an ASCII `\n`. + // For more details, see: https://en.wikipedia.org/wiki/UTF-8#Encoding + memchr::memchr(NEWLINE, buffer) +} + +/// Csv states used by the state machine in `validate_csv_record`. +#[derive(Clone)] +enum CsvState { + FieldStart, + RecordEnd, + UnquotedField, + QuotedField, + Unquote, +} + +/// State machine that validates whether the current buffer starts at a valid csv record. +/// See `get_file_chunk` for more details. +fn validate_csv_record( + buffer: &[u8], + num_fields: usize, + quote_char: u8, + field_delimiter: u8, + escape_char: Option, + double_quote_escape_allowed: bool, +) -> bool { + let mut state = CsvState::FieldStart; + let mut index = 0; + let mut num_fields_seen = 0; + loop { + if index >= buffer.len() { + // We've reached the end of the buffer without seeing a valid record. + return false; + } + match state { + CsvState::FieldStart => { + let byte = buffer[index]; + if byte == NEWLINE { + state = CsvState::RecordEnd; + } else if byte == quote_char { + state = CsvState::QuotedField; + index += 1; + } else { + state = CsvState::UnquotedField; + } + } + CsvState::RecordEnd => { + return num_fields_seen == num_fields; + } + CsvState::UnquotedField => { + // We follow the convention where an unquoted field does consider escape characters. + while index < buffer.len() { + let byte = buffer[index]; + if byte == NEWLINE { + num_fields_seen += 1; + state = CsvState::RecordEnd; + break; + } + if byte == field_delimiter { + num_fields_seen += 1; + state = CsvState::FieldStart; + index += 1; + break; + } + index += 1; + } + } + CsvState::QuotedField => { + while index < buffer.len() { + let byte = buffer[index]; + if byte == quote_char { + state = CsvState::Unquote; + index += 1; + break; + } + if let Some(escape_char) = escape_char + && byte == escape_char + { + // Skip the next character. + index += 1; + } + index += 1; + } + } + CsvState::Unquote => { + let byte = buffer[index]; + if let Some(escape_char) = escape_char + && byte == escape_char + && escape_char == quote_char + && (byte != DOUBLE_QUOTE || double_quote_escape_allowed) + { + state = CsvState::QuotedField; + index += 1; + continue; + } + if byte == NEWLINE { + num_fields_seen += 1; + state = CsvState::RecordEnd; + continue; + } + if byte == field_delimiter { + num_fields_seen += 1; + state = CsvState::FieldStart; + index += 1; + continue; + } + // Other characters are not allowed after a quote. This is invalid CSV. + return false; + } + } + } +} + +/// Helper function that takes in a BufferSource, calls parse_csv() to extract table values from +/// the buffer source, then streams the results to `sender`. +#[allow(clippy::too_many_arguments)] +fn dispatch_to_parse_csv( + has_header: bool, + parse_options: CsvParseOptions, + buffer_source: BufferSource, + projection_indices: Arc>, + fields: Vec, + read_daft_fields: Arc>>, + read_schema: Arc, + csv_buffer: CsvBuffer, + include_columns: &Option>, + predicate: Option>, + sender: Sender>, + rows_read: Arc, +) { + let table_results = { + let rdr = ReaderBuilder::new() + .has_headers(has_header) + .delimiter(parse_options.delimiter) + .double_quote(parse_options.double_quote) + .quote(parse_options.quote) + .escape(parse_options.escape_char) + .comment(parse_options.comment) + .flexible(parse_options.allow_variable_columns) + .from_reader(buffer_source); + parse_csv_chunk( + rdr, + projection_indices, + fields, + read_daft_fields, + read_schema, + csv_buffer, + include_columns, + predicate, + ) + }; + match table_results { + Ok(tables) => { + for table in tables { + let table_len = table.len(); + sender.send(Ok(table)).unwrap(); + // Atomically update the number of rows read only after the result has + // been sent. In theory we could wrap these steps in a mutex, but + // applying limit at this layer can be best-effort with no adverse + // side effects. + rows_read.fetch_add(table_len, Ordering::Relaxed); + } + } + Err(e) => sender.send(Err(e)).unwrap(), + } +} + +/// Helper function that consumes a CSV reader and turns it into a vector of Daft tables. +#[allow(clippy::too_many_arguments)] +fn parse_csv_chunk( + mut reader: Reader, + projection_indices: Arc>, + fields: Vec, + read_daft_fields: Arc>>, + read_schema: Arc, + csv_buffer: CsvBuffer, + include_columns: &Option>, + predicate: Option>, +) -> DaftResult> +where + R: std::io::Read, +{ + let mut chunk_buffer = csv_buffer.buffer; + let mut tables = vec![]; + loop { + //let time = Instant::now(); + let (rows_read, has_more) = + local_read_rows(&mut reader, chunk_buffer.as_mut_slice()).context(ArrowSnafu {})?; + //let time = Instant::now(); + let chunk = projection_indices + .par_iter() + .enumerate() + .map(|(i, proj_idx)| { + let deserialized_col = deserialize_column( + &chunk_buffer[0..rows_read], + *proj_idx, + fields[*proj_idx].data_type().clone(), + 0, + ); + Series::try_from_field_and_arrow_array( + read_daft_fields[i].clone(), + cast_array_for_daft_if_needed(deserialized_col?), + ) + }) + .collect::>>()?; + let num_rows = chunk.first().map(|s| s.len()).unwrap_or(0); + let table = Table::new_unchecked(read_schema.clone(), chunk, num_rows); + let table = if let Some(predicate) = &predicate { + let filtered = table.filter(&[predicate.clone()])?; + if let Some(include_columns) = &include_columns { + filtered.get_columns(include_columns.as_slice())? + } else { + filtered + } + } else { + table + }; + tables.push(table); + + // The number of record might exceed the number of byte records we've allocated. + // Retry until all byte records in this chunk are read. + if !has_more { + break; + } + } + csv_buffer.pool.return_buffer(chunk_buffer); + Ok(tables) +} diff --git a/src/daft-csv/src/read.rs b/src/daft-csv/src/read.rs index ce5cb556e4..1991fb81ae 100644 --- a/src/daft-csv/src/read.rs +++ b/src/daft-csv/src/read.rs @@ -1,8 +1,10 @@ +use core::str; use std::{collections::HashMap, num::NonZeroUsize, sync::Arc}; use arrow2::{ datatypes::Field, - io::csv::read_async::{read_rows, AsyncReaderBuilder, ByteRecord}, + io::csv::read_async, + io::csv::read_async::{read_rows, AsyncReaderBuilder}, }; use async_compat::{Compat, CompatExt}; use common_error::{DaftError, DaftResult}; @@ -11,7 +13,7 @@ use daft_compression::CompressionCodec; use daft_core::{prelude::*, utils::arrow::cast_array_for_daft_if_needed}; use daft_decoding::deserialize::deserialize_column; use daft_dsl::optimization::get_required_columns; -use daft_io::{get_runtime, GetResult, IOClient, IOStatsRef}; +use daft_io::{get_runtime, parse_url, GetResult, IOClient, IOStatsRef, SourceType}; use daft_table::Table; use futures::{stream::BoxStream, Stream, StreamExt, TryStreamExt}; use rayon::{ @@ -34,8 +36,13 @@ use crate::{ CsvReadOptions, }; -trait ByteRecordChunkStream: Stream>> {} -impl ByteRecordChunkStream for S where S: Stream>> {} +trait ByteRecordChunkStream: Stream>> {} +impl ByteRecordChunkStream for S where + S: Stream>> +{ +} + +use crate::{local::read_csv_local, local::stream_csv_local}; type TableChunkResult = super::Result>, super::JoinSnafu, super::Error>>; @@ -145,23 +152,37 @@ pub async fn stream_csv( io_stats: Option, max_chunks_in_flight: Option, ) -> DaftResult>> { - let stream = stream_csv_single( - &uri, - convert_options, - parse_options, - read_options, - io_client, - io_stats, - max_chunks_in_flight, - ) - .await?; - - Ok(Box::pin(stream)) + let uri = uri.as_str(); + let (source_type, _) = parse_url(uri)?; + let is_compressed = CompressionCodec::from_uri(uri).is_some(); + if matches!(source_type, SourceType::File) && !is_compressed { + let stream = stream_csv_local( + uri, + convert_options, + parse_options.unwrap_or_default(), + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) + .await?; + Ok(Box::pin(stream)) + } else { + let stream = stream_csv_single( + uri, + convert_options, + parse_options, + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) + .await?; + Ok(Box::pin(stream)) + } } -// Parallel version of table concat -// get rid of this once Table APIs are parallel -fn tables_concat(mut tables: Vec
) -> DaftResult
{ +pub fn tables_concat(mut tables: Vec
) -> DaftResult
{ if tables.is_empty() { return Err(DaftError::ValueError( "Need at least 1 Table to perform concat".to_string(), @@ -209,6 +230,21 @@ async fn read_csv_single_into_table( io_stats: Option, max_chunks_in_flight: Option, ) -> DaftResult
{ + let (source_type, _) = parse_url(uri)?; + let is_compressed = CompressionCodec::from_uri(uri).is_some(); + if matches!(source_type, SourceType::File) && !is_compressed { + return read_csv_local( + uri, + convert_options, + parse_options.unwrap_or_default(), + read_options, + io_client, + io_stats, + max_chunks_in_flight, + ) + .await; + } + let predicate = convert_options .as_ref() .and_then(|opts| opts.predicate.clone()); @@ -326,7 +362,7 @@ async fn read_csv_single_into_table( } } -async fn stream_csv_single( +pub async fn stream_csv_single( uri: &str, convert_options: Option, parse_options: Option, @@ -557,7 +593,7 @@ where estimated_rows_per_desired_chunk.max(8).min(num_rows - total_rows_read) }; let mut chunk_buffer = vec![ - ByteRecord::with_capacity(record_buffer_size, num_fields); + read_async::ByteRecord::with_capacity(record_buffer_size, num_fields); chunk_size_rows ]; @@ -576,7 +612,7 @@ where chunk_buffer.truncate(rows_read); if rows_read > 0 { - yield chunk_buffer + yield chunk_buffer; } } } @@ -638,7 +674,7 @@ fn parse_into_column_array_chunk_stream( })) } -fn fields_to_projection_indices( +pub fn fields_to_projection_indices( fields: &[arrow2::datatypes::Field], include_columns: &Option>, ) -> Arc> { diff --git a/src/daft-decoding/src/deserialize.rs b/src/daft-decoding/src/deserialize.rs index 3552cc57a8..495c7cd904 100644 --- a/src/daft-decoding/src/deserialize.rs +++ b/src/daft-decoding/src/deserialize.rs @@ -5,12 +5,13 @@ use arrow2::{ }, datatypes::{DataType, TimeUnit}, error::{Error, Result}, + io::csv, offset::Offset, temporal_conversions, types::NativeType, }; use chrono::{Datelike, Timelike}; -use csv_async::ByteRecord; +use csv_async; pub(crate) const ISO8601: &str = "%+"; pub(crate) const ISO8601_NO_TIME_ZONE: &str = "%Y-%m-%dT%H:%M:%S%.f"; @@ -38,7 +39,14 @@ pub trait ByteRecordGeneric { fn get(&self, index: usize) -> Option<&[u8]>; } -impl ByteRecordGeneric for ByteRecord { +impl ByteRecordGeneric for csv_async::ByteRecord { + #[inline] + fn get(&self, index: usize) -> Option<&[u8]> { + self.get(index) + } +} + +impl ByteRecordGeneric for csv::read::ByteRecord { #[inline] fn get(&self, index: usize) -> Option<&[u8]> { self.get(index) From 58ad170954b68a08d53f1eff3463c3c3928d6919 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 15 Oct 2024 19:33:08 -0700 Subject: [PATCH 02/38] Fix rebase --- src/daft-csv/src/lib.rs | 1 + src/daft-csv/src/local.rs | 69 +++++++++++++++++++++++---------------- src/daft-csv/src/read.rs | 8 +++-- 3 files changed, 46 insertions(+), 32 deletions(-) diff --git a/src/daft-csv/src/lib.rs b/src/daft-csv/src/lib.rs index 1e90f2eb25..ce1a3615a9 100644 --- a/src/daft-csv/src/lib.rs +++ b/src/daft-csv/src/lib.rs @@ -1,5 +1,6 @@ #![feature(async_closure)] #![feature(let_chains)] +#![feature(new_uninit)] #![feature(trait_alias)] #![feature(trait_upcasting)] #![feature(test)] diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index dad5f7b280..28a6d124f3 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -1,20 +1,27 @@ use core::str; -use std::io::{Chain, Cursor, Read}; -use std::sync::atomic::{AtomicUsize, Ordering}; -use std::{num::NonZeroUsize, sync::Arc, sync::Condvar, sync::Mutex}; +use std::{ + io::{Chain, Cursor, Read}, + num::NonZeroUsize, + sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, Condvar, Mutex, + }, +}; -use crate::metadata::read_csv_schema_single; -use crate::ArrowSnafu; -use crate::{CsvConvertOptions, CsvParseOptions, CsvReadOptions}; use arrow2::{ datatypes::Field, - io::csv::read, - io::csv::read::{Reader, ReaderBuilder}, - io::csv::read_async::local_read_rows, + io::csv::{ + read, + read::{Reader, ReaderBuilder}, + read_async::local_read_rows, + }, }; use common_error::{DaftError, DaftResult}; use crossbeam_channel::Sender; -use daft_core::{schema::Schema, utils::arrow::cast_array_for_daft_if_needed, Series}; +use daft_core::{ + prelude::{Schema, Series}, + utils::arrow::cast_array_for_daft_if_needed, +}; use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; @@ -26,9 +33,13 @@ use rayon::{ }; use snafu::ResultExt; -use crate::read::{fields_to_projection_indices, tables_concat}; +use crate::{ + metadata::read_csv_schema_single, + read::{fields_to_projection_indices, tables_concat}, + ArrowSnafu, CsvConvertOptions, CsvParseOptions, CsvReadOptions, +}; -#[allow(clippy::doc_lazy_continuation)] +#[allow(clippy::doc_lazy_continuation, clippy::empty_line_after_doc_comments)] /// Our local CSV reader takes the following approach to reading CSV files: /// 1. Read the CSV file in 4MB chunks from a slab pool. /// 2. Adjust the chunks so that chunks are contiguous and contain complete @@ -86,7 +97,7 @@ impl CsvBufferPool { ]; initial_pool_size ]; - CsvBufferPool { + Self { buffers: Mutex::new(chunk_buffers), buffer_size: chunk_size_rows, record_buffer_size, @@ -125,7 +136,7 @@ const SLABPOOL_DEFAULT_SIZE: usize = 20; /// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] struct SlabPool { - buffers: Mutex>>, + buffers: Mutex>>, condvar: Condvar, } @@ -135,7 +146,7 @@ struct Slab { pool: Arc, // We wrap the Arc in an Option so that when a Slab is being dropped, we can move the Slab's reference // to the Arc back to the slab pool. - buffer: Option>, + buffer: Option>, } impl Drop for Slab { @@ -149,16 +160,18 @@ impl Drop for Slab { impl SlabPool { pub fn new() -> Self { - let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) - .map(|_| Arc::new([0; SLABSIZE])) + let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + .map(|_| Box::new_uninit_slice(SLABSIZE)) + .map(|x| unsafe { x.assume_init() }) + .map(Arc::from) .collect(); - SlabPool { + Self { buffers: Mutex::new(chunk_buffers), condvar: Condvar::new(), } } - pub fn get_buffer(self: &Arc) -> Arc<[u8; SLABSIZE]> { + pub fn get_buffer(self: &Arc) -> Arc<[u8]> { let mut buffers = self.buffers.lock().unwrap(); while buffers.is_empty() { // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. @@ -168,7 +181,7 @@ impl SlabPool { buffers.pop().unwrap() } - fn return_buffer(&self, buffer: Arc<[u8; SLABSIZE]>) { + fn return_buffer(&self, buffer: Arc<[u8]>) { let mut buffers = self.buffers.lock().unwrap(); buffers.push(buffer); self.condvar.notify_one(); @@ -257,7 +270,7 @@ pub async fn stream_csv_local( let required_columns_for_predicate = get_required_columns(predicate); for rc in required_columns_for_predicate { if include_columns.iter().all(|c| c.as_str() != rc.as_str()) { - include_columns.push(rc) + include_columns.push(rc); } } } @@ -280,16 +293,14 @@ pub async fn stream_csv_local( .as_ref() .and_then(|opt| opt.chunk_size.or_else(|| opt.buffer_size.map(|bs| bs / 8))) .unwrap_or(DEFAULT_CHUNK_SIZE); - let projection_indices = fields_to_projection_indices( - &schema.clone().fields, - &convert_options.clone().include_columns, - ); + let projection_indices = + fields_to_projection_indices(&schema.fields, &convert_options.clone().include_columns); let fields = schema.clone().fields; let fields_subset = projection_indices .iter() .map(|i| fields.get(*i).unwrap().into()) .collect::>(); - let read_schema = Arc::new(daft_core::schema::Schema::new(fields_subset)?); + let read_schema = Arc::new(Schema::new(fields_subset)?); let read_daft_fields = Arc::new( read_schema .fields @@ -573,7 +584,7 @@ fn consume_csv_file( /// Unsafe helper function that extracts the buffer from an &Option>. Users should /// ensure that the buffer is Some, otherwise this function causes the process to panic. -fn unsafe_clone_buffer(buffer: &Option>) -> Arc<[u8; SLABSIZE]> { +fn unsafe_clone_buffer(buffer: &Option>) -> Arc<[u8]> { match buffer { Some(buffer) => Arc::clone(buffer), None => panic!("Tried to clone a CSV slab that doesn't exist. Please report this error."), @@ -638,9 +649,9 @@ fn unsafe_clone_buffer(buffer: &Option>) -> Arc<[u8; SLABSIZ /// \n character and go back to 2. #[allow(clippy::too_many_arguments)] fn get_file_chunk( - current_buffer: Arc<[u8; SLABSIZE]>, + current_buffer: Arc<[u8]>, current_buffer_len: usize, - next_buffer: Option>, + next_buffer: Option>, next_buffer_len: usize, first_buffer: bool, num_fields: usize, diff --git a/src/daft-csv/src/read.rs b/src/daft-csv/src/read.rs index 1991fb81ae..07c53448f4 100644 --- a/src/daft-csv/src/read.rs +++ b/src/daft-csv/src/read.rs @@ -3,8 +3,10 @@ use std::{collections::HashMap, num::NonZeroUsize, sync::Arc}; use arrow2::{ datatypes::Field, - io::csv::read_async, - io::csv::read_async::{read_rows, AsyncReaderBuilder}, + io::csv::{ + read_async, + read_async::{read_rows, AsyncReaderBuilder}, + }, }; use async_compat::{Compat, CompatExt}; use common_error::{DaftError, DaftResult}; @@ -42,7 +44,7 @@ impl ByteRecordChunkStream for S where { } -use crate::{local::read_csv_local, local::stream_csv_local}; +use crate::local::{read_csv_local, stream_csv_local}; type TableChunkResult = super::Result>, super::JoinSnafu, super::Error>>; From 1c4881ca3f6220158824757bec4ea43ebf98d279 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 15 Oct 2024 19:38:44 -0700 Subject: [PATCH 03/38] Remove unused deps --- Cargo.lock | 2 -- src/daft-csv/Cargo.toml | 2 -- 2 files changed, 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b78da50d76..ade5517d93 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1771,9 +1771,7 @@ dependencies = [ "daft-io", "daft-table", "futures", - "indexmap 2.5.0", "memchr", - "memmap2", "pyo3", "rayon", "rstest", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index 27f1cc2275..ca74b9c2ec 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -13,9 +13,7 @@ daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} -indexmap = {workspace = true, features = ["serde"]} memchr = "2.7.2" -memmap2 = "0.9.4" pyo3 = {workspace = true, optional = true} rayon = {workspace = true} serde = {workspace = true} From 6cbd1f85f73324955472b542bd1ee72005d26246 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 15 Oct 2024 19:40:14 -0700 Subject: [PATCH 04/38] Remove rebase artifact --- Makefile | 4 ---- 1 file changed, 4 deletions(-) diff --git a/Makefile b/Makefile index 327f00513c..d1d96e8df6 100644 --- a/Makefile +++ b/Makefile @@ -52,10 +52,6 @@ build: check-toolchain .venv ## Compile and install Daft for development build-release: check-toolchain .venv ## Compile and install a faster Daft binary @unset CONDA_PREFIX && PYO3_PYTHON=$(VENV_BIN)/python $(VENV_BIN)/maturin develop --release -.PHONY: build-bench -build-bench: check-toolchain .venv ## Compile and install a faster Daft binary - @unset CONDA_PREFIX && PYO3_PYTHON=$(VENV_BIN)/python $(VENV_BIN)/maturin develop --profile dev-bench - .PHONY: test test: .venv build ## Run tests HYPOTHESIS_MAX_EXAMPLES=$(HYPOTHESIS_MAX_EXAMPLES) $(VENV_BIN)/pytest --hypothesis-seed=$(HYPOTHESIS_SEED) From 3136a1c80dfca437634e8d64eae5c9b9b2d6ace1 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 15 Oct 2024 21:20:39 -0700 Subject: [PATCH 05/38] Fix tests --- src/daft-csv/src/local.rs | 99 +++++++++++++++++++++++++++------------ 1 file changed, 68 insertions(+), 31 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 28a6d124f3..8453c229a5 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -26,7 +26,7 @@ use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; -use futures::{stream::BoxStream, Stream, StreamExt}; +use futures::Stream; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -189,7 +189,7 @@ impl SlabPool { } /// A data structure that holds either a single slice of bytes, or a chain of two slices of bytes. -/// See the description to `parse_json` for more details. +/// See the description to `parse_csv` for more details. #[derive(Debug)] enum BufferSource<'a> { Single(Cursor<&'a [u8]>), @@ -217,27 +217,59 @@ pub async fn read_csv_local( ) -> DaftResult
{ let stream = stream_csv_local( uri, - convert_options, - parse_options, + convert_options.clone(), + parse_options.clone(), read_options, - io_client, - io_stats, + io_client.clone(), + io_stats.clone(), max_chunks_in_flight, ) .await?; - tables_concat(tables_stream_collect(Box::pin(stream)).await) -} - -async fn tables_stream_collect(stream: BoxStream<'static, DaftResult
>) -> Vec
{ - stream - .filter_map(|result| async { - match result { - Ok(table) => Some(table), - Err(_) => None, // Skips errors; you could log them or handle differently + let tables = Box::pin(stream); + // Apply limit. + let limit = convert_options.as_ref().and_then(|opts| opts.limit); + let mut remaining_rows = limit.map(|limit| limit as i64); + use futures::TryStreamExt; + let collected_tables = tables + .try_take_while(|result| { + match (result, remaining_rows) { + // Limit has been met, early-terminate. + (_, Some(rows_left)) if rows_left <= 0 => futures::future::ready(Ok(false)), + // Limit has not yet been met, update remaining limit slack and continue. + (table, Some(rows_left)) => { + remaining_rows = Some(rows_left - table.len() as i64); + futures::future::ready(Ok(true)) + } + // (1) No limit, never early-terminate. + // (2) Encountered error, propagate error to try_collect to allow it to short-circuit. + (_, None) => futures::future::ready(Ok(true)), } }) - .collect() - .await + .try_collect::>() + .await? + .into_iter() + .collect::>(); + // Handle empty table case. + if collected_tables.is_empty() { + let (schema, _, _) = get_schema_and_estimators( + uri, + &convert_options.unwrap_or_default(), + &parse_options, + io_client, + io_stats, + ) + .await?; + return Table::empty(Some(Arc::new(Schema::try_from(&schema)?))); + } + let concated_table = tables_concat(collected_tables)?; + if let Some(limit) = limit + && concated_table.len() > limit + { + // apply head in case that last chunk went over limit + concated_table.head(limit) + } else { + Ok(concated_table) + } } pub async fn stream_csv_local( @@ -252,16 +284,14 @@ pub async fn stream_csv_local( let uri = uri.trim_start_matches("file://"); let file = std::fs::File::open(uri)?; + // Process the CV convert options. let predicate = convert_options .as_ref() .and_then(|opts| opts.predicate.clone()); - let limit = convert_options.as_ref().and_then(|opts| opts.limit); - let include_columns = convert_options .as_ref() .and_then(|opts| opts.include_columns.clone()); - let convert_options = match (convert_options, &predicate) { (None, _) => None, (co, None) => co, @@ -281,18 +311,13 @@ pub async fn stream_csv_local( } .unwrap_or_default(); + // Create slab pool for file reads. let slabpool = Arc::new(SlabPool::new()); + + // Get schema and row estimations. let (schema, estimated_mean_row_size, estimated_std_row_size) = get_schema_and_estimators(uri, &convert_options, &parse_options, io_client, io_stats) .await?; - - let n_threads: usize = std::thread::available_parallelism() - .unwrap_or(NonZeroUsize::new(2).unwrap()) - .into(); - let chunk_size = read_options - .as_ref() - .and_then(|opt| opt.chunk_size.or_else(|| opt.buffer_size.map(|bs| bs / 8))) - .unwrap_or(DEFAULT_CHUNK_SIZE); let projection_indices = fields_to_projection_indices(&schema.fields, &convert_options.clone().include_columns); let fields = schema.clone().fields; @@ -308,7 +333,16 @@ pub async fn stream_csv_local( .map(|f| Arc::new(f.clone())) .collect::>(), ); + + // Create CSV buffer pool. + let n_threads: usize = std::thread::available_parallelism() + .unwrap_or(NonZeroUsize::new(2).unwrap()) + .into(); let record_buffer_size = (estimated_mean_row_size + estimated_std_row_size).ceil() as usize; + let chunk_size = read_options + .as_ref() + .and_then(|opt| opt.chunk_size.or_else(|| opt.buffer_size.map(|bs| bs / 8))) + .unwrap_or(DEFAULT_CHUNK_SIZE); let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; let num_fields = schema.fields.len(); // TODO(desmond): We might consider creating per-process buffer pools and slab pools. @@ -318,10 +352,13 @@ pub async fn stream_csv_local( chunk_size_rows, n_threads * 2, )); + // We suppose that each slab of CSV data produces (chunk size / slab size) number of Daft tables. We // then double this capacity to ensure that our channel is never full and our threads won't deadlock. let (sender, receiver) = crossbeam_channel::bounded(max_chunks_in_flight.unwrap_or(2 * chunk_size / SLABSIZE)); + + // Consume the CSV file asynchronously. rayon::spawn(move || { consume_csv_file( file, @@ -380,8 +417,8 @@ async fn get_schema_and_estimators( } Ok(( schema, - read_stats.mean_record_size_bytes, - read_stats.stddev_record_size_bytes, + read_stats.mean_record_size_bytes.max(8_f64), + read_stats.stddev_record_size_bytes.max(8_f64), )) } @@ -570,7 +607,7 @@ fn consume_csv_file( } } _ => panic!( - "Something went wrong when parsing the CSV file. Please report this issue." + "Reached an unreachable state when parsing the CSV file. Please report this issue." ), }; } From 61b58c0fc92915c4975708e3c71260b42caa57ff Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 16 Oct 2024 11:34:21 -0700 Subject: [PATCH 06/38] Add fallback --- src/daft-csv/src/local.rs | 91 +++++++++++++++++++++++++++------------ 1 file changed, 63 insertions(+), 28 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 8453c229a5..35a4569d72 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -1,6 +1,6 @@ use core::str; use std::{ - io::{Chain, Cursor, Read}, + io::{prelude::Seek, Chain, Cursor, Read, SeekFrom}, num::NonZeroUsize, sync::{ atomic::{AtomicUsize, Ordering}, @@ -83,6 +83,9 @@ struct CsvBuffer { buffer: Vec, } +// If regular 4 MiB buffers are too small, we fall back to using large 2 GiB buffers. +const HUGE_BUFFER_SIZE: usize = 2 * 1024 * 1024 * 1024; + impl CsvBufferPool { pub fn new( record_buffer_size: usize, @@ -122,6 +125,16 @@ impl CsvBufferPool { } } + // Creates a single huge 2 GiB buffer for the fallback reader. + pub fn create_huge_buffer(self: &Arc) -> CsvBuffer { + let buffer = vec![read::ByteRecord::with_capacity(HUGE_BUFFER_SIZE, self.num_fields); 1]; + + CsvBuffer { + pool: Arc::clone(self), + buffer, + } + } + fn return_buffer(&self, buffer: Vec) { let mut buffers = self.buffers.lock().unwrap(); buffers.push(buffer); @@ -135,21 +148,21 @@ const SLABPOOL_DEFAULT_SIZE: usize = 20; /// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] -struct SlabPool { +struct FileSlabPool { buffers: Mutex>>, condvar: Condvar, } /// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. #[derive(Clone)] -struct Slab { - pool: Arc, +struct FileSlab { + pool: Arc, // We wrap the Arc in an Option so that when a Slab is being dropped, we can move the Slab's reference // to the Arc back to the slab pool. buffer: Option>, } -impl Drop for Slab { +impl Drop for FileSlab { fn drop(&mut self) { // Move the buffer back to the slab pool. if let Some(buffer) = self.buffer.take() { @@ -158,7 +171,7 @@ impl Drop for Slab { } } -impl SlabPool { +impl FileSlabPool { pub fn new() -> Self { let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) .map(|_| Box::new_uninit_slice(SLABSIZE)) @@ -191,17 +204,19 @@ impl SlabPool { /// A data structure that holds either a single slice of bytes, or a chain of two slices of bytes. /// See the description to `parse_csv` for more details. #[derive(Debug)] -enum BufferSource<'a> { +enum CsvBufferSource<'a> { Single(Cursor<&'a [u8]>), Chain(Chain, Cursor<&'a [u8]>>), + File(std::fs::File), } /// Read implementation that allows BufferSource to be used by csv::read::Reader. -impl<'a> Read for BufferSource<'a> { +impl<'a> Read for CsvBufferSource<'a> { fn read(&mut self, buf: &mut [u8]) -> std::io::Result { match self { - BufferSource::Single(cursor) => std::io::Read::read(cursor, buf), - BufferSource::Chain(chain) => chain.read(buf), + CsvBufferSource::Single(cursor) => std::io::Read::read(cursor, buf), + CsvBufferSource::Chain(chain) => chain.read(buf), + CsvBufferSource::File(file) => file.read(buf), } } } @@ -312,7 +327,7 @@ pub async fn stream_csv_local( .unwrap_or_default(); // Create slab pool for file reads. - let slabpool = Arc::new(SlabPool::new()); + let slabpool = Arc::new(FileSlabPool::new()); // Get schema and row estimations. let (schema, estimated_mean_row_size, estimated_std_row_size) = @@ -427,7 +442,7 @@ async fn get_schema_and_estimators( fn consume_csv_file( mut file: std::fs::File, buffer_pool: Arc, - slabpool: Arc, + slabpool: Arc, parse_options: CsvParseOptions, projection_indices: Arc>, read_daft_fields: Arc>>, @@ -474,7 +489,7 @@ fn consume_csv_file( } total_bytes_read += bytes_read; ( - Arc::new(Slab { + Arc::new(FileSlab { pool: Arc::clone(&slabpool), buffer: Some(buffer), }), @@ -498,7 +513,7 @@ fn consume_csv_file( (None, 0) } else { ( - Some(Arc::new(Slab { + Some(Arc::new(FileSlab { pool: Arc::clone(&slabpool), buffer: Some(next_buffer), })), @@ -530,12 +545,34 @@ fn consume_csv_file( ); first_buffer = false; if let (None, _) = file_chunk { - // Return the buffer. It doesn't matter that we still have a reference to the slab. We're going to fallback - // and the slabs will be useless. - slabpool.return_buffer(unsafe_clone_buffer(¤t_slab.buffer)); - // Exit early before spawning a new thread. + // The current file slab does not have records that fit into 4 MiB. Fallback to a standard reader. + total_bytes_read -= current_buffer_len; + // Seek to the undecoded portion of the file. + match file.seek(SeekFrom::Start(total_bytes_read as u64)) { + Ok(_) => { + // Since this CSV file has records larger than 4 MiB, get a huge buffer that accepts 2 GiB records. + let csv_buffer = buffer_pool.create_huge_buffer(); + dispatch_to_parse_csv( + has_header, + parse_options, + CsvBufferSource::File(file), + projection_indices, + fields, + read_daft_fields, + read_schema, + csv_buffer, + include_columns.clone(), + predicate, + sender, + rows_read, + ); + } + Err(e) => { + sender.send(Err(DaftError::IoError(e))).unwrap(); + } + } + // Exit after falling back. break; - // TODO(desmond): we should fallback instead. } let current_slab_clone = Arc::clone(¤t_slab); let next_slab_clone = next_slab.clone(); @@ -558,7 +595,7 @@ fn consume_csv_file( match file_chunk { (Some(start), None) => { if let Some(buffer) = ¤t_slab_clone.buffer { - let buffer_source = BufferSource::Single(Cursor::new( + let buffer_source = CsvBufferSource::Single(Cursor::new( &buffer[start..current_buffer_len], )); dispatch_to_parse_csv( @@ -570,7 +607,7 @@ fn consume_csv_file( read_daft_fields, read_schema, csv_buffer, - &include_columns, + include_columns, predicate, sender, rows_read, @@ -584,7 +621,7 @@ fn consume_csv_file( && let Some(current_buffer) = ¤t_slab_clone.buffer && let Some(next_buffer) = &next_slab_clone.buffer { - let buffer_source = BufferSource::Chain(std::io::Read::chain( + let buffer_source = CsvBufferSource::Chain(std::io::Read::chain( Cursor::new(¤t_buffer[start..current_buffer_len]), Cursor::new(&next_buffer[..end]), )); @@ -597,7 +634,7 @@ fn consume_csv_file( read_daft_fields, read_schema, csv_buffer, - &include_columns, + include_columns, predicate, sender, rows_read, @@ -896,13 +933,13 @@ fn validate_csv_record( fn dispatch_to_parse_csv( has_header: bool, parse_options: CsvParseOptions, - buffer_source: BufferSource, + buffer_source: CsvBufferSource, projection_indices: Arc>, fields: Vec, read_daft_fields: Arc>>, read_schema: Arc, csv_buffer: CsvBuffer, - include_columns: &Option>, + include_columns: Option>, predicate: Option>, sender: Sender>, rows_read: Arc, @@ -953,7 +990,7 @@ fn parse_csv_chunk( read_daft_fields: Arc>>, read_schema: Arc, csv_buffer: CsvBuffer, - include_columns: &Option>, + include_columns: Option>, predicate: Option>, ) -> DaftResult> where @@ -962,10 +999,8 @@ where let mut chunk_buffer = csv_buffer.buffer; let mut tables = vec![]; loop { - //let time = Instant::now(); let (rows_read, has_more) = local_read_rows(&mut reader, chunk_buffer.as_mut_slice()).context(ArrowSnafu {})?; - //let time = Instant::now(); let chunk = projection_indices .par_iter() .enumerate() From ef23acb04daaf6c94c38cae1246128a8d816b0ad Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 16 Oct 2024 13:19:23 -0700 Subject: [PATCH 07/38] Cleanup --- src/daft-csv/src/local.rs | 74 +++++++++++++++++++++------------------ 1 file changed, 40 insertions(+), 34 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 35a4569d72..0fabc05efe 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -39,34 +39,33 @@ use crate::{ ArrowSnafu, CsvConvertOptions, CsvParseOptions, CsvReadOptions, }; -#[allow(clippy::doc_lazy_continuation, clippy::empty_line_after_doc_comments)] -/// Our local CSV reader takes the following approach to reading CSV files: -/// 1. Read the CSV file in 4MB chunks from a slab pool. -/// 2. Adjust the chunks so that chunks are contiguous and contain complete -/// CSV records. See `get_file_chunk` for more details. -/// 3. In parallel with the above, convert the adjusted chunks into byte records, -/// which are stored within pre-allocated CSV buffers. -/// 4. In parallel with the above, deserialize each CSV buffer into a Daft table -/// and stream the results. -/// -/// Slab Pool CSV Buffer Pool -/// ┌────────────────────┐ ┌────────────────────┐ -/// │ 4MB Chunks │ │ CSV Buffers │ -/// │┌───┐┌───┐┌───┐ │ │┌───┐┌───┐┌───┐ │ -/// ││ ││ ││ │ ... │ ││ ││ ││ │ ... │ -/// │└─┬─┘└─┬─┘└───┘ │ │└─┬─┘└─┬─┘└───┘ │ -/// └──┼────┼────────────┘ └──┼────┼────────────┘ -/// │ │ │ │ -/// ───────┐ │ │ │ │ -/// /│ │ │ │ │ │ -/// /─┘ │ │ │ │ │ -/// │ │ ▼ ▼ ▼ ▼ -/// │ | ┌───┐ ┌───┐ ┌────┐ ┬--─┐ ┌───┐ ┌───┐ ┌───┐ ┌───┐ -/// │ ─┼───►│ │ │ │ ──────► │ ┬┘┌─┘ ┬─┘ ───────► │ │ │ │ ──────────► │ │ │ │ -/// │ CSV File │ └───┘ └───┘ └───┴ └───┘ └───┘ └───┘ └───┘ └───┘ -/// │ │ Chain of buffers Adjusted chunks Vectors of ByteRecords Stream of Daft tables -/// │ │ -/// └──────────┘ +// Our local CSV reader takes the following approach to reading CSV files: +// 1. Read the CSV file in 4MB chunks from a slab pool. +// 2. Adjust the chunks so that chunks are contiguous and contain complete +// CSV records. See `get_file_chunk` for more details. +// 3. In parallel with the above, convert the adjusted chunks into byte records, +// which are stored within pre-allocated CSV buffers. +// 4. In parallel with the above, deserialize each CSV buffer into a Daft table +// and stream the results. +// +// Slab Pool CSV Buffer Pool +// ┌────────────────────┐ ┌────────────────────┐ +// │ 4MB Chunks │ │ CSV Buffers │ +// │┌───┐┌───┐┌───┐ │ │┌───┐┌───┐┌───┐ │ +// ││ ││ ││ │ ... │ ││ ││ ││ │ ... │ +// │└─┬─┘└─┬─┘└───┘ │ │└─┬─┘└─┬─┘└───┘ │ +// └──┼────┼────────────┘ └──┼────┼────────────┘ +// │ │ │ │ +// ───────┐ │ │ │ │ +// /│ │ │ │ │ │ +// /─┘ │ │ │ │ │ +// │ │ ▼ ▼ ▼ ▼ +// │ | ┌───┐ ┌───┐ ┌────┐ ┬--─┐ ┌───┐ ┌───┐ ┌───┐ ┌───┐ +// │ ─┼───►│ │ │ │ ──────► │ ┬┘┌─┘ ┬─┘ ───────► │ │ │ │ ──────────► │ │ │ │ +// │ CSV File │ └───┘ └───┘ └───┴ └───┘ └───┘ └───┘ └───┘ └───┘ +// │ │ Chain of buffers Adjusted chunks Vectors of ByteRecords Stream of Daft tables +// │ │ +// └──────────┘ /// A pool of ByteRecord slabs. Used for deserializing CSV. #[derive(Debug)] @@ -143,7 +142,7 @@ impl CsvBufferPool { // The default size of a slab used for reading CSV files in chunks. Currently set to 4 MiB. This can be tuned. const SLABSIZE: usize = 4 * 1024 * 1024; -// The default number of slabs in a slab pool. +// The default number of slabs in a slab pool. With 20 slabs, we reserve a total of 80 MiB of memory for reading file data. const SLABPOOL_DEFAULT_SIZE: usize = 20; /// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. @@ -162,9 +161,9 @@ struct FileSlab { buffer: Option>, } +// Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. impl Drop for FileSlab { fn drop(&mut self) { - // Move the buffer back to the slab pool. if let Some(buffer) = self.buffer.take() { self.pool.return_buffer(buffer); } @@ -174,6 +173,7 @@ impl Drop for FileSlab { impl FileSlabPool { pub fn new() -> Self { let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + // We get uninitialized buffers because we will always populate the buffers with a file read before use. .map(|_| Box::new_uninit_slice(SLABSIZE)) .map(|x| unsafe { x.assume_init() }) .map(Arc::from) @@ -201,8 +201,12 @@ impl FileSlabPool { } } -/// A data structure that holds either a single slice of bytes, or a chain of two slices of bytes. -/// See the description to `parse_csv` for more details. +/// A data structure that holds either: +/// 1. A single slice of bytes. +/// 2. A chain of two slices of bytes. +/// 3. A file. +/// +/// See the description to `dispatch_to_parse_csv` for more details. #[derive(Debug)] enum CsvBufferSource<'a> { Single(Cursor<&'a [u8]>), @@ -221,6 +225,7 @@ impl<'a> Read for CsvBufferSource<'a> { } } +/// Reads a single local CSV file in a non-streaming fashion. pub async fn read_csv_local( uri: &str, convert_options: Option, @@ -280,13 +285,14 @@ pub async fn read_csv_local( if let Some(limit) = limit && concated_table.len() > limit { - // apply head in case that last chunk went over limit + // Apply head in case that last chunk went over limit. concated_table.head(limit) } else { Ok(concated_table) } } +/// Reads a single local CSV file in a streaming fashion. pub async fn stream_csv_local( uri: &str, convert_options: Option, @@ -299,7 +305,7 @@ pub async fn stream_csv_local( let uri = uri.trim_start_matches("file://"); let file = std::fs::File::open(uri)?; - // Process the CV convert options. + // Process the CSV convert options. let predicate = convert_options .as_ref() .and_then(|opts| opts.predicate.clone()); From 04493391a51969f265b83e1721fa5d823ec5c537 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 16 Oct 2024 13:29:15 -0700 Subject: [PATCH 08/38] Fix comment --- src/daft-csv/src/local.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 0fabc05efe..16e721265d 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -870,7 +870,7 @@ fn validate_csv_record( return num_fields_seen == num_fields; } CsvState::UnquotedField => { - // We follow the convention where an unquoted field does consider escape characters. + // We follow the convention where an unquoted field does not consider escape characters. while index < buffer.len() { let byte = buffer[index]; if byte == NEWLINE { From b64a1358cf0b0864fdc6bdd3880b0691c7605c56 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Wed, 16 Oct 2024 19:12:13 -0700 Subject: [PATCH 09/38] wip --- Cargo.lock | 1 + src/daft-csv/Cargo.toml | 1 + src/daft-csv/src/local.rs | 145 ++++++++++++++++++++++++++++++++++++++ 3 files changed, 147 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index ade5517d93..573ac3fd7d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1771,6 +1771,7 @@ dependencies = [ "daft-io", "daft-table", "futures", + "itertools 0.11.0", "memchr", "pyo3", "rayon", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index ca74b9c2ec..bfb99cddac 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -13,6 +13,7 @@ daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} +itertools = {workspace = true} memchr = "2.7.2" pyo3 = {workspace = true, optional = true} rayon = {workspace = true} diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 16e721265d..eaf4973395 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -27,6 +27,7 @@ use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; use futures::Stream; +use itertools::Itertools; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -159,6 +160,7 @@ struct FileSlab { // We wrap the Arc in an Option so that when a Slab is being dropped, we can move the Slab's reference // to the Arc back to the slab pool. buffer: Option>, + valid_bytes: usize, } // Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. @@ -443,6 +445,147 @@ async fn get_schema_and_estimators( )) } +struct SlabIterator { + file: std::fs::File, + slabpool: Arc, + total_bytes_read: usize, +} + +impl SlabIterator { + fn new(file: std::fs::File, slabpool: Arc) -> Self { + Self { + file, + slabpool, + total_bytes_read: 0, + } + } +} + +impl Iterator for SlabIterator { + type Item = Arc; + fn next(&mut self) -> Option { + let mut buffer = self.slabpool.get_buffer(); + match Arc::get_mut(&mut buffer) { + Some(inner_buffer) => { + let bytes_read = self.file.read(inner_buffer).unwrap(); + if bytes_read == 0 { + self.slabpool.return_buffer(buffer); + return None; + } + self.total_bytes_read += bytes_read; + Some(Arc::new(FileSlab { + pool: Arc::clone(&self.slabpool), + buffer: Some(buffer), + valid_bytes: bytes_read, + })) + } + None => { + self.slabpool.return_buffer(buffer); + panic!("We should have exclusive access to this mutable buffer"); + } + } + } +} + +fn consume_slab_iterator( + iter: impl Iterator>, + num_fields: usize, + parse_options: &CsvParseOptions, +) { + let field_delimiter = parse_options.delimiter; + let escape_char = parse_options.escape_char; + let quote_char = parse_options.quote; + let double_quote_escape_allowed = parse_options.double_quote; + + let mut iter = iter.peekable(); + let mut first_buffer = true; + while let Some(first) = iter.next() { + let (second, second_num_bytes) = if let Some(second) = iter.peek() { + (Some(second), second.valid_bytes) + } else { + (None, 0) + }; + + let file_chunk = get_file_chunk( + unsafe_clone_buffer(&first.buffer), + first.valid_bytes, + second.map(|s| unsafe_clone_buffer(&s.buffer)), + second_num_bytes, + first_buffer, + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ); + + first_buffer = false; + } +} + +enum ChunkState { + Start { + slab: Arc, + start: usize, + }, + StartAndFinal { + slab: Arc, + start: usize, + end: usize, + }, + Continue { + slab: Arc, + }, + Final { + slab: Arc, + end: usize, + }, +} + +impl ChunkState { + #[inline] + fn is_final(&self) -> bool { + matches!( + self, + ChunkState::Final { .. } | ChunkState::StartAndFinal { .. } + ) + } +} + +struct SlabConsumer<'a, I> { + slab_iter: &'a mut I, +} + +impl<'a, I> Iterator for SlabConsumer<'a, I> +where + I: Iterator>, +{ + type Item = ChunkState; + fn next(&mut self) -> Option { + let mut so_far = vec![]; + while let Some(data) = self.slab_iter.next() { + todo!() + } + + // verify last item is final or StartAndFinal + todo!("emit state") + } +} + +struct MultiSliceReader<'a> { + // https://stackoverflow.com/questions/71801199/how-can-concatenated-u8-slices-implement-the-read-trait-without-additional-co + // use small vec + state: Vec<&'a [u8]>, + curr_idx: usize, + curr_offset: usize, +} + +impl Read for MultiSliceReader { + fn read(&mut self, buf: &mut [u8]) -> std::io::Result {} + fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} + fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} +} + /// Consumes the CSV file and sends the results to `sender`. #[allow(clippy::too_many_arguments)] fn consume_csv_file( @@ -498,6 +641,7 @@ fn consume_csv_file( Arc::new(FileSlab { pool: Arc::clone(&slabpool), buffer: Some(buffer), + valid_bytes: bytes_read, }), bytes_read, ) @@ -522,6 +666,7 @@ fn consume_csv_file( Some(Arc::new(FileSlab { pool: Arc::clone(&slabpool), buffer: Some(next_buffer), + valid_bytes: bytes_read, })), bytes_read, ) From c5c8cbda062b165bfee51dbe28d4920ae303e747 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 18 Oct 2024 11:49:46 -0700 Subject: [PATCH 10/38] Make it work --- src/daft-csv/src/local.rs | 1051 ++++++++++++++++++++----------------- 1 file changed, 559 insertions(+), 492 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index eaf4973395..dee825012d 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -1,6 +1,6 @@ use core::str; use std::{ - io::{prelude::Seek, Chain, Cursor, Read, SeekFrom}, + io::Read, num::NonZeroUsize, sync::{ atomic::{AtomicUsize, Ordering}, @@ -27,7 +27,6 @@ use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; use futures::Stream; -use itertools::Itertools; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -83,9 +82,6 @@ struct CsvBuffer { buffer: Vec, } -// If regular 4 MiB buffers are too small, we fall back to using large 2 GiB buffers. -const HUGE_BUFFER_SIZE: usize = 2 * 1024 * 1024 * 1024; - impl CsvBufferPool { pub fn new( record_buffer_size: usize, @@ -125,16 +121,6 @@ impl CsvBufferPool { } } - // Creates a single huge 2 GiB buffer for the fallback reader. - pub fn create_huge_buffer(self: &Arc) -> CsvBuffer { - let buffer = vec![read::ByteRecord::with_capacity(HUGE_BUFFER_SIZE, self.num_fields); 1]; - - CsvBuffer { - pool: Arc::clone(self), - buffer, - } - } - fn return_buffer(&self, buffer: Vec) { let mut buffers = self.buffers.lock().unwrap(); buffers.push(buffer); @@ -154,7 +140,7 @@ struct FileSlabPool { } /// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. -#[derive(Clone)] +#[derive(Clone, Debug)] struct FileSlab { pool: Arc, // We wrap the Arc in an Option so that when a Slab is being dropped, we can move the Slab's reference @@ -163,6 +149,16 @@ struct FileSlab { valid_bytes: usize, } +impl FileSlab { + // Check whether this FileSlab filled up its internal buffer. We use this as a means of checking whether this + // is the last FileSlab in the current file. + // + // This assumption is only true if the user does not append to the current file while we are reading it. + pub fn filled_buffer(&self) -> bool { + self.valid_bytes >= unsafe_clone_buffer(&self.buffer).len() + } +} + // Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. impl Drop for FileSlab { fn drop(&mut self) { @@ -203,30 +199,6 @@ impl FileSlabPool { } } -/// A data structure that holds either: -/// 1. A single slice of bytes. -/// 2. A chain of two slices of bytes. -/// 3. A file. -/// -/// See the description to `dispatch_to_parse_csv` for more details. -#[derive(Debug)] -enum CsvBufferSource<'a> { - Single(Cursor<&'a [u8]>), - Chain(Chain, Cursor<&'a [u8]>>), - File(std::fs::File), -} - -/// Read implementation that allows BufferSource to be used by csv::read::Reader. -impl<'a> Read for CsvBufferSource<'a> { - fn read(&mut self, buf: &mut [u8]) -> std::io::Result { - match self { - CsvBufferSource::Single(cursor) => std::io::Read::read(cursor, buf), - CsvBufferSource::Chain(chain) => chain.read(buf), - CsvBufferSource::File(file) => file.read(buf), - } - } -} - /// Reads a single local CSV file in a non-streaming fashion. pub async fn read_csv_local( uri: &str, @@ -383,20 +355,22 @@ pub async fn stream_csv_local( // Consume the CSV file asynchronously. rayon::spawn(move || { - consume_csv_file( - file, + let slab_iterator = SlabIterator::new(file, slabpool); + let rows_read = Arc::new(AtomicUsize::new(0)); + consume_slab_iterator( + slab_iterator, buffer_pool, - slabpool, + num_fields, parse_options, projection_indices, read_daft_fields, read_schema, fields, - num_fields, - &include_columns, + include_columns, predicate, limit, sender, + rows_read, ); }); @@ -481,48 +455,416 @@ impl Iterator for SlabIterator { } None => { self.slabpool.return_buffer(buffer); - panic!("We should have exclusive access to this mutable buffer"); + panic!("We should have exclusive access to this mutable buffer."); } } } } +#[allow(clippy::too_many_arguments)] fn consume_slab_iterator( iter: impl Iterator>, + buffer_pool: Arc, num_fields: usize, - parse_options: &CsvParseOptions, + parse_options: CsvParseOptions, + projection_indices: Arc>, + read_daft_fields: Arc>>, + read_schema: Arc, + fields: Vec, + include_columns: Option>, + predicate: Option>, + limit: Option, + sender: Sender>, + rows_read: Arc, ) { + let mut has_header = parse_options.has_header; let field_delimiter = parse_options.delimiter; let escape_char = parse_options.escape_char; let quote_char = parse_options.quote; let double_quote_escape_allowed = parse_options.double_quote; let mut iter = iter.peekable(); - let mut first_buffer = true; - while let Some(first) = iter.next() { - let (second, second_num_bytes) = if let Some(second) = iter.peek() { - (Some(second), second.valid_bytes) - } else { - (None, 0) - }; + let mut curr_chunk = ChunkStateHolder::empty(); + loop { + // Check limit. + let limit_reached = limit.map_or(false, |limit| { + let current_rows_read = rows_read.load(Ordering::Relaxed); + current_rows_read >= limit + }); + if limit_reached { + break; + } + // Grab a starting file slab if the current CSV chunk is empty. + if curr_chunk.is_empty() { + if let Some(next) = iter.next() { + curr_chunk.states.push(ChunkState::Start { + slab: next, + start: 0, + }); + curr_chunk.reset(); + } else { + // EOF. + break; + } + continue; + } + // Grab file slabs until we find a valid CSV chunk. + loop { + if let Some(next) = iter.next() { + // If the next buffer is not completely filled, we take this to mean that we've reached EOF. + if !next.filled_buffer() { + curr_chunk.states.push(ChunkState::Final { + end: next.valid_bytes, + slab: next, + }); + break; + } + curr_chunk.states.push(ChunkState::Continue { slab: next }); + while curr_chunk.goto_next_newline() { + if curr_chunk.validate_csv_record( + &mut iter, + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ) { + break; + } + } + if curr_chunk.is_valid() { + break; + } + } else { + // If there is no next file slab, turn the last ChunkState into a final ChunkState. + if let Some(last_state) = curr_chunk.states.pop() { + match last_state { + ChunkState::Start { slab, start } => { + curr_chunk.states.push(ChunkState::StartAndFinal { + end: slab.valid_bytes, + slab, + start, + }); + } + ChunkState::Continue { slab } => { + curr_chunk.states.push(ChunkState::Final { + end: slab.valid_bytes, + slab, + }); + } + _ => panic!("There should be no final CSV chunk states at this point."), + } + } else { + panic!("There should be at least one CSV chunk state at this point.") + } + break; + } + } + let states_to_read = curr_chunk.states; + curr_chunk.states = std::mem::take(&mut curr_chunk.next_states); + curr_chunk.reset(); + let parse_options = parse_options.clone(); + let csv_buffer = buffer_pool.get_buffer(); + let projection_indices = projection_indices.clone(); + let fields = fields.clone(); + let read_daft_fields = read_daft_fields.clone(); + let read_schema = read_schema.clone(); + let include_columns = include_columns.clone(); + let predicate = predicate.clone(); + let sender = sender.clone(); + let rows_read = Arc::clone(&rows_read); + rayon::spawn(move || { + if let Some(state) = states_to_read.last() { + assert!(state.is_final()); + } else { + return; + } + let multi_slice_reader = MultiSliceReader::new(states_to_read); + dispatch_to_parse_csv( + has_header, + parse_options, + multi_slice_reader, + projection_indices, + fields, + read_daft_fields, + read_schema, + csv_buffer, + include_columns, + predicate, + sender, + rows_read, + ); + }); + has_header = false; + } +} - let file_chunk = get_file_chunk( - unsafe_clone_buffer(&first.buffer), - first.valid_bytes, - second.map(|s| unsafe_clone_buffer(&s.buffer)), - second_num_bytes, - first_buffer, - num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ); +struct ChunkStateHolder { + states: Vec, + next_states: Vec, + curr_newline_idx: usize, + curr_newline_offset: usize, + curr_byte_read_idx: usize, + curr_byte_read_offset: usize, + valid_chunk: bool, +} + +impl ChunkStateHolder { + fn new(states: Vec) -> Self { + Self { + states, + next_states: vec![], + curr_newline_idx: 1, + curr_newline_offset: 0, + curr_byte_read_idx: 0, + curr_byte_read_offset: 0, + valid_chunk: false, + } + } + + fn empty() -> Self { + Self::new(vec![]) + } - first_buffer = false; + fn is_empty(&self) -> bool { + self.states.is_empty() + } + + #[allow(clippy::doc_lazy_continuation)] + /// `goto_next_line` and `validate_csv_record` are two helper function that determines what chunk of + /// data to parse given a starting position within the file, and the desired initial chunk size. + /// + /// Given a starting position, we use our chunk size to compute a preliminary start and stop + /// position. For example, we can visualize all preliminary chunks in a file as follows. + /// + /// Chunk 1 Chunk 2 Chunk 3 Chunk N + /// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ + /// │ │ │\n │ │ \n │ │ \n │ + /// │ │ │ │ │ │ │ │ + /// │ │ │ \n │ │ │ │ │ + /// │ \n │ │ │ │ \n │ │ │ + /// │ │ │ │ │ │ ... │ \n │ + /// │ │ │ \n │ │ │ │ │ + /// │ \n │ │ │ │ │ │ │ + /// │ │ │ │ │ \n │ │ \n │ + /// └──────────┘ └──────────┘ └──────────┘ └──────────┘ + /// + /// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary + /// chunk boundaries. So we adjust each preliminary chunk as follows: + /// - Find the first record terminator from the chunk's start. This is the new starting position. + /// - Find the first record terminator from the chunk's end. This is the new ending position. + /// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. + /// + /// For example: + /// + /// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N + /// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ + /// │ \n││ \n│ │ \n│ \n │ │ + /// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ + /// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ + /// │ \n │ │ │ │ \n │ │ │ + /// │ │ │ │ │ │ ... │ \n │ + /// │ │ │ \n │ │ │ │ │ + /// │ \n │ │ │ │ │ │ │ + /// │ │ │ │ │ \n │ │ \n │ + /// └──────────┘ └──────────┘ └──────────┘ └──────────┘ + /// + /// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do + /// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can + /// now happen in parallel. + /// + /// This is the same method as described in: + /// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). + /// + /// Another observation is that seeing a pure \n character is not necessarily indicative of a record + /// terminator. We need to consider whether the \n character was seen within a quoted field, since the + /// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following + /// algorithm: + /// 1. Find a \n character. + /// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse + /// as valid CSV, and does it produce the same number of fields as our schema. + /// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. + /// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next + /// \n character and go back to 2. + fn goto_next_newline(&mut self) -> bool { + //println!("newline idx {}, newline offset {}", self.curr_newline_idx, self.curr_newline_offset); + self.valid_chunk = false; + loop { + if self.curr_newline_idx >= self.states.len() { + //println!("no newline found"); + return false; + } + if self.curr_newline_offset >= self.states[self.curr_newline_idx].len() { + //println!("offset larger than length of current state"); + self.curr_newline_offset = 0; + self.curr_newline_idx += 1; + continue; + } + if let Some(pos) = + self.states[self.curr_newline_idx].find_newline(self.curr_newline_offset) + { + //println!("found newline at {}, {}", self.curr_newline_idx, pos); + self.curr_newline_offset = pos + 1; + if self.curr_newline_offset >= self.states[self.curr_newline_idx].len() { + self.curr_newline_offset = 0; + self.curr_newline_idx += 1; + continue; + } + self.curr_byte_read_idx = self.curr_newline_idx; + self.curr_byte_read_offset = self.curr_newline_offset; + return true; + } else { + //println!("no newline found, going to next idx.."); + self.curr_newline_offset = 0; + self.curr_newline_idx += 1; + continue; + } + } + } + + fn validate_csv_record( + &mut self, + iter: &mut impl Iterator>, + num_fields: usize, + quote_char: u8, + field_delimiter: u8, + escape_char: Option, + double_quote_escape_allowed: bool, + ) -> bool { + let mut csv_state = CsvState::FieldStart; + let mut num_fields_seen: usize = 0; + loop { + // Run the CSV state machine to see if we're currently at a valid record boundary. + if let Some(valid) = validate_csv_record( + self.into_iter(), + &mut csv_state, + num_fields, + &mut num_fields_seen, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ) { + // println!("validity: {valid}"); + self.valid_chunk = valid; + if valid { + // println!("Valid! states: {:?}", self.states); + let (ending_idx, ending_offset) = if self.curr_byte_read_offset == 0 { + ( + self.curr_byte_read_idx - 1, + self.states[self.curr_byte_read_idx - 1].len(), + ) + } else { + (self.curr_byte_read_idx, self.curr_byte_read_offset) + }; + self.next_states = self.states.split_off(ending_idx); + if let Some(front_of_next) = self.next_states.get_mut(0) { + self.states.push(ChunkState::Final { + slab: front_of_next.clone_slab(), + end: ending_offset, + }); + *front_of_next = ChunkState::Start { + slab: front_of_next.clone_slab(), + start: ending_offset, + }; + } else { + panic!("There should be at least one chunk state that's split off."); + } + // println!("After transformation: {:?}", self.states); + // println!("leftovers:{:?}", self.next_states); + } + return valid; + } else { + //println!("state machine needs more bytes??"); + // We ran out of bytes while running the CSV state machine. Read another file slab then + // continue running the state machine. + if let Some(next) = iter.next() { + if !next.filled_buffer() { + // EOF. Make this chunk state holder valid and exit. + self.states.push(ChunkState::Final { + end: next.valid_bytes, + slab: next, + }); + self.valid_chunk = true; + return true; + } + self.states.push(ChunkState::Continue { slab: next }); + } else { + // EOF. Make this chunk state holder valid and exit. + self.valid_chunk = true; + return true; + } + } + } + } + + fn is_valid(&self) -> bool { + self.valid_chunk + } + + fn reset(&mut self) { + self.curr_newline_idx = 1; + self.curr_newline_offset = 0; + self.curr_byte_read_idx = 0; + self.curr_byte_read_offset = 0; } } +impl Iterator for ChunkStateHolder { + type Item = u8; + + fn next(&mut self) -> Option { + loop { + if self.curr_byte_read_idx >= self.states.len() { + return None; + } + if self.curr_byte_read_offset >= self.states[self.curr_byte_read_idx].len() { + self.curr_byte_read_offset = 0; + self.curr_byte_read_idx += 1; + continue; + } + let byte = self.states[self.curr_byte_read_idx].get_idx(self.curr_byte_read_offset); + self.curr_byte_read_offset += 1; + return Some(byte); + } + } +} + +// impl Read for ChunkStateHolder { +// fn read(&mut self, buf: &mut [u8]) -> std::io::Result { +// let current_state = loop { +// if self.curr_read_idx >= self.states.len() { +// return Ok(0); // EOF +// } +// let current_state = &self.states[self.curr_read_idx]; +// if self.curr_read_offset < current_state.len() { +// break current_state; +// } +// self.curr_read_offset = 0; +// self.curr_read_idx += 1; +// }; +// let slice = match current_state { +// ChunkState::Start { slab, start } => { +// &unsafe_clone_buffer(&slab.buffer)[*start..slab.valid_bytes] +// } +// ChunkState::StartAndFinal { slab, start, end } => { +// &unsafe_clone_buffer(&slab.buffer)[*start..*end] +// } +// ChunkState::Continue { slab } => &unsafe_clone_buffer(&slab.buffer)[..slab.valid_bytes], +// ChunkState::Final { slab, end } => &unsafe_clone_buffer(&slab.buffer)[..*end], +// }; +// let read_size = buf.len().min(slice.len() - self.curr_read_offset); +// buf[..read_size].copy_from_slice(&slice[self.curr_read_offset..][..read_size]); +// self.curr_read_offset += read_size; +// Ok(read_size) +// } + +// // fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} + +// // fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} +// } + +#[derive(Debug)] enum ChunkState { Start { slab: Arc, @@ -545,266 +887,135 @@ enum ChunkState { impl ChunkState { #[inline] fn is_final(&self) -> bool { - matches!( - self, - ChunkState::Final { .. } | ChunkState::StartAndFinal { .. } - ) + matches!(self, Self::Final { .. } | Self::StartAndFinal { .. }) } -} -struct SlabConsumer<'a, I> { - slab_iter: &'a mut I, -} + #[inline] + fn len(&self) -> usize { + match self { + Self::Start { slab, start } => slab.valid_bytes - start, + Self::StartAndFinal { start, end, .. } => end - start, + Self::Continue { slab } => slab.valid_bytes, + Self::Final { end, .. } => *end, + } + } -impl<'a, I> Iterator for SlabConsumer<'a, I> -where - I: Iterator>, -{ - type Item = ChunkState; - fn next(&mut self) -> Option { - let mut so_far = vec![]; - while let Some(data) = self.slab_iter.next() { - todo!() + #[inline] + fn get_idx(&self, idx: usize) -> u8 { + match self { + Self::Start { slab, start } => unsafe_clone_buffer(&slab.buffer)[start + idx], + Self::StartAndFinal { slab, start, .. } => { + unsafe_clone_buffer(&slab.buffer)[start + idx] + } + Self::Continue { slab } => unsafe_clone_buffer(&slab.buffer)[idx], + Self::Final { slab, .. } => unsafe_clone_buffer(&slab.buffer)[idx], + } + } + + #[inline] + fn find_newline(&self, offset: usize) -> Option { + match self { + Self::Start { slab, start } => newline_position( + &unsafe_clone_buffer(&slab.buffer)[*start + offset..slab.valid_bytes], + ), + Self::StartAndFinal { slab, start, end } => { + newline_position(&unsafe_clone_buffer(&slab.buffer)[*start + offset..*end]) + } + Self::Continue { slab } => { + newline_position(&unsafe_clone_buffer(&slab.buffer)[offset..slab.valid_bytes]) + } + Self::Final { slab, end } => { + newline_position(&unsafe_clone_buffer(&slab.buffer)[offset..*end]) + } + } + } + + #[inline] + fn get_slab(&self) -> &Arc { + match self { + Self::Start { slab, .. } => slab, + Self::StartAndFinal { slab, .. } => slab, + Self::Continue { slab } => slab, + Self::Final { slab, .. } => slab, } + } - // verify last item is final or StartAndFinal - todo!("emit state") + #[inline] + fn clone_slab(&self) -> Arc { + self.get_slab().clone() } } -struct MultiSliceReader<'a> { +// struct SlabConsumer<'a, I> { +// slab_iter: &'a mut I, +// } + +// impl<'a, I> Iterator for SlabConsumer<'a, I> +// where +// I: Iterator>, +// { +// type Item = ChunkState; +// fn next(&mut self) -> Option { +// // let mut so_far = vec![]; +// while let Some(data) = self.slab_iter.next() { +// todo!() +// } + +// // verify last item is final or StartAndFinal +// todo!("emit state") +// } +// } + +struct MultiSliceReader { // https://stackoverflow.com/questions/71801199/how-can-concatenated-u8-slices-implement-the-read-trait-without-additional-co // use small vec - state: Vec<&'a [u8]>, - curr_idx: usize, - curr_offset: usize, + states: Vec, + curr_read_idx: usize, + curr_read_offset: usize, } -impl Read for MultiSliceReader { - fn read(&mut self, buf: &mut [u8]) -> std::io::Result {} - fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} - fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} +impl MultiSliceReader { + fn new(states: Vec) -> Self { + Self { + states, + curr_read_idx: 0, + curr_read_offset: 0, + } + } } -/// Consumes the CSV file and sends the results to `sender`. -#[allow(clippy::too_many_arguments)] -fn consume_csv_file( - mut file: std::fs::File, - buffer_pool: Arc, - slabpool: Arc, - parse_options: CsvParseOptions, - projection_indices: Arc>, - read_daft_fields: Arc>>, - read_schema: Arc, - fields: Vec, - num_fields: usize, - include_columns: &Option>, - predicate: Option>, - limit: Option, - sender: Sender>, -) { - let rows_read = Arc::new(AtomicUsize::new(0)); - let mut has_header = parse_options.has_header; - let total_len = file.metadata().unwrap().len() as usize; - let field_delimiter = parse_options.delimiter; - let escape_char = parse_options.escape_char; - let quote_char = parse_options.quote; - let double_quote_escape_allowed = parse_options.double_quote; - let mut total_bytes_read = 0; - let mut next_slab = None; - let mut next_buffer_len = 0; - let mut first_buffer = true; - loop { - let limit_reached = limit.map_or(false, |limit| { - let current_rows_read = rows_read.load(Ordering::Relaxed); - current_rows_read >= limit - }); - if limit_reached { - break; - } - let (current_slab, current_buffer_len) = match next_slab.take() { - Some(next_slab) => { - total_bytes_read += next_buffer_len; - (next_slab, next_buffer_len) - } - None => { - let mut buffer = slabpool.get_buffer(); - match Arc::get_mut(&mut buffer) { - Some(inner_buffer) => { - let bytes_read = file.read(inner_buffer).unwrap(); - if bytes_read == 0 { - slabpool.return_buffer(buffer); - break; - } - total_bytes_read += bytes_read; - ( - Arc::new(FileSlab { - pool: Arc::clone(&slabpool), - buffer: Some(buffer), - valid_bytes: bytes_read, - }), - bytes_read, - ) - } - None => { - slabpool.return_buffer(buffer); - break; - } - } +impl Read for MultiSliceReader { + fn read(&mut self, buf: &mut [u8]) -> std::io::Result { + let current_state = loop { + if self.curr_read_idx >= self.states.len() { + return Ok(0); // EOF } - }; - (next_slab, next_buffer_len) = if total_bytes_read < total_len { - let mut next_buffer = slabpool.get_buffer(); - match Arc::get_mut(&mut next_buffer) { - Some(inner_buffer) => { - let bytes_read = file.read(inner_buffer).unwrap(); - if bytes_read == 0 { - slabpool.return_buffer(next_buffer); - (None, 0) - } else { - ( - Some(Arc::new(FileSlab { - pool: Arc::clone(&slabpool), - buffer: Some(next_buffer), - valid_bytes: bytes_read, - })), - bytes_read, - ) - } - } - None => { - slabpool.return_buffer(next_buffer); - break; - } + let current_state = &self.states[self.curr_read_idx]; + if self.curr_read_offset < current_state.len() { + break current_state; } - } else { - (None, 0) + self.curr_read_offset = 0; + self.curr_read_idx += 1; }; - let file_chunk = get_file_chunk( - unsafe_clone_buffer(¤t_slab.buffer), - current_buffer_len, - next_slab - .as_ref() - .map(|slab| unsafe_clone_buffer(&slab.buffer)), - next_buffer_len, - first_buffer, - num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ); - first_buffer = false; - if let (None, _) = file_chunk { - // The current file slab does not have records that fit into 4 MiB. Fallback to a standard reader. - total_bytes_read -= current_buffer_len; - // Seek to the undecoded portion of the file. - match file.seek(SeekFrom::Start(total_bytes_read as u64)) { - Ok(_) => { - // Since this CSV file has records larger than 4 MiB, get a huge buffer that accepts 2 GiB records. - let csv_buffer = buffer_pool.create_huge_buffer(); - dispatch_to_parse_csv( - has_header, - parse_options, - CsvBufferSource::File(file), - projection_indices, - fields, - read_daft_fields, - read_schema, - csv_buffer, - include_columns.clone(), - predicate, - sender, - rows_read, - ); - } - Err(e) => { - sender.send(Err(DaftError::IoError(e))).unwrap(); - } + let slice = match current_state { + ChunkState::Start { slab, start } => { + &unsafe_clone_buffer(&slab.buffer)[*start..slab.valid_bytes] } - // Exit after falling back. - break; - } - let current_slab_clone = Arc::clone(¤t_slab); - let next_slab_clone = next_slab.clone(); - let parse_options = parse_options.clone(); - let csv_buffer = buffer_pool.get_buffer(); - let projection_indices = projection_indices.clone(); - let fields = fields.clone(); - let read_daft_fields = read_daft_fields.clone(); - let read_schema = read_schema.clone(); - let include_columns = include_columns.clone(); - let predicate = predicate.clone(); - let sender = sender.clone(); - let rows_read = Arc::clone(&rows_read); - rayon::spawn(move || { - let limit_reached = limit.map_or(false, |limit| { - let current_rows_read = rows_read.load(Ordering::Relaxed); - current_rows_read >= limit - }); - if !limit_reached { - match file_chunk { - (Some(start), None) => { - if let Some(buffer) = ¤t_slab_clone.buffer { - let buffer_source = CsvBufferSource::Single(Cursor::new( - &buffer[start..current_buffer_len], - )); - dispatch_to_parse_csv( - has_header, - parse_options, - buffer_source, - projection_indices, - fields, - read_daft_fields, - read_schema, - csv_buffer, - include_columns, - predicate, - sender, - rows_read, - ); - } else { - panic!("Trying to read from a CSV buffer that doesn't exist. Please report this issue.") - } - } - (Some(start), Some(end)) => { - if let Some(next_slab_clone) = next_slab_clone - && let Some(current_buffer) = ¤t_slab_clone.buffer - && let Some(next_buffer) = &next_slab_clone.buffer - { - let buffer_source = CsvBufferSource::Chain(std::io::Read::chain( - Cursor::new(¤t_buffer[start..current_buffer_len]), - Cursor::new(&next_buffer[..end]), - )); - dispatch_to_parse_csv( - has_header, - parse_options, - buffer_source, - projection_indices, - fields, - read_daft_fields, - read_schema, - csv_buffer, - include_columns, - predicate, - sender, - rows_read, - ); - } else { - panic!("Trying to read from an overflow CSV buffer that doesn't exist. Please report this issue.") - } - } - _ => panic!( - "Reached an unreachable state when parsing the CSV file. Please report this issue." - ), - }; + ChunkState::StartAndFinal { slab, start, end } => { + &unsafe_clone_buffer(&slab.buffer)[*start..*end] } - }); - has_header = false; - if total_bytes_read >= total_len { - break; - } + ChunkState::Continue { slab } => &unsafe_clone_buffer(&slab.buffer)[..slab.valid_bytes], + ChunkState::Final { slab, end } => &unsafe_clone_buffer(&slab.buffer)[..*end], + }; + let read_size = buf.len().min(slice.len() - self.curr_read_offset); + buf[..read_size].copy_from_slice(&slice[self.curr_read_offset..][..read_size]); + self.curr_read_offset += read_size; + Ok(read_size) } + + // fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} + + // fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} } /// Unsafe helper function that extracts the buffer from an &Option>. Users should @@ -816,147 +1027,6 @@ fn unsafe_clone_buffer(buffer: &Option>) -> Arc<[u8]> { } } -#[allow(clippy::doc_lazy_continuation)] -/// Helper function that determines what chunk of data to parse given a starting position within the -/// file, and the desired initial chunk size. -/// -/// Given a starting position, we use our chunk size to compute a preliminary start and stop -/// position. For example, we can visualize all preliminary chunks in a file as follows. -/// -/// Chunk 1 Chunk 2 Chunk 3 Chunk N -/// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ -/// │ │ │\n │ │ \n │ │ \n │ -/// │ │ │ │ │ │ │ │ -/// │ │ │ \n │ │ │ │ │ -/// │ \n │ │ │ │ \n │ │ │ -/// │ │ │ │ │ │ ... │ \n │ -/// │ │ │ \n │ │ │ │ │ -/// │ \n │ │ │ │ │ │ │ -/// │ │ │ │ │ \n │ │ \n │ -/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ -/// -/// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary -/// chunk boundaries. So we adjust each preliminary chunk as follows: -/// - Find the first record terminator from the chunk's start. This is the new starting position. -/// - Find the first record terminator from the chunk's end. This is the new ending position. -/// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. -/// -/// For example: -/// -/// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N -/// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ -/// │ \n││ \n│ │ \n│ \n │ │ -/// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ -/// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ -/// │ \n │ │ │ │ \n │ │ │ -/// │ │ │ │ │ │ ... │ \n │ -/// │ │ │ \n │ │ │ │ │ -/// │ \n │ │ │ │ │ │ │ -/// │ │ │ │ │ \n │ │ \n │ -/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ -/// -/// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do -/// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can -/// now happen in parallel. -/// -/// This is the same method as described in: -/// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). -/// -/// Another observation is that seeing a pure \n character is not necessarily indicative of a record -/// terminator. We need to consider whether the \n character was seen within a quoted field, since the -/// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following -/// algorithm: -/// 1. Find a \n character. -/// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse -/// as valid CSV, and does it produce the same number of fields as our schema. -/// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. -/// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next -/// \n character and go back to 2. -#[allow(clippy::too_many_arguments)] -fn get_file_chunk( - current_buffer: Arc<[u8]>, - current_buffer_len: usize, - next_buffer: Option>, - next_buffer_len: usize, - first_buffer: bool, - num_fields: usize, - quote_char: u8, - field_delimiter: u8, - escape_char: Option, - double_quote_escape_allowed: bool, -) -> (Option, Option) { - // TODO(desmond): There is a potential fast path here when `escape_char` is None: simply check for \n characters. - let start = if !first_buffer { - let start = next_line_position( - ¤t_buffer[..current_buffer_len], - 0, - num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ); - match start { - Some(_) => start, - None => return (None, None), // If the record size is >= 4MB, return None and fallback. - } - } else { - Some(0) - }; - // If there is a next buffer, find the adjusted chunk in that buffer. If there's no next buffer, we're at the end of the file. - let end = if let Some(next_buffer) = next_buffer { - let end = next_line_position( - &next_buffer[..next_buffer_len], - 0, - num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ); - match end { - Some(_) => end, - None => return (None, None), // If the record size is >= 4MB, return None and fallback. - } - } else { - None - }; - (start, end) -} - -/// Helper function that finds the first valid record terminator in a buffer. -fn next_line_position( - buffer: &[u8], - offset: usize, - num_fields: usize, - quote_char: u8, - field_delimiter: u8, - escape_char: Option, - double_quote_escape_allowed: bool, -) -> Option { - let mut start = offset; - loop { - start = match newline_position(&buffer[start..]) { - // Start reading after the first record terminator from the start of the chunk. - Some(pos) => start + pos + 1, - None => return None, - }; - if start >= buffer.len() { - return None; - } - if validate_csv_record( - &buffer[start..], - num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ) { - return Some(start); - } - } -} - // Daft does not currently support non-\n record terminators (e.g. carriage return \r, which only // matters for pre-Mac OS X). const NEWLINE: u8 = b'\n'; @@ -983,96 +1053,89 @@ enum CsvState { /// State machine that validates whether the current buffer starts at a valid csv record. /// See `get_file_chunk` for more details. +#[allow(clippy::too_many_arguments)] fn validate_csv_record( - buffer: &[u8], + iter: &mut impl Iterator, + state: &mut CsvState, num_fields: usize, + num_fields_seen: &mut usize, quote_char: u8, field_delimiter: u8, escape_char: Option, double_quote_escape_allowed: bool, -) -> bool { - let mut state = CsvState::FieldStart; - let mut index = 0; - let mut num_fields_seen = 0; +) -> Option { loop { - if index >= buffer.len() { - // We've reached the end of the buffer without seeing a valid record. - return false; - } match state { CsvState::FieldStart => { - let byte = buffer[index]; + *num_fields_seen += 1; + if *num_fields_seen > num_fields { + return Some(false); + } + let byte = iter.next()?; + // println!("field start byte {}", byte); if byte == NEWLINE { - state = CsvState::RecordEnd; + *state = CsvState::RecordEnd; } else if byte == quote_char { - state = CsvState::QuotedField; - index += 1; - } else { - state = CsvState::UnquotedField; + *state = CsvState::QuotedField; + } else if byte != field_delimiter { + *state = CsvState::UnquotedField; } } CsvState::RecordEnd => { - return num_fields_seen == num_fields; + return Some(*num_fields_seen == num_fields); } CsvState::UnquotedField => { // We follow the convention where an unquoted field does not consider escape characters. - while index < buffer.len() { - let byte = buffer[index]; + loop { + let byte = iter.next()?; + // println!("unquoted field byte {}", byte); if byte == NEWLINE { - num_fields_seen += 1; - state = CsvState::RecordEnd; + *state = CsvState::RecordEnd; break; } if byte == field_delimiter { - num_fields_seen += 1; - state = CsvState::FieldStart; - index += 1; + *state = CsvState::FieldStart; break; } - index += 1; } } CsvState::QuotedField => { - while index < buffer.len() { - let byte = buffer[index]; + loop { + let byte = iter.next()?; + // println!("quoted field byte {}", byte); if byte == quote_char { - state = CsvState::Unquote; - index += 1; + *state = CsvState::Unquote; break; } if let Some(escape_char) = escape_char && byte == escape_char { // Skip the next character. - index += 1; + iter.next()?; } - index += 1; } } CsvState::Unquote => { - let byte = buffer[index]; + let byte = iter.next()?; + // println!("unquote byte {}", byte); if let Some(escape_char) = escape_char && byte == escape_char && escape_char == quote_char && (byte != DOUBLE_QUOTE || double_quote_escape_allowed) { - state = CsvState::QuotedField; - index += 1; + *state = CsvState::QuotedField; continue; } if byte == NEWLINE { - num_fields_seen += 1; - state = CsvState::RecordEnd; + *state = CsvState::RecordEnd; continue; } if byte == field_delimiter { - num_fields_seen += 1; - state = CsvState::FieldStart; - index += 1; + *state = CsvState::FieldStart; continue; } // Other characters are not allowed after a quote. This is invalid CSV. - return false; + return Some(false); } } } @@ -1081,10 +1144,10 @@ fn validate_csv_record( /// Helper function that takes in a BufferSource, calls parse_csv() to extract table values from /// the buffer source, then streams the results to `sender`. #[allow(clippy::too_many_arguments)] -fn dispatch_to_parse_csv( +fn dispatch_to_parse_csv( has_header: bool, parse_options: CsvParseOptions, - buffer_source: CsvBufferSource, + buffer_source: R, projection_indices: Arc>, fields: Vec, read_daft_fields: Arc>>, @@ -1094,7 +1157,9 @@ fn dispatch_to_parse_csv( predicate: Option>, sender: Sender>, rows_read: Arc, -) { +) where + R: std::io::Read, +{ let table_results = { let rdr = ReaderBuilder::new() .has_headers(has_header) @@ -1120,7 +1185,7 @@ fn dispatch_to_parse_csv( Ok(tables) => { for table in tables { let table_len = table.len(); - sender.send(Ok(table)).unwrap(); + let _ = sender.send(Ok(table)); // Atomically update the number of rows read only after the result has // been sent. In theory we could wrap these steps in a mutex, but // applying limit at this layer can be best-effort with no adverse @@ -1128,7 +1193,9 @@ fn dispatch_to_parse_csv( rows_read.fetch_add(table_len, Ordering::Relaxed); } } - Err(e) => sender.send(Err(e)).unwrap(), + Err(e) => { + let _ = sender.send(Err(e)); + } } } From 423c2f036d2d9adeadc03947ed847c6e2ff2541f Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 18 Oct 2024 13:55:13 -0700 Subject: [PATCH 11/38] Remove unused deps --- Cargo.lock | 1 - src/daft-csv/Cargo.toml | 1 - src/daft-csv/src/local.rs | 68 --------------------------------------- 3 files changed, 70 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 573ac3fd7d..ade5517d93 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1771,7 +1771,6 @@ dependencies = [ "daft-io", "daft-table", "futures", - "itertools 0.11.0", "memchr", "pyo3", "rayon", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index bfb99cddac..ca74b9c2ec 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -13,7 +13,6 @@ daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} -itertools = {workspace = true} memchr = "2.7.2" pyo3 = {workspace = true, optional = true} rayon = {workspace = true} diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index dee825012d..414cc9c75e 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -687,15 +687,12 @@ impl ChunkStateHolder { /// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next /// \n character and go back to 2. fn goto_next_newline(&mut self) -> bool { - //println!("newline idx {}, newline offset {}", self.curr_newline_idx, self.curr_newline_offset); self.valid_chunk = false; loop { if self.curr_newline_idx >= self.states.len() { - //println!("no newline found"); return false; } if self.curr_newline_offset >= self.states[self.curr_newline_idx].len() { - //println!("offset larger than length of current state"); self.curr_newline_offset = 0; self.curr_newline_idx += 1; continue; @@ -703,7 +700,6 @@ impl ChunkStateHolder { if let Some(pos) = self.states[self.curr_newline_idx].find_newline(self.curr_newline_offset) { - //println!("found newline at {}, {}", self.curr_newline_idx, pos); self.curr_newline_offset = pos + 1; if self.curr_newline_offset >= self.states[self.curr_newline_idx].len() { self.curr_newline_offset = 0; @@ -714,7 +710,6 @@ impl ChunkStateHolder { self.curr_byte_read_offset = self.curr_newline_offset; return true; } else { - //println!("no newline found, going to next idx.."); self.curr_newline_offset = 0; self.curr_newline_idx += 1; continue; @@ -745,10 +740,8 @@ impl ChunkStateHolder { escape_char, double_quote_escape_allowed, ) { - // println!("validity: {valid}"); self.valid_chunk = valid; if valid { - // println!("Valid! states: {:?}", self.states); let (ending_idx, ending_offset) = if self.curr_byte_read_offset == 0 { ( self.curr_byte_read_idx - 1, @@ -770,12 +763,9 @@ impl ChunkStateHolder { } else { panic!("There should be at least one chunk state that's split off."); } - // println!("After transformation: {:?}", self.states); - // println!("leftovers:{:?}", self.next_states); } return valid; } else { - //println!("state machine needs more bytes??"); // We ran out of bytes while running the CSV state machine. Read another file slab then // continue running the state machine. if let Some(next) = iter.next() { @@ -830,40 +820,6 @@ impl Iterator for ChunkStateHolder { } } -// impl Read for ChunkStateHolder { -// fn read(&mut self, buf: &mut [u8]) -> std::io::Result { -// let current_state = loop { -// if self.curr_read_idx >= self.states.len() { -// return Ok(0); // EOF -// } -// let current_state = &self.states[self.curr_read_idx]; -// if self.curr_read_offset < current_state.len() { -// break current_state; -// } -// self.curr_read_offset = 0; -// self.curr_read_idx += 1; -// }; -// let slice = match current_state { -// ChunkState::Start { slab, start } => { -// &unsafe_clone_buffer(&slab.buffer)[*start..slab.valid_bytes] -// } -// ChunkState::StartAndFinal { slab, start, end } => { -// &unsafe_clone_buffer(&slab.buffer)[*start..*end] -// } -// ChunkState::Continue { slab } => &unsafe_clone_buffer(&slab.buffer)[..slab.valid_bytes], -// ChunkState::Final { slab, end } => &unsafe_clone_buffer(&slab.buffer)[..*end], -// }; -// let read_size = buf.len().min(slice.len() - self.curr_read_offset); -// buf[..read_size].copy_from_slice(&slice[self.curr_read_offset..][..read_size]); -// self.curr_read_offset += read_size; -// Ok(read_size) -// } - -// // fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} - -// // fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} -// } - #[derive(Debug)] enum ChunkState { Start { @@ -946,26 +902,6 @@ impl ChunkState { } } -// struct SlabConsumer<'a, I> { -// slab_iter: &'a mut I, -// } - -// impl<'a, I> Iterator for SlabConsumer<'a, I> -// where -// I: Iterator>, -// { -// type Item = ChunkState; -// fn next(&mut self) -> Option { -// // let mut so_far = vec![]; -// while let Some(data) = self.slab_iter.next() { -// todo!() -// } - -// // verify last item is final or StartAndFinal -// todo!("emit state") -// } -// } - struct MultiSliceReader { // https://stackoverflow.com/questions/71801199/how-can-concatenated-u8-slices-implement-the-read-trait-without-additional-co // use small vec @@ -1072,7 +1008,6 @@ fn validate_csv_record( return Some(false); } let byte = iter.next()?; - // println!("field start byte {}", byte); if byte == NEWLINE { *state = CsvState::RecordEnd; } else if byte == quote_char { @@ -1088,7 +1023,6 @@ fn validate_csv_record( // We follow the convention where an unquoted field does not consider escape characters. loop { let byte = iter.next()?; - // println!("unquoted field byte {}", byte); if byte == NEWLINE { *state = CsvState::RecordEnd; break; @@ -1102,7 +1036,6 @@ fn validate_csv_record( CsvState::QuotedField => { loop { let byte = iter.next()?; - // println!("quoted field byte {}", byte); if byte == quote_char { *state = CsvState::Unquote; break; @@ -1117,7 +1050,6 @@ fn validate_csv_record( } CsvState::Unquote => { let byte = iter.next()?; - // println!("unquote byte {}", byte); if let Some(escape_char) = escape_char && byte == escape_char && escape_char == quote_char From 11850aa517b63b037ade0c983e7353c8bf961109 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 18 Oct 2024 15:11:03 -0700 Subject: [PATCH 12/38] Rework state machine --- src/daft-csv/src/local.rs | 222 ++++++++++++++++++++------------------ 1 file changed, 118 insertions(+), 104 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 414cc9c75e..dafeea552d 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -485,6 +485,13 @@ fn consume_slab_iterator( let mut iter = iter.peekable(); let mut curr_chunk = ChunkStateHolder::empty(); + let mut csv_validator = CsvValidator::new( + num_fields, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + ); loop { // Check limit. let limit_reached = limit.map_or(false, |limit| { @@ -521,14 +528,7 @@ fn consume_slab_iterator( } curr_chunk.states.push(ChunkState::Continue { slab: next }); while curr_chunk.goto_next_newline() { - if curr_chunk.validate_csv_record( - &mut iter, - num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ) { + if curr_chunk.validate_csv_record(&mut csv_validator, &mut iter) { break; } } @@ -719,27 +719,13 @@ impl ChunkStateHolder { fn validate_csv_record( &mut self, + validator: &mut CsvValidator, iter: &mut impl Iterator>, - num_fields: usize, - quote_char: u8, - field_delimiter: u8, - escape_char: Option, - double_quote_escape_allowed: bool, ) -> bool { - let mut csv_state = CsvState::FieldStart; - let mut num_fields_seen: usize = 0; + validator.reset(); loop { // Run the CSV state machine to see if we're currently at a valid record boundary. - if let Some(valid) = validate_csv_record( - self.into_iter(), - &mut csv_state, - num_fields, - &mut num_fields_seen, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, - ) { + if let Some(valid) = validator.validate_record(self.into_iter()) { self.valid_chunk = valid; if valid { let (ending_idx, ending_offset) = if self.curr_byte_read_offset == 0 { @@ -977,99 +963,127 @@ fn newline_position(buffer: &[u8]) -> Option { memchr::memchr(NEWLINE, buffer) } +struct CsvValidator { + state: CsvState, + num_fields: usize, + num_fields_seen: usize, + quote_char: u8, + field_delimiter: u8, + escape_char: Option, + double_quote_escape_allowed: bool, + // The transition table only needs to consider 256 possible inputs, because the only characters + // that are valid for transitioning the table state are single-byte ASCII characters. Furthermore, + // even when reading UTF-8, upon encountering a byte that matches the value for an ASCII character, + // this byte will always correspond to the ASCII character. + // For more details, see: https://en.wikipedia.org/wiki/UTF-8#Encoding + transition_table: [[CsvState; 256]; 6], +} + /// Csv states used by the state machine in `validate_csv_record`. -#[derive(Clone)] +#[derive(Clone, Copy, Debug, PartialEq, Eq)] enum CsvState { FieldStart, RecordEnd, UnquotedField, QuotedField, Unquote, + Escape, + Invalid, } -/// State machine that validates whether the current buffer starts at a valid csv record. -/// See `get_file_chunk` for more details. -#[allow(clippy::too_many_arguments)] -fn validate_csv_record( - iter: &mut impl Iterator, - state: &mut CsvState, - num_fields: usize, - num_fields_seen: &mut usize, - quote_char: u8, - field_delimiter: u8, - escape_char: Option, - double_quote_escape_allowed: bool, -) -> Option { - loop { - match state { - CsvState::FieldStart => { - *num_fields_seen += 1; - if *num_fields_seen > num_fields { - return Some(false); - } - let byte = iter.next()?; - if byte == NEWLINE { - *state = CsvState::RecordEnd; - } else if byte == quote_char { - *state = CsvState::QuotedField; - } else if byte != field_delimiter { - *state = CsvState::UnquotedField; - } - } - CsvState::RecordEnd => { - return Some(*num_fields_seen == num_fields); - } - CsvState::UnquotedField => { - // We follow the convention where an unquoted field does not consider escape characters. - loop { - let byte = iter.next()?; - if byte == NEWLINE { - *state = CsvState::RecordEnd; - break; - } - if byte == field_delimiter { - *state = CsvState::FieldStart; - break; - } - } - } - CsvState::QuotedField => { - loop { - let byte = iter.next()?; - if byte == quote_char { - *state = CsvState::Unquote; - break; - } - if let Some(escape_char) = escape_char - && byte == escape_char - { - // Skip the next character. - iter.next()?; +impl CsvValidator { + fn new( + num_fields: usize, + quote_char: u8, + field_delimiter: u8, + escape_char: Option, + double_quote_escape_allowed: bool, + ) -> Self { + let mut validator = Self { + state: CsvState::FieldStart, + num_fields, + num_fields_seen: 0, + quote_char, + field_delimiter, + escape_char, + double_quote_escape_allowed, + transition_table: [[CsvState::Invalid; 256]; 6], + }; + validator.build_transition_table(); + validator + } + + fn build_transition_table(&mut self) { + // FieldStart transitions. + self.transition_table[CsvState::FieldStart as usize] = [CsvState::UnquotedField; 256]; + self.transition_table[CsvState::FieldStart as usize][NEWLINE as usize] = + CsvState::RecordEnd; + self.transition_table[CsvState::FieldStart as usize][self.quote_char as usize] = + CsvState::QuotedField; + self.transition_table[CsvState::FieldStart as usize][self.field_delimiter as usize] = + CsvState::FieldStart; + + // UnquotedField transitions. + self.transition_table[CsvState::UnquotedField as usize] = [CsvState::UnquotedField; 256]; + self.transition_table[CsvState::UnquotedField as usize][NEWLINE as usize] = + CsvState::RecordEnd; + self.transition_table[CsvState::UnquotedField as usize][self.field_delimiter as usize] = + CsvState::FieldStart; + + // QuotedField transitions. + self.transition_table[CsvState::QuotedField as usize] = [CsvState::QuotedField; 256]; + if let Some(escape_char) = self.escape_char { + self.transition_table[CsvState::QuotedField as usize][escape_char as usize] = + CsvState::Escape; + } + // The quote char transition must be defined after the escape transition, because the most common + // escape char in CSV is the quote char itself ("._.) + self.transition_table[CsvState::QuotedField as usize][self.quote_char as usize] = + CsvState::Unquote; + + // Unquote transitions. + self.transition_table[CsvState::Unquote as usize][NEWLINE as usize] = CsvState::RecordEnd; + self.transition_table[CsvState::Unquote as usize][self.field_delimiter as usize] = + CsvState::FieldStart; + if let Some(escape_char) = self.escape_char + && escape_char == self.quote_char + && (self.quote_char != DOUBLE_QUOTE || self.double_quote_escape_allowed) + { + self.transition_table[CsvState::Unquote as usize][self.quote_char as usize] = + CsvState::QuotedField; + } + + // Escape transitions. + self.transition_table[CsvState::Escape as usize] = [CsvState::QuotedField; 256]; + } + + fn reset(&mut self) { + self.num_fields_seen = 1; + self.state = CsvState::FieldStart; + } + + fn validate_record(&mut self, iter: &mut impl Iterator) -> Option { + for byte in iter { + let next_state = self.transition_table[self.state as usize][byte as usize]; + + match next_state { + CsvState::FieldStart => { + self.num_fields_seen += 1; + if self.num_fields_seen > self.num_fields { + return Some(false); } } - } - CsvState::Unquote => { - let byte = iter.next()?; - if let Some(escape_char) = escape_char - && byte == escape_char - && escape_char == quote_char - && (byte != DOUBLE_QUOTE || double_quote_escape_allowed) - { - *state = CsvState::QuotedField; - continue; - } - if byte == NEWLINE { - *state = CsvState::RecordEnd; - continue; + CsvState::RecordEnd => { + return Some(self.num_fields_seen == self.num_fields); } - if byte == field_delimiter { - *state = CsvState::FieldStart; - continue; - } - // Other characters are not allowed after a quote. This is invalid CSV. - return Some(false); + CsvState::Invalid => return Some(false), + _ => {} } + + self.state = next_state; } + + None } } From 516596abdf939150d376767749e8b7260379f3d2 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 18 Oct 2024 15:54:22 -0700 Subject: [PATCH 13/38] Remove option wrapping around file slab buffers --- src/daft-csv/src/local.rs | 62 +++++++++++++-------------------------- 1 file changed, 20 insertions(+), 42 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index dafeea552d..855b1c46b9 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -1,4 +1,4 @@ -use core::str; +use core::{mem::ManuallyDrop, str}; use std::{ io::Read, num::NonZeroUsize, @@ -143,9 +143,7 @@ struct FileSlabPool { #[derive(Clone, Debug)] struct FileSlab { pool: Arc, - // We wrap the Arc in an Option so that when a Slab is being dropped, we can move the Slab's reference - // to the Arc back to the slab pool. - buffer: Option>, + buffer: ManuallyDrop>, valid_bytes: usize, } @@ -155,16 +153,15 @@ impl FileSlab { // // This assumption is only true if the user does not append to the current file while we are reading it. pub fn filled_buffer(&self) -> bool { - self.valid_bytes >= unsafe_clone_buffer(&self.buffer).len() + self.valid_bytes >= self.buffer.len() } } // Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. impl Drop for FileSlab { fn drop(&mut self) { - if let Some(buffer) = self.buffer.take() { - self.pool.return_buffer(buffer); - } + let buffer = unsafe { ManuallyDrop::take(&mut self.buffer) }; + self.pool.return_buffer(buffer); } } @@ -449,7 +446,7 @@ impl Iterator for SlabIterator { self.total_bytes_read += bytes_read; Some(Arc::new(FileSlab { pool: Arc::clone(&self.slabpool), - buffer: Some(buffer), + buffer: ManuallyDrop::new(buffer), valid_bytes: bytes_read, })) } @@ -845,30 +842,24 @@ impl ChunkState { #[inline] fn get_idx(&self, idx: usize) -> u8 { match self { - Self::Start { slab, start } => unsafe_clone_buffer(&slab.buffer)[start + idx], - Self::StartAndFinal { slab, start, .. } => { - unsafe_clone_buffer(&slab.buffer)[start + idx] - } - Self::Continue { slab } => unsafe_clone_buffer(&slab.buffer)[idx], - Self::Final { slab, .. } => unsafe_clone_buffer(&slab.buffer)[idx], + Self::Start { slab, start } => slab.buffer[start + idx], + Self::StartAndFinal { slab, start, .. } => slab.buffer[start + idx], + Self::Continue { slab } => slab.buffer[idx], + Self::Final { slab, .. } => slab.buffer[idx], } } #[inline] fn find_newline(&self, offset: usize) -> Option { match self { - Self::Start { slab, start } => newline_position( - &unsafe_clone_buffer(&slab.buffer)[*start + offset..slab.valid_bytes], - ), - Self::StartAndFinal { slab, start, end } => { - newline_position(&unsafe_clone_buffer(&slab.buffer)[*start + offset..*end]) + Self::Start { slab, start } => { + newline_position(&slab.buffer[*start + offset..slab.valid_bytes]) } - Self::Continue { slab } => { - newline_position(&unsafe_clone_buffer(&slab.buffer)[offset..slab.valid_bytes]) - } - Self::Final { slab, end } => { - newline_position(&unsafe_clone_buffer(&slab.buffer)[offset..*end]) + Self::StartAndFinal { slab, start, end } => { + newline_position(&slab.buffer[*start + offset..*end]) } + Self::Continue { slab } => newline_position(&slab.buffer[offset..slab.valid_bytes]), + Self::Final { slab, end } => newline_position(&slab.buffer[offset..*end]), } } @@ -920,14 +911,10 @@ impl Read for MultiSliceReader { self.curr_read_idx += 1; }; let slice = match current_state { - ChunkState::Start { slab, start } => { - &unsafe_clone_buffer(&slab.buffer)[*start..slab.valid_bytes] - } - ChunkState::StartAndFinal { slab, start, end } => { - &unsafe_clone_buffer(&slab.buffer)[*start..*end] - } - ChunkState::Continue { slab } => &unsafe_clone_buffer(&slab.buffer)[..slab.valid_bytes], - ChunkState::Final { slab, end } => &unsafe_clone_buffer(&slab.buffer)[..*end], + ChunkState::Start { slab, start } => &slab.buffer[*start..slab.valid_bytes], + ChunkState::StartAndFinal { slab, start, end } => &slab.buffer[*start..*end], + ChunkState::Continue { slab } => &slab.buffer[..slab.valid_bytes], + ChunkState::Final { slab, end } => &slab.buffer[..*end], }; let read_size = buf.len().min(slice.len() - self.curr_read_offset); buf[..read_size].copy_from_slice(&slice[self.curr_read_offset..][..read_size]); @@ -940,15 +927,6 @@ impl Read for MultiSliceReader { // fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} } -/// Unsafe helper function that extracts the buffer from an &Option>. Users should -/// ensure that the buffer is Some, otherwise this function causes the process to panic. -fn unsafe_clone_buffer(buffer: &Option>) -> Arc<[u8]> { - match buffer { - Some(buffer) => Arc::clone(buffer), - None => panic!("Tried to clone a CSV slab that doesn't exist. Please report this error."), - } -} - // Daft does not currently support non-\n record terminators (e.g. carriage return \r, which only // matters for pre-Mac OS X). const NEWLINE: u8 = b'\n'; From c44871d4f2fe4f1a33b99efb2957ac4502752a16 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 18 Oct 2024 16:24:27 -0700 Subject: [PATCH 14/38] Remove Arcs --- src/daft-csv/src/local.rs | 39 +++++++++++++++------------------------ 1 file changed, 15 insertions(+), 24 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 855b1c46b9..226245c05b 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -135,7 +135,7 @@ const SLABPOOL_DEFAULT_SIZE: usize = 20; /// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] struct FileSlabPool { - buffers: Mutex>>, + buffers: Mutex>>, condvar: Condvar, } @@ -143,7 +143,7 @@ struct FileSlabPool { #[derive(Clone, Debug)] struct FileSlab { pool: Arc, - buffer: ManuallyDrop>, + buffer: ManuallyDrop>, valid_bytes: usize, } @@ -167,11 +167,10 @@ impl Drop for FileSlab { impl FileSlabPool { pub fn new() -> Self { - let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) // We get uninitialized buffers because we will always populate the buffers with a file read before use. .map(|_| Box::new_uninit_slice(SLABSIZE)) .map(|x| unsafe { x.assume_init() }) - .map(Arc::from) .collect(); Self { buffers: Mutex::new(chunk_buffers), @@ -179,7 +178,7 @@ impl FileSlabPool { } } - pub fn get_buffer(self: &Arc) -> Arc<[u8]> { + pub fn get_buffer(self: &Arc) -> Box<[u8]> { let mut buffers = self.buffers.lock().unwrap(); while buffers.is_empty() { // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. @@ -189,7 +188,7 @@ impl FileSlabPool { buffers.pop().unwrap() } - fn return_buffer(&self, buffer: Arc<[u8]>) { + fn return_buffer(&self, buffer: Box<[u8]>) { let mut buffers = self.buffers.lock().unwrap(); buffers.push(buffer); self.condvar.notify_one(); @@ -436,25 +435,17 @@ impl Iterator for SlabIterator { type Item = Arc; fn next(&mut self) -> Option { let mut buffer = self.slabpool.get_buffer(); - match Arc::get_mut(&mut buffer) { - Some(inner_buffer) => { - let bytes_read = self.file.read(inner_buffer).unwrap(); - if bytes_read == 0 { - self.slabpool.return_buffer(buffer); - return None; - } - self.total_bytes_read += bytes_read; - Some(Arc::new(FileSlab { - pool: Arc::clone(&self.slabpool), - buffer: ManuallyDrop::new(buffer), - valid_bytes: bytes_read, - })) - } - None => { - self.slabpool.return_buffer(buffer); - panic!("We should have exclusive access to this mutable buffer."); - } + let bytes_read = self.file.read(buffer.as_mut()).unwrap(); + if bytes_read == 0 { + self.slabpool.return_buffer(buffer); + return None; } + self.total_bytes_read += bytes_read; + Some(Arc::new(FileSlab { + pool: Arc::clone(&self.slabpool), + buffer: ManuallyDrop::new(buffer), + valid_bytes: bytes_read, + })) } } From 68d520b100b36642bf7f00933fe5d1c625c36d91 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Fri, 18 Oct 2024 16:23:48 -0700 Subject: [PATCH 15/38] clean up multislice reader --- src/daft-csv/src/local.rs | 60 +++++++++++++++++++++------------------ 1 file changed, 33 insertions(+), 27 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 226245c05b..f8578f2b6f 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -567,7 +567,7 @@ fn consume_slab_iterator( } else { return; } - let multi_slice_reader = MultiSliceReader::new(states_to_read); + let multi_slice_reader = MultiSliceReader::new(states_to_read.as_slice()); dispatch_to_parse_csv( has_header, parse_options, @@ -870,47 +870,53 @@ impl ChunkState { } } -struct MultiSliceReader { +struct MultiSliceReader<'a> { // https://stackoverflow.com/questions/71801199/how-can-concatenated-u8-slices-implement-the-read-trait-without-additional-co // use small vec - states: Vec, + slices: Vec<&'a [u8]>, curr_read_idx: usize, curr_read_offset: usize, } -impl MultiSliceReader { - fn new(states: Vec) -> Self { +impl<'a> MultiSliceReader<'a> { + fn new(states: &'a [ChunkState]) -> Self { + let mut slices = Vec::with_capacity(states.len()); + for state in states { + let slice = match state { + ChunkState::Start { slab, start } => &slab.buffer[*start..slab.valid_bytes], + ChunkState::StartAndFinal { slab, start, end } => &slab.buffer[*start..*end], + ChunkState::Continue { slab } => &slab.buffer[..slab.valid_bytes], + ChunkState::Final { slab, end } => &slab.buffer[..*end], + }; + slices.push(slice); + } + Self { - states, + slices, curr_read_idx: 0, curr_read_offset: 0, } } } -impl Read for MultiSliceReader { +impl<'a> Read for MultiSliceReader<'a> { fn read(&mut self, buf: &mut [u8]) -> std::io::Result { - let current_state = loop { - if self.curr_read_idx >= self.states.len() { - return Ok(0); // EOF - } - let current_state = &self.states[self.curr_read_idx]; - if self.curr_read_offset < current_state.len() { - break current_state; + let buf_len = buf.len(); + let mut position = 0; + while self.curr_read_idx < self.slices.len() && position < buf_len { + let slice = self.slices[self.curr_read_idx]; + if self.curr_read_offset < slice.len() { + let read_size = (buf_len - position).min(slice.len() - self.curr_read_offset); + buf[position..position + read_size] + .copy_from_slice(&slice[self.curr_read_offset..][..read_size]); + self.curr_read_offset += read_size; + position += read_size; + } else { + self.curr_read_offset = 0; + self.curr_read_idx += 1; } - self.curr_read_offset = 0; - self.curr_read_idx += 1; - }; - let slice = match current_state { - ChunkState::Start { slab, start } => &slab.buffer[*start..slab.valid_bytes], - ChunkState::StartAndFinal { slab, start, end } => &slab.buffer[*start..*end], - ChunkState::Continue { slab } => &slab.buffer[..slab.valid_bytes], - ChunkState::Final { slab, end } => &slab.buffer[..*end], - }; - let read_size = buf.len().min(slice.len() - self.curr_read_offset); - buf[..read_size].copy_from_slice(&slice[self.curr_read_offset..][..read_size]); - self.curr_read_offset += read_size; - Ok(read_size) + } + Ok(position) } // fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} From f617d45ca1b907717b24ac8a05427d1614c1e69d Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Fri, 18 Oct 2024 16:48:56 -0700 Subject: [PATCH 16/38] rwlock example --- Cargo.lock | 1 + src/daft-csv/Cargo.toml | 2 +- src/daft-csv/src/local.rs | 53 ++++++++++++++++++++++++++++++++++++++- 3 files changed, 54 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ade5517d93..514becbb94 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1772,6 +1772,7 @@ dependencies = [ "daft-table", "futures", "memchr", + "parking_lot", "pyo3", "rayon", "rstest", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index ca74b9c2ec..00b8899856 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -20,7 +20,7 @@ serde = {workspace = true} snafu = {workspace = true} tokio = {workspace = true} tokio-util = {workspace = true} - +parking_lot = "*" [dev-dependencies] rstest = {workspace = true} diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index f8578f2b6f..53093c0561 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -4,7 +4,7 @@ use std::{ num::NonZeroUsize, sync::{ atomic::{AtomicUsize, Ordering}, - Arc, Condvar, Mutex, + Arc, Condvar, Mutex, RwLock, }, }; @@ -139,6 +139,57 @@ struct FileSlabPool { condvar: Condvar, } +struct FileSlabState { + + +} + + +struct FileSlab2 { + state: RwLock, + pool: Arc +} + +#[derive(Debug)] +struct FileSlabPool2 { + buffers: Mutex>>>, + condvar: Condvar, +} + + +impl FileSlabPool2 { + pub fn get_slab(self: &Arc) -> Arc> { + let mut buffers = self.buffers.lock().unwrap(); + while buffers.is_empty() { + // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. + // This potentially allows us to rate limit the CSV reader until downstream consumers are ready for data. + buffers = self.condvar.wait(buffers).unwrap(); + } + buffers.pop().unwrap() + } +} + + +fn does_stuff(fs: &Arc) { + let slab = fs.get_slab(); + { + let mut writer = slab.write().unwrap(); + writer.valid_bytes = 1; + } + rayon::spawn(move || { + let reader = slab.read().unwrap(); + + }); + + + + + +} + + + + /// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. #[derive(Clone, Debug)] struct FileSlab { From 8e86310c469e6fccc7a9913a463073c7c742a660 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Fri, 18 Oct 2024 18:54:15 -0700 Subject: [PATCH 17/38] Rwlocking --- src/daft-csv/src/local.rs | 261 ++++++++++++++++++-------------------- 1 file changed, 123 insertions(+), 138 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 53093c0561..ae9543d604 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -1,4 +1,4 @@ -use core::{mem::ManuallyDrop, str}; +use core::str; use std::{ io::Read, num::NonZeroUsize, @@ -10,11 +10,10 @@ use std::{ use arrow2::{ datatypes::Field, - io::csv::{ - read, - read::{Reader, ReaderBuilder}, + io::{csv::{ + read::{self, Reader, ReaderBuilder}, read_async::local_read_rows, - }, + }}, }; use common_error::{DaftError, DaftResult}; use crossbeam_channel::Sender; @@ -135,101 +134,94 @@ const SLABPOOL_DEFAULT_SIZE: usize = 20; /// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] struct FileSlabPool { - buffers: Mutex>>, + buffers: Mutex>>, condvar: Condvar, } +#[derive(Debug)] struct FileSlabState { - - -} - - -struct FileSlab2 { - state: RwLock, - pool: Arc + buffer: Box<[u8]>, + valid_bytes: usize, } +/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] -struct FileSlabPool2 { - buffers: Mutex>>>, - condvar: Condvar, +struct FileSlab { + state: RwLock, + pool: Arc } - -impl FileSlabPool2 { - pub fn get_slab(self: &Arc) -> Arc> { - let mut buffers = self.buffers.lock().unwrap(); - while buffers.is_empty() { - // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. - // This potentially allows us to rate limit the CSV reader until downstream consumers are ready for data. - buffers = self.condvar.wait(buffers).unwrap(); - } - buffers.pop().unwrap() +impl FileSlab { + // Check whether this FileSlabState filled up its internal buffer. We use this as a means of checking whether this + // is the last FileSlab in the current file. + // + // This assumption is only true if the user does not append to the current file while we are reading it. + fn filled_buffer(&self) -> bool { + let reader = self.state.read().unwrap(); + reader.valid_bytes >= reader.buffer.len() } -} - -fn does_stuff(fs: &Arc) { - let slab = fs.get_slab(); - { - let mut writer = slab.write().unwrap(); - writer.valid_bytes = 1; + fn get_valid_bytes(&self) -> usize { + let reader = self.state.read().unwrap(); + reader.valid_bytes } - rayon::spawn(move || { - let reader = slab.read().unwrap(); - - }); - - - - - -} - - + fn get_byte(&self, idx: usize) -> u8 { + let reader = self.state.read().unwrap(); + reader.buffer[idx] + } -/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. -#[derive(Clone, Debug)] -struct FileSlab { - pool: Arc, - buffer: ManuallyDrop>, - valid_bytes: usize, -} + fn find_newline(&self, offset: usize) -> Option { + let reader = self.state.read().unwrap(); + newline_position(&reader.buffer[offset..reader.valid_bytes]) + } -impl FileSlab { - // Check whether this FileSlab filled up its internal buffer. We use this as a means of checking whether this - // is the last FileSlab in the current file. - // - // This assumption is only true if the user does not append to the current file while we are reading it. - pub fn filled_buffer(&self) -> bool { - self.valid_bytes >= self.buffer.len() + fn get_slice(&self) -> &[u8] { + let reader = self.state.read().unwrap(); + &reader.buffer[..] } } // Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. impl Drop for FileSlab { fn drop(&mut self) { - let buffer = unsafe { ManuallyDrop::take(&mut self.buffer) }; - self.pool.return_buffer(buffer); + // We need to reconstruct an Arc here. + // This is safe because we know we're in the Drop impl, + // so there are no other strong references to this FileSlab. + let slab = unsafe { Arc::from_raw(self as *const FileSlab) }; + + // Prevent the Drop from being called again when this Arc is dropped + std::mem::forget(slab.clone()); + + // Return the slab to the pool + self.pool.return_slab(slab); } } impl FileSlabPool { - pub fn new() -> Self { - let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + fn new() -> Arc { + let pool = Arc::new(Self { buffers: Mutex::new(vec![]), condvar: Condvar::new() }); + { + let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) // We get uninitialized buffers because we will always populate the buffers with a file read before use. .map(|_| Box::new_uninit_slice(SLABSIZE)) .map(|x| unsafe { x.assume_init() }) + .map(|buffer| + Arc::new( + FileSlab{ + state: RwLock::new(FileSlabState {buffer, valid_bytes: 0}), + pool: Arc::clone(&pool), + } + ) + ) .collect(); - Self { - buffers: Mutex::new(chunk_buffers), - condvar: Condvar::new(), + let mut buffers = pool.buffers.lock().unwrap(); + *buffers = chunk_buffers; } + pool } - pub fn get_buffer(self: &Arc) -> Box<[u8]> { + fn get_slab(self: &Arc) -> Arc { let mut buffers = self.buffers.lock().unwrap(); while buffers.is_empty() { // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. @@ -239,9 +231,9 @@ impl FileSlabPool { buffers.pop().unwrap() } - fn return_buffer(&self, buffer: Box<[u8]>) { - let mut buffers = self.buffers.lock().unwrap(); - buffers.push(buffer); + fn return_slab(&self, slab: Arc) { + let mut slabs = self.buffers.lock().unwrap(); + slabs.push(slab); self.condvar.notify_one(); } } @@ -353,9 +345,6 @@ pub async fn stream_csv_local( } .unwrap_or_default(); - // Create slab pool for file reads. - let slabpool = Arc::new(FileSlabPool::new()); - // Get schema and row estimations. let (schema, estimated_mean_row_size, estimated_std_row_size) = get_schema_and_estimators(uri, &convert_options, &parse_options, io_client, io_stats) @@ -402,10 +391,8 @@ pub async fn stream_csv_local( // Consume the CSV file asynchronously. rayon::spawn(move || { - let slab_iterator = SlabIterator::new(file, slabpool); - let rows_read = Arc::new(AtomicUsize::new(0)); consume_slab_iterator( - slab_iterator, + file, buffer_pool, num_fields, parse_options, @@ -417,7 +404,6 @@ pub async fn stream_csv_local( predicate, limit, sender, - rows_read, ); }); @@ -485,24 +471,24 @@ impl SlabIterator { impl Iterator for SlabIterator { type Item = Arc; fn next(&mut self) -> Option { - let mut buffer = self.slabpool.get_buffer(); - let bytes_read = self.file.read(buffer.as_mut()).unwrap(); - if bytes_read == 0 { - self.slabpool.return_buffer(buffer); - return None; + let slab = self.slabpool.get_slab(); + { + let mut writer = slab.state.write().unwrap(); + let bytes_read = self.file.read(&mut writer.buffer).unwrap(); + if bytes_read == 0 { + return None; + } + self.total_bytes_read += bytes_read; + writer.valid_bytes = bytes_read; } - self.total_bytes_read += bytes_read; - Some(Arc::new(FileSlab { - pool: Arc::clone(&self.slabpool), - buffer: ManuallyDrop::new(buffer), - valid_bytes: bytes_read, - })) + + Some(slab) } } #[allow(clippy::too_many_arguments)] fn consume_slab_iterator( - iter: impl Iterator>, + file: std::fs::File, buffer_pool: Arc, num_fields: usize, parse_options: CsvParseOptions, @@ -514,15 +500,17 @@ fn consume_slab_iterator( predicate: Option>, limit: Option, sender: Sender>, - rows_read: Arc, ) { + // Create slab pool for file reads. + let slabpool = FileSlabPool::new(); + let rows_read = Arc::new(AtomicUsize::new(0)); + let mut slab_iterator = SlabIterator::new(file, slabpool); let mut has_header = parse_options.has_header; let field_delimiter = parse_options.delimiter; let escape_char = parse_options.escape_char; let quote_char = parse_options.quote; let double_quote_escape_allowed = parse_options.double_quote; - let mut iter = iter.peekable(); let mut curr_chunk = ChunkStateHolder::empty(); let mut csv_validator = CsvValidator::new( num_fields, @@ -542,7 +530,7 @@ fn consume_slab_iterator( } // Grab a starting file slab if the current CSV chunk is empty. if curr_chunk.is_empty() { - if let Some(next) = iter.next() { + if let Some(next) = slab_iterator.next() { curr_chunk.states.push(ChunkState::Start { slab: next, start: 0, @@ -556,18 +544,18 @@ fn consume_slab_iterator( } // Grab file slabs until we find a valid CSV chunk. loop { - if let Some(next) = iter.next() { + if let Some(next) = slab_iterator.next() { // If the next buffer is not completely filled, we take this to mean that we've reached EOF. if !next.filled_buffer() { curr_chunk.states.push(ChunkState::Final { - end: next.valid_bytes, + end: next.get_valid_bytes(), slab: next, }); break; } curr_chunk.states.push(ChunkState::Continue { slab: next }); while curr_chunk.goto_next_newline() { - if curr_chunk.validate_csv_record(&mut csv_validator, &mut iter) { + if curr_chunk.validate_csv_record(&mut csv_validator, &mut slab_iterator) { break; } } @@ -580,14 +568,14 @@ fn consume_slab_iterator( match last_state { ChunkState::Start { slab, start } => { curr_chunk.states.push(ChunkState::StartAndFinal { - end: slab.valid_bytes, + end: slab.get_valid_bytes(), slab, start, }); } ChunkState::Continue { slab } => { curr_chunk.states.push(ChunkState::Final { - end: slab.valid_bytes, + end: slab.get_valid_bytes(), slab, }); } @@ -797,7 +785,7 @@ impl ChunkStateHolder { if !next.filled_buffer() { // EOF. Make this chunk state holder valid and exit. self.states.push(ChunkState::Final { - end: next.valid_bytes, + end: next.get_valid_bytes(), slab: next, }); self.valid_chunk = true; @@ -825,7 +813,7 @@ impl ChunkStateHolder { } } -impl Iterator for ChunkStateHolder { +impl<'a> Iterator for ChunkStateHolder { type Item = u8; fn next(&mut self) -> Option { @@ -874,9 +862,9 @@ impl ChunkState { #[inline] fn len(&self) -> usize { match self { - Self::Start { slab, start } => slab.valid_bytes - start, + Self::Start { slab, start } => slab.get_valid_bytes() - start, Self::StartAndFinal { start, end, .. } => end - start, - Self::Continue { slab } => slab.valid_bytes, + Self::Continue { slab } => slab.get_valid_bytes(), Self::Final { end, .. } => *end, } } @@ -884,47 +872,35 @@ impl ChunkState { #[inline] fn get_idx(&self, idx: usize) -> u8 { match self { - Self::Start { slab, start } => slab.buffer[start + idx], - Self::StartAndFinal { slab, start, .. } => slab.buffer[start + idx], - Self::Continue { slab } => slab.buffer[idx], - Self::Final { slab, .. } => slab.buffer[idx], + Self::Start { slab, start } => slab.get_byte(start + idx), + Self::StartAndFinal { slab, start, .. } => slab.get_byte(start + idx), + Self::Continue { slab } => slab.get_byte(idx), + Self::Final { slab, .. } => slab.get_byte(idx), } } #[inline] fn find_newline(&self, offset: usize) -> Option { match self { - Self::Start { slab, start } => { - newline_position(&slab.buffer[*start + offset..slab.valid_bytes]) - } - Self::StartAndFinal { slab, start, end } => { - newline_position(&slab.buffer[*start + offset..*end]) - } - Self::Continue { slab } => newline_position(&slab.buffer[offset..slab.valid_bytes]), - Self::Final { slab, end } => newline_position(&slab.buffer[offset..*end]), + Self::Continue { slab } =>slab.find_newline(offset), + // This function is not needed for non-Continue chunk states. + _ => None, } } #[inline] - fn get_slab(&self) -> &Arc { + fn clone_slab(&self) -> Arc { match self { - Self::Start { slab, .. } => slab, - Self::StartAndFinal { slab, .. } => slab, - Self::Continue { slab } => slab, - Self::Final { slab, .. } => slab, + Self::Start { slab, .. } => slab.clone(), + Self::StartAndFinal { slab, .. } => slab.clone(), + Self::Continue { slab } => slab.clone(), + Self::Final { slab, .. } => slab.clone(), } } - - #[inline] - fn clone_slab(&self) -> Arc { - self.get_slab().clone() - } } struct MultiSliceReader<'a> { - // https://stackoverflow.com/questions/71801199/how-can-concatenated-u8-slices-implement-the-read-trait-without-additional-co - // use small vec - slices: Vec<&'a [u8]>, + slices: Vec<(usize, usize, std::sync::RwLockReadGuard<'a, FileSlabState>)>, curr_read_idx: usize, curr_read_offset: usize, } @@ -933,13 +909,25 @@ impl<'a> MultiSliceReader<'a> { fn new(states: &'a [ChunkState]) -> Self { let mut slices = Vec::with_capacity(states.len()); for state in states { - let slice = match state { - ChunkState::Start { slab, start } => &slab.buffer[*start..slab.valid_bytes], - ChunkState::StartAndFinal { slab, start, end } => &slab.buffer[*start..*end], - ChunkState::Continue { slab } => &slab.buffer[..slab.valid_bytes], - ChunkState::Final { slab, end } => &slab.buffer[..*end], + let (start, end, guard) = match state { + ChunkState::Start { slab, start } => { + let guard = slab.state.read().unwrap(); + (*start, guard.valid_bytes, guard) + }, + ChunkState::StartAndFinal { slab, start, end } => { + let guard = slab.state.read().unwrap(); + (*start, *end, guard) + }, + ChunkState::Continue { slab } => { + let guard = slab.state.read().unwrap(); + (0, guard.valid_bytes, guard) + }, + ChunkState::Final { slab, end } => { + let guard = slab.state.read().unwrap(); + (0, *end, guard) + }, }; - slices.push(slice); + slices.push((start, end, guard)); } Self { @@ -955,7 +943,8 @@ impl<'a> Read for MultiSliceReader<'a> { let buf_len = buf.len(); let mut position = 0; while self.curr_read_idx < self.slices.len() && position < buf_len { - let slice = self.slices[self.curr_read_idx]; + let (start, end, reader) = &self.slices[self.curr_read_idx]; + let slice = &reader.buffer[*start..*end]; if self.curr_read_offset < slice.len() { let read_size = (buf_len - position).min(slice.len() - self.curr_read_offset); buf[position..position + read_size] @@ -969,10 +958,6 @@ impl<'a> Read for MultiSliceReader<'a> { } Ok(position) } - - // fn read_vectored(&mut self, bufs: &mut [std::io::IoSliceMut<'_>]) -> std::io::Result {} - - // fn read_to_end(&mut self, buf: &mut Vec) -> std::io::Result {} } // Daft does not currently support non-\n record terminators (e.g. carriage return \r, which only From 0df4f0440f327c27f4bd3807a54bb5ebbc848fd1 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Mon, 21 Oct 2024 20:24:04 -0700 Subject: [PATCH 18/38] Fix poisoning --- src/daft-csv/src/local.rs | 431 +++++++++++++++++++------------------- 1 file changed, 213 insertions(+), 218 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index ae9543d604..793671dbd5 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -4,16 +4,16 @@ use std::{ num::NonZeroUsize, sync::{ atomic::{AtomicUsize, Ordering}, - Arc, Condvar, Mutex, RwLock, + Arc, Mutex, RwLock, }, }; use arrow2::{ datatypes::Field, - io::{csv::{ + io::csv::{ read::{self, Reader, ReaderBuilder}, read_async::local_read_rows, - }}, + }, }; use common_error::{DaftError, DaftResult}; use crossbeam_channel::Sender; @@ -104,14 +104,18 @@ impl CsvBufferPool { } pub fn get_buffer(self: &Arc) -> CsvBuffer { - let mut buffers = self.buffers.lock().unwrap(); - let buffer = buffers.pop(); - let buffer = match buffer { - Some(buffer) => buffer, - None => vec![ - read::ByteRecord::with_capacity(self.record_buffer_size, self.num_fields); - self.buffer_size - ], + let buffer = { + let mut buffers = self.buffers.lock().unwrap(); + let buffer = buffers.pop(); + match buffer { + Some(buffer) => buffer, + None => { + vec![ + read::ByteRecord::with_capacity(self.record_buffer_size, self.num_fields); + self.buffer_size + ] + } + } }; CsvBuffer { @@ -121,8 +125,17 @@ impl CsvBufferPool { } fn return_buffer(&self, buffer: Vec) { - let mut buffers = self.buffers.lock().unwrap(); - buffers.push(buffer); + if let Ok(mut buffers) = self.buffers.lock() { + buffers.push(buffer); + } + } +} + +impl Drop for CsvBuffer { + fn drop(&mut self) { + // Take ownership of the buffer using std::mem::take + let buffer = std::mem::take(&mut self.buffer); + self.pool.return_buffer(buffer); } } @@ -135,7 +148,6 @@ const SLABPOOL_DEFAULT_SIZE: usize = 20; #[derive(Debug)] struct FileSlabPool { buffers: Mutex>>, - condvar: Condvar, } #[derive(Debug)] @@ -144,77 +156,67 @@ struct FileSlabState { valid_bytes: usize, } +impl FileSlabState { + fn find_newline(&self, offset: usize) -> Option { + newline_position(&self.buffer[offset..self.valid_bytes]) + } + + fn validate_record(&self, validator: &mut CsvValidator, start: usize) -> Option { + validator.validate_record(&mut self.buffer[start..self.valid_bytes].iter()) + } +} + /// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] struct FileSlab { state: RwLock, - pool: Arc + pool: Arc, } impl FileSlab { - // Check whether this FileSlabState filled up its internal buffer. We use this as a means of checking whether this - // is the last FileSlab in the current file. - // - // This assumption is only true if the user does not append to the current file while we are reading it. - fn filled_buffer(&self) -> bool { - let reader = self.state.read().unwrap(); - reader.valid_bytes >= reader.buffer.len() - } - - fn get_valid_bytes(&self) -> usize { - let reader = self.state.read().unwrap(); - reader.valid_bytes - } - - fn get_byte(&self, idx: usize) -> u8 { - let reader = self.state.read().unwrap(); - reader.buffer[idx] - } - fn find_newline(&self, offset: usize) -> Option { - let reader = self.state.read().unwrap(); - newline_position(&reader.buffer[offset..reader.valid_bytes]) - } - - fn get_slice(&self) -> &[u8] { - let reader = self.state.read().unwrap(); - &reader.buffer[..] + let guard = self.state.read().unwrap(); + guard.find_newline(offset) } } // Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. impl Drop for FileSlab { fn drop(&mut self) { - // We need to reconstruct an Arc here. - // This is safe because we know we're in the Drop impl, - // so there are no other strong references to this FileSlab. - let slab = unsafe { Arc::from_raw(self as *const FileSlab) }; - - // Prevent the Drop from being called again when this Arc is dropped - std::mem::forget(slab.clone()); - - // Return the slab to the pool - self.pool.return_slab(slab); + let pool = Arc::clone(&self.pool); + + let new_slab = Arc::new(Self { + state: RwLock::new(FileSlabState { + buffer: std::mem::take(&mut self.state.get_mut().unwrap().buffer), + valid_bytes: self.state.get_mut().unwrap().valid_bytes, + }), + pool, + }); + + self.pool.return_slab(new_slab); } } impl FileSlabPool { fn new() -> Arc { - let pool = Arc::new(Self { buffers: Mutex::new(vec![]), condvar: Condvar::new() }); + let pool = Arc::new(Self { + buffers: Mutex::new(vec![]), + }); { let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) - // We get uninitialized buffers because we will always populate the buffers with a file read before use. - .map(|_| Box::new_uninit_slice(SLABSIZE)) - .map(|x| unsafe { x.assume_init() }) - .map(|buffer| - Arc::new( - FileSlab{ - state: RwLock::new(FileSlabState {buffer, valid_bytes: 0}), + // We get uninitialized buffers because we will always populate the buffers with a file read before use. + .map(|_| Box::new_uninit_slice(SLABSIZE)) + .map(|x| unsafe { x.assume_init() }) + .map(|buffer| { + Arc::new(FileSlab { + state: RwLock::new(FileSlabState { + buffer, + valid_bytes: 0, + }), pool: Arc::clone(&pool), - } - ) - ) - .collect(); + }) + }) + .collect(); let mut buffers = pool.buffers.lock().unwrap(); *buffers = chunk_buffers; } @@ -223,18 +225,23 @@ impl FileSlabPool { fn get_slab(self: &Arc) -> Arc { let mut buffers = self.buffers.lock().unwrap(); - while buffers.is_empty() { - // Instead of creating a new slab when we're out, we wait for a slab to be returned before waking up. - // This potentially allows us to rate limit the CSV reader until downstream consumers are ready for data. - buffers = self.condvar.wait(buffers).unwrap(); + let buffer = buffers.pop(); + match buffer { + Some(buffer) => buffer, + None => Arc::new(FileSlab { + state: RwLock::new(FileSlabState { + buffer: unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, + valid_bytes: 0, + }), + pool: Arc::clone(self), + }), } - buffers.pop().unwrap() } fn return_slab(&self, slab: Arc) { - let mut slabs = self.buffers.lock().unwrap(); - slabs.push(slab); - self.condvar.notify_one(); + if let Ok(mut slabs) = self.buffers.lock() { + slabs.push(slab); + } } } @@ -376,13 +383,6 @@ pub async fn stream_csv_local( .unwrap_or(DEFAULT_CHUNK_SIZE); let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; let num_fields = schema.fields.len(); - // TODO(desmond): We might consider creating per-process buffer pools and slab pools. - let buffer_pool = Arc::new(CsvBufferPool::new( - record_buffer_size, - num_fields, - chunk_size_rows, - n_threads * 2, - )); // We suppose that each slab of CSV data produces (chunk size / slab size) number of Daft tables. We // then double this capacity to ensure that our channel is never full and our threads won't deadlock. @@ -391,6 +391,13 @@ pub async fn stream_csv_local( // Consume the CSV file asynchronously. rayon::spawn(move || { + // TODO(desmond): We might consider creating per-process buffer pools and slab pools. + let buffer_pool = Arc::new(CsvBufferPool::new( + record_buffer_size, + num_fields, + chunk_size_rows, + n_threads * 2, + )); consume_slab_iterator( file, buffer_pool, @@ -469,20 +476,22 @@ impl SlabIterator { } impl Iterator for SlabIterator { - type Item = Arc; + type Item = (Arc, usize, bool); fn next(&mut self) -> Option { let slab = self.slabpool.get_slab(); - { + let (bytes_read, filled_buffer) = { let mut writer = slab.state.write().unwrap(); let bytes_read = self.file.read(&mut writer.buffer).unwrap(); if bytes_read == 0 { return None; } + let filled_buffer = bytes_read == writer.buffer.len(); self.total_bytes_read += bytes_read; writer.valid_bytes = bytes_read; - } - - Some(slab) + (bytes_read, filled_buffer) + }; + + Some((slab, bytes_read, filled_buffer)) } } @@ -530,10 +539,11 @@ fn consume_slab_iterator( } // Grab a starting file slab if the current CSV chunk is empty. if curr_chunk.is_empty() { - if let Some(next) = slab_iterator.next() { + if let Some((next, bytes_read, _)) = slab_iterator.next() { curr_chunk.states.push(ChunkState::Start { slab: next, start: 0, + end: bytes_read, }); curr_chunk.reset(); } else { @@ -544,16 +554,19 @@ fn consume_slab_iterator( } // Grab file slabs until we find a valid CSV chunk. loop { - if let Some(next) = slab_iterator.next() { + if let Some((next, bytes_read, filled_buffer)) = slab_iterator.next() { // If the next buffer is not completely filled, we take this to mean that we've reached EOF. - if !next.filled_buffer() { + if !filled_buffer { curr_chunk.states.push(ChunkState::Final { - end: next.get_valid_bytes(), + end: bytes_read, slab: next, }); break; } - curr_chunk.states.push(ChunkState::Continue { slab: next }); + curr_chunk.states.push(ChunkState::Continue { + slab: next, + end: bytes_read, + }); while curr_chunk.goto_next_newline() { if curr_chunk.validate_csv_record(&mut csv_validator, &mut slab_iterator) { break; @@ -566,18 +579,13 @@ fn consume_slab_iterator( // If there is no next file slab, turn the last ChunkState into a final ChunkState. if let Some(last_state) = curr_chunk.states.pop() { match last_state { - ChunkState::Start { slab, start } => { - curr_chunk.states.push(ChunkState::StartAndFinal { - end: slab.get_valid_bytes(), - slab, - start, - }); + ChunkState::Start { slab, start, end } => { + curr_chunk + .states + .push(ChunkState::StartAndFinal { slab, start, end }); } - ChunkState::Continue { slab } => { - curr_chunk.states.push(ChunkState::Final { - end: slab.get_valid_bytes(), - slab, - }); + ChunkState::Continue { slab, end } => { + curr_chunk.states.push(ChunkState::Final { slab, end }); } _ => panic!("There should be no final CSV chunk states at this point."), } @@ -591,7 +599,6 @@ fn consume_slab_iterator( curr_chunk.states = std::mem::take(&mut curr_chunk.next_states); curr_chunk.reset(); let parse_options = parse_options.clone(); - let csv_buffer = buffer_pool.get_buffer(); let projection_indices = projection_indices.clone(); let fields = fields.clone(); let read_daft_fields = read_daft_fields.clone(); @@ -600,6 +607,7 @@ fn consume_slab_iterator( let predicate = predicate.clone(); let sender = sender.clone(); let rows_read = Arc::clone(&rows_read); + let mut csv_buffer = buffer_pool.get_buffer(); rayon::spawn(move || { if let Some(state) = states_to_read.last() { assert!(state.is_final()); @@ -615,7 +623,7 @@ fn consume_slab_iterator( fields, read_daft_fields, read_schema, - csv_buffer, + &mut csv_buffer, include_columns, predicate, sender, @@ -634,6 +642,7 @@ struct ChunkStateHolder { curr_byte_read_idx: usize, curr_byte_read_offset: usize, valid_chunk: bool, + // current_reader: Option>, } impl ChunkStateHolder { @@ -646,6 +655,7 @@ impl ChunkStateHolder { curr_byte_read_idx: 0, curr_byte_read_offset: 0, valid_chunk: false, + // current_reader: None, } } @@ -747,56 +757,72 @@ impl ChunkStateHolder { fn validate_csv_record( &mut self, validator: &mut CsvValidator, - iter: &mut impl Iterator>, + slab_iter: &mut impl Iterator, usize, bool)>, ) -> bool { validator.reset(); loop { // Run the CSV state machine to see if we're currently at a valid record boundary. - if let Some(valid) = validator.validate_record(self.into_iter()) { - self.valid_chunk = valid; - if valid { - let (ending_idx, ending_offset) = if self.curr_byte_read_offset == 0 { - ( - self.curr_byte_read_idx - 1, - self.states[self.curr_byte_read_idx - 1].len(), - ) - } else { - (self.curr_byte_read_idx, self.curr_byte_read_offset) - }; - self.next_states = self.states.split_off(ending_idx); - if let Some(front_of_next) = self.next_states.get_mut(0) { - self.states.push(ChunkState::Final { - slab: front_of_next.clone_slab(), - end: ending_offset, - }); - *front_of_next = ChunkState::Start { - slab: front_of_next.clone_slab(), - start: ending_offset, + if self.curr_byte_read_idx < self.states.len() { + let validity = { + let state = &self.states[self.curr_byte_read_idx]; + // Only the first slab that we're validating might not start from offset 0 (it starts from the offset where we found a newline). + if self.curr_byte_read_offset >= state.len() { + self.curr_byte_read_idx += 1; + self.curr_byte_read_offset = 0; + continue; + } + let guard = state.get_slab().state.read().unwrap(); + guard.validate_record(validator, self.curr_byte_read_offset) + }; + if let Some(valid) = validity { + self.valid_chunk = valid; + if valid { + let (ending_idx, ending_offset) = if self.curr_byte_read_offset == 0 { + ( + self.curr_byte_read_idx - 1, + self.states[self.curr_byte_read_idx - 1].len(), + ) + } else { + (self.curr_byte_read_idx, self.curr_byte_read_offset) }; - } else { - panic!("There should be at least one chunk state that's split off."); + self.next_states = self.states.split_off(ending_idx); + if let Some(front_of_next) = self.next_states.get_mut(0) { + self.states.push(ChunkState::Final { + slab: Arc::clone(front_of_next.get_slab()), + end: ending_offset, + }); + *front_of_next = ChunkState::Start { + slab: Arc::clone(front_of_next.get_slab()), + start: ending_offset, + end: front_of_next.len(), + }; + } else { + panic!("There should be at least one chunk state that's split off."); + } } + return valid; } - return valid; - } else { - // We ran out of bytes while running the CSV state machine. Read another file slab then - // continue running the state machine. - if let Some(next) = iter.next() { - if !next.filled_buffer() { - // EOF. Make this chunk state holder valid and exit. - self.states.push(ChunkState::Final { - end: next.get_valid_bytes(), - slab: next, - }); - self.valid_chunk = true; - return true; - } - self.states.push(ChunkState::Continue { slab: next }); - } else { + } + // We ran out of bytes while running the CSV state machine. Read another file slab then + // continue running the state machine. + if let Some((next, bytes_read, filled_buffer)) = slab_iter.next() { + if !filled_buffer { // EOF. Make this chunk state holder valid and exit. + self.states.push(ChunkState::Final { + slab: next, + end: bytes_read, + }); self.valid_chunk = true; return true; } + self.states.push(ChunkState::Continue { + slab: next, + end: bytes_read, + }); + } else { + // EOF. Make this chunk state holder valid and exit. + self.valid_chunk = true; + return true; } } } @@ -813,31 +839,12 @@ impl ChunkStateHolder { } } -impl<'a> Iterator for ChunkStateHolder { - type Item = u8; - - fn next(&mut self) -> Option { - loop { - if self.curr_byte_read_idx >= self.states.len() { - return None; - } - if self.curr_byte_read_offset >= self.states[self.curr_byte_read_idx].len() { - self.curr_byte_read_offset = 0; - self.curr_byte_read_idx += 1; - continue; - } - let byte = self.states[self.curr_byte_read_idx].get_idx(self.curr_byte_read_offset); - self.curr_byte_read_offset += 1; - return Some(byte); - } - } -} - #[derive(Debug)] enum ChunkState { Start { slab: Arc, start: usize, + end: usize, }, StartAndFinal { slab: Arc, @@ -846,6 +853,7 @@ enum ChunkState { }, Continue { slab: Arc, + end: usize, }, Final { slab: Arc, @@ -862,76 +870,43 @@ impl ChunkState { #[inline] fn len(&self) -> usize { match self { - Self::Start { slab, start } => slab.get_valid_bytes() - start, + Self::Start { start, end, .. } => end - start, Self::StartAndFinal { start, end, .. } => end - start, - Self::Continue { slab } => slab.get_valid_bytes(), + Self::Continue { end, .. } => *end, Self::Final { end, .. } => *end, } } - #[inline] - fn get_idx(&self, idx: usize) -> u8 { - match self { - Self::Start { slab, start } => slab.get_byte(start + idx), - Self::StartAndFinal { slab, start, .. } => slab.get_byte(start + idx), - Self::Continue { slab } => slab.get_byte(idx), - Self::Final { slab, .. } => slab.get_byte(idx), - } - } - #[inline] fn find_newline(&self, offset: usize) -> Option { match self { - Self::Continue { slab } =>slab.find_newline(offset), + Self::Continue { slab, .. } => slab.find_newline(offset), // This function is not needed for non-Continue chunk states. _ => None, } } #[inline] - fn clone_slab(&self) -> Arc { + fn get_slab(&self) -> &Arc { match self { - Self::Start { slab, .. } => slab.clone(), - Self::StartAndFinal { slab, .. } => slab.clone(), - Self::Continue { slab } => slab.clone(), - Self::Final { slab, .. } => slab.clone(), + Self::Start { slab, .. } => slab, + Self::StartAndFinal { slab, .. } => slab, + Self::Continue { slab, .. } => slab, + Self::Final { slab, .. } => slab, } } } struct MultiSliceReader<'a> { - slices: Vec<(usize, usize, std::sync::RwLockReadGuard<'a, FileSlabState>)>, + states: &'a [ChunkState], curr_read_idx: usize, curr_read_offset: usize, } impl<'a> MultiSliceReader<'a> { fn new(states: &'a [ChunkState]) -> Self { - let mut slices = Vec::with_capacity(states.len()); - for state in states { - let (start, end, guard) = match state { - ChunkState::Start { slab, start } => { - let guard = slab.state.read().unwrap(); - (*start, guard.valid_bytes, guard) - }, - ChunkState::StartAndFinal { slab, start, end } => { - let guard = slab.state.read().unwrap(); - (*start, *end, guard) - }, - ChunkState::Continue { slab } => { - let guard = slab.state.read().unwrap(); - (0, guard.valid_bytes, guard) - }, - ChunkState::Final { slab, end } => { - let guard = slab.state.read().unwrap(); - (0, *end, guard) - }, - }; - slices.push((start, end, guard)); - } - Self { - slices, + states, curr_read_idx: 0, curr_read_offset: 0, } @@ -942,16 +917,37 @@ impl<'a> Read for MultiSliceReader<'a> { fn read(&mut self, buf: &mut [u8]) -> std::io::Result { let buf_len = buf.len(); let mut position = 0; - while self.curr_read_idx < self.slices.len() && position < buf_len { - let (start, end, reader) = &self.slices[self.curr_read_idx]; - let slice = &reader.buffer[*start..*end]; + while self.curr_read_idx < self.states.len() && position < buf_len { + let state = &self.states[self.curr_read_idx]; + let (start, end, guard) = match state { + ChunkState::Start { slab, start, end } => { + let guard: std::sync::RwLockReadGuard<'_, FileSlabState> = + slab.state.read().unwrap(); + (*start, *end, guard) + } + ChunkState::StartAndFinal { slab, start, end } => { + let guard = slab.state.read().unwrap(); + (*start, *end, guard) + } + ChunkState::Continue { slab, end } => { + let guard = slab.state.read().unwrap(); + (0, *end, guard) + } + ChunkState::Final { slab, end } => { + let guard = slab.state.read().unwrap(); + (0, *end, guard) + } + }; + let slice = &guard.buffer[start..end]; if self.curr_read_offset < slice.len() { let read_size = (buf_len - position).min(slice.len() - self.curr_read_offset); - buf[position..position + read_size] - .copy_from_slice(&slice[self.curr_read_offset..][..read_size]); + buf[position..position + read_size].copy_from_slice( + &slice[self.curr_read_offset..self.curr_read_offset + read_size], + ); self.curr_read_offset += read_size; position += read_size; - } else { + } + if self.curr_read_offset >= slice.len() { self.curr_read_offset = 0; self.curr_read_idx += 1; } @@ -1073,8 +1069,8 @@ impl CsvValidator { self.state = CsvState::FieldStart; } - fn validate_record(&mut self, iter: &mut impl Iterator) -> Option { - for byte in iter { + fn validate_record<'a>(&mut self, iter: &mut impl Iterator) -> Option { + for &byte in iter { let next_state = self.transition_table[self.state as usize][byte as usize]; match next_state { @@ -1109,7 +1105,7 @@ fn dispatch_to_parse_csv( fields: Vec, read_daft_fields: Arc>>, read_schema: Arc, - csv_buffer: CsvBuffer, + csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, sender: Sender>, @@ -1164,24 +1160,24 @@ fn parse_csv_chunk( fields: Vec, read_daft_fields: Arc>>, read_schema: Arc, - csv_buffer: CsvBuffer, + csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, ) -> DaftResult> where R: std::io::Read, { - let mut chunk_buffer = csv_buffer.buffer; + // let mut chunk_buffer = &mut csv_buffer.buffer; let mut tables = vec![]; loop { - let (rows_read, has_more) = - local_read_rows(&mut reader, chunk_buffer.as_mut_slice()).context(ArrowSnafu {})?; + let (rows_read, has_more) = local_read_rows(&mut reader, csv_buffer.buffer.as_mut_slice()) + .context(ArrowSnafu {})?; let chunk = projection_indices .par_iter() .enumerate() .map(|(i, proj_idx)| { let deserialized_col = deserialize_column( - &chunk_buffer[0..rows_read], + &csv_buffer.buffer[0..rows_read], *proj_idx, fields[*proj_idx].data_type().clone(), 0, @@ -1212,6 +1208,5 @@ where break; } } - csv_buffer.pool.return_buffer(chunk_buffer); Ok(tables) } From 024fbd7c74e1038386784c796799c769756f93a1 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 22 Oct 2024 11:36:08 -0700 Subject: [PATCH 19/38] Clean up buffer pools --- src/daft-csv/src/local.rs | 96 ++++++++++++++++++--------------------- 1 file changed, 43 insertions(+), 53 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 793671dbd5..12434c6af2 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -4,7 +4,7 @@ use std::{ num::NonZeroUsize, sync::{ atomic::{AtomicUsize, Ordering}, - Arc, Mutex, RwLock, + Arc, Mutex, RwLock, Weak, }, }; @@ -77,8 +77,8 @@ struct CsvBufferPool { /// A slab of ByteRecords. Used for deserializing CSV. struct CsvBuffer { - pool: Arc, buffer: Vec, + pool: Weak, } impl CsvBufferPool { @@ -119,8 +119,8 @@ impl CsvBufferPool { }; CsvBuffer { - pool: Arc::clone(self), buffer, + pool: Arc::downgrade(self), } } @@ -133,9 +133,10 @@ impl CsvBufferPool { impl Drop for CsvBuffer { fn drop(&mut self) { - // Take ownership of the buffer using std::mem::take - let buffer = std::mem::take(&mut self.buffer); - self.pool.return_buffer(buffer); + if let Some(pool) = self.pool.upgrade() { + let buffer = std::mem::take(&mut self.buffer); + pool.return_buffer(buffer); + } } } @@ -147,10 +148,10 @@ const SLABPOOL_DEFAULT_SIZE: usize = 20; /// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. #[derive(Debug)] struct FileSlabPool { - buffers: Mutex>>, + slabs: Mutex>>, } -#[derive(Debug)] +#[derive(Debug, Default)] struct FileSlabState { buffer: Box<[u8]>, valid_bytes: usize, @@ -170,7 +171,7 @@ impl FileSlabState { #[derive(Debug)] struct FileSlab { state: RwLock, - pool: Arc, + pool: Weak, } impl FileSlab { @@ -180,66 +181,55 @@ impl FileSlab { } } -// Modify the Drop method for FileSlabs so that they're returned to their parent slab pool. +// Modify the Drop method for FileSlabs so that their states are returned to their parent slab pool. impl Drop for FileSlab { fn drop(&mut self) { - let pool = Arc::clone(&self.pool); - - let new_slab = Arc::new(Self { - state: RwLock::new(FileSlabState { - buffer: std::mem::take(&mut self.state.get_mut().unwrap().buffer), - valid_bytes: self.state.get_mut().unwrap().valid_bytes, - }), - pool, - }); - - self.pool.return_slab(new_slab); + if let Some(pool) = self.pool.upgrade() { + let file_slab_state = std::mem::take(&mut self.state); + pool.return_slab(file_slab_state); + } } } impl FileSlabPool { fn new() -> Arc { - let pool = Arc::new(Self { - buffers: Mutex::new(vec![]), - }); - { - let chunk_buffers: Vec> = (0..SLABPOOL_DEFAULT_SIZE) - // We get uninitialized buffers because we will always populate the buffers with a file read before use. - .map(|_| Box::new_uninit_slice(SLABSIZE)) - .map(|x| unsafe { x.assume_init() }) - .map(|buffer| { - Arc::new(FileSlab { - state: RwLock::new(FileSlabState { - buffer, - valid_bytes: 0, - }), - pool: Arc::clone(&pool), - }) + let slabs: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + // We get uninitialized buffers because we will always populate the buffers with a file read before use. + .map(|_| Box::new_uninit_slice(SLABSIZE)) + .map(|x| unsafe { x.assume_init() }) + .map(|buffer| { + RwLock::new(FileSlabState { + buffer, + valid_bytes: 0, }) - .collect(); - let mut buffers = pool.buffers.lock().unwrap(); - *buffers = chunk_buffers; - } - pool + }) + .collect(); + Arc::new(Self { + slabs: Mutex::new(slabs), + }) } fn get_slab(self: &Arc) -> Arc { - let mut buffers = self.buffers.lock().unwrap(); - let buffer = buffers.pop(); - match buffer { - Some(buffer) => buffer, - None => Arc::new(FileSlab { - state: RwLock::new(FileSlabState { + let slab = { + let mut slabs = self.slabs.lock().unwrap(); + let slab = slabs.pop(); + match slab { + Some(slab) => slab, + None => RwLock::new(FileSlabState { buffer: unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, valid_bytes: 0, }), - pool: Arc::clone(self), - }), - } + } + }; + + Arc::new(FileSlab { + state: slab, + pool: Arc::downgrade(self), + }) } - fn return_slab(&self, slab: Arc) { - if let Ok(mut slabs) = self.buffers.lock() { + fn return_slab(&self, slab: RwLock) { + if let Ok(mut slabs) = self.slabs.lock() { slabs.push(slab); } } From f9a0cb44b85ef383632526f8bfa5be01ea132929 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 22 Oct 2024 12:19:31 -0700 Subject: [PATCH 20/38] Enforce ordering of results with oneshot channels --- src/daft-csv/src/local.rs | 36 +++++++++++++++++++++++------------- 1 file changed, 23 insertions(+), 13 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 12434c6af2..589ed9b3b3 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -25,7 +25,7 @@ use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; -use futures::Stream; +use futures::{Stream, StreamExt}; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -404,7 +404,17 @@ pub async fn stream_csv_local( ); }); - Ok(futures::stream::iter(receiver)) + let stream = futures::stream::iter(receiver) + .map(|oneshot_rx| async move { + oneshot_rx + .await + .context(super::OneShotRecvSnafu {}) + .map_err(DaftError::from) + .and_then(|result| result) + }) + .then(|future| future); + + Ok(stream) } /// Helper function that reads up to 1 MiB of the CSV file to estimate stats and/or infer the schema of the file. @@ -498,7 +508,7 @@ fn consume_slab_iterator( include_columns: Option>, predicate: Option>, limit: Option, - sender: Sender>, + sender: Sender>>, ) { // Create slab pool for file reads. let slabpool = FileSlabPool::new(); @@ -598,6 +608,7 @@ fn consume_slab_iterator( let sender = sender.clone(); let rows_read = Arc::clone(&rows_read); let mut csv_buffer = buffer_pool.get_buffer(); + let (oneshot_send, oneshot_recv) = tokio::sync::oneshot::channel(); rayon::spawn(move || { if let Some(state) = states_to_read.last() { assert!(state.is_final()); @@ -616,11 +627,13 @@ fn consume_slab_iterator( &mut csv_buffer, include_columns, predicate, - sender, + oneshot_send, rows_read, ); }); has_header = false; + + let _ = sender.send(oneshot_recv); } } @@ -1098,7 +1111,7 @@ fn dispatch_to_parse_csv( csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, - sender: Sender>, + sender: tokio::sync::oneshot::Sender>, rows_read: Arc, ) where R: std::io::Read, @@ -1126,15 +1139,12 @@ fn dispatch_to_parse_csv( }; match table_results { Ok(tables) => { - for table in tables { - let table_len = table.len(); - let _ = sender.send(Ok(table)); - // Atomically update the number of rows read only after the result has - // been sent. In theory we could wrap these steps in a mutex, but - // applying limit at this layer can be best-effort with no adverse - // side effects. - rows_read.fetch_add(table_len, Ordering::Relaxed); + let concated_tables = tables_concat(tables); + if let Ok(table) = &concated_tables { + // Update the number of rows read so the main thread can apply the limit, if any. + rows_read.fetch_add(table.len(), Ordering::SeqCst); } + let _ = sender.send(concated_tables); } Err(e) => { let _ = sender.send(Err(e)); From 16c3e0e73ab88e834ffbc7ef04e383df796f91f4 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 22 Oct 2024 14:57:50 -0700 Subject: [PATCH 21/38] Avoid concat --- src/daft-csv/src/local.rs | 50 +++++++++++++++++++-------------------- 1 file changed, 24 insertions(+), 26 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 589ed9b3b3..b0c8582c43 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -25,7 +25,7 @@ use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; -use futures::{Stream, StreamExt}; +use futures::Stream; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -314,6 +314,7 @@ pub async fn stream_csv_local( ) -> DaftResult> + Send> { let uri = uri.trim_start_matches("file://"); let file = std::fs::File::open(uri)?; + let file_len = file.metadata()?.len(); // Process the CSV convert options. let predicate = convert_options @@ -374,10 +375,12 @@ pub async fn stream_csv_local( let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; let num_fields = schema.fields.len(); - // We suppose that each slab of CSV data produces (chunk size / slab size) number of Daft tables. We - // then double this capacity to ensure that our channel is never full and our threads won't deadlock. - let (sender, receiver) = - crossbeam_channel::bounded(max_chunks_in_flight.unwrap_or(2 * chunk_size / SLABSIZE)); + // We produce `file_len / SLABSIZE` number of file slabs, each of which might be split into two, so the maximum number + // of tasks we might spawn is 2 times this number. + let (sender, receiver) = crossbeam_channel::bounded( + max_chunks_in_flight + .unwrap_or_else(|| 2 * (file_len as f64 / SLABSIZE as f64).ceil() as usize), + ); // Consume the CSV file asynchronously. rayon::spawn(move || { @@ -404,16 +407,8 @@ pub async fn stream_csv_local( ); }); - let stream = futures::stream::iter(receiver) - .map(|oneshot_rx| async move { - oneshot_rx - .await - .context(super::OneShotRecvSnafu {}) - .map_err(DaftError::from) - .and_then(|result| result) - }) - .then(|future| future); - + let flattened_receiver = receiver.into_iter().flat_map(|rx| rx.into_iter()); + let stream = futures::stream::iter(flattened_receiver); Ok(stream) } @@ -508,7 +503,7 @@ fn consume_slab_iterator( include_columns: Option>, predicate: Option>, limit: Option, - sender: Sender>>, + sender: Sender>>, ) { // Create slab pool for file reads. let slabpool = FileSlabPool::new(); @@ -605,10 +600,14 @@ fn consume_slab_iterator( let read_schema = read_schema.clone(); let include_columns = include_columns.clone(); let predicate = predicate.clone(); - let sender = sender.clone(); + let stream_sender = sender.clone(); let rows_read = Arc::clone(&rows_read); let mut csv_buffer = buffer_pool.get_buffer(); - let (oneshot_send, oneshot_recv) = tokio::sync::oneshot::channel(); + // We produce roughly `SLABSIZE / DEFAULT_CHUNK_SIZE` tables per state, with maybe an overflow of 1 table per state. + let (tx, rx) = crossbeam_channel::bounded( + 2 * (states_to_read.len() as f64 * SLABSIZE as f64 / DEFAULT_CHUNK_SIZE as f64).ceil() + as usize, + ); rayon::spawn(move || { if let Some(state) = states_to_read.last() { assert!(state.is_final()); @@ -627,13 +626,13 @@ fn consume_slab_iterator( &mut csv_buffer, include_columns, predicate, - oneshot_send, + tx, rows_read, ); }); has_header = false; - let _ = sender.send(oneshot_recv); + let _ = stream_sender.send(rx); } } @@ -1111,7 +1110,7 @@ fn dispatch_to_parse_csv( csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, - sender: tokio::sync::oneshot::Sender>, + sender: crossbeam_channel::Sender>, rows_read: Arc, ) where R: std::io::Read, @@ -1139,12 +1138,11 @@ fn dispatch_to_parse_csv( }; match table_results { Ok(tables) => { - let concated_tables = tables_concat(tables); - if let Ok(table) = &concated_tables { - // Update the number of rows read so the main thread can apply the limit, if any. - rows_read.fetch_add(table.len(), Ordering::SeqCst); + for table in tables { + let table_len = table.len(); + let _ = sender.send(Ok(table)); + rows_read.fetch_add(table_len, Ordering::SeqCst); } - let _ = sender.send(concated_tables); } Err(e) => { let _ = sender.send(Err(e)); From 93350f0ef10976ffad6a967ea4ec465635e3dd9b Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 22 Oct 2024 15:19:26 -0700 Subject: [PATCH 22/38] Clean up unused deps --- Cargo.lock | 1 - src/daft-csv/Cargo.toml | 2 +- src/daft-csv/src/local.rs | 8 +++----- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 514becbb94..ade5517d93 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1772,7 +1772,6 @@ dependencies = [ "daft-table", "futures", "memchr", - "parking_lot", "pyo3", "rayon", "rstest", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index 00b8899856..ca74b9c2ec 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -20,7 +20,7 @@ serde = {workspace = true} snafu = {workspace = true} tokio = {workspace = true} tokio-util = {workspace = true} -parking_lot = "*" + [dev-dependencies] rstest = {workspace = true} diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index b0c8582c43..a1e2857d56 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -347,12 +347,12 @@ pub async fn stream_csv_local( let (schema, estimated_mean_row_size, estimated_std_row_size) = get_schema_and_estimators(uri, &convert_options, &parse_options, io_client, io_stats) .await?; + let num_fields = schema.fields.len(); let projection_indices = fields_to_projection_indices(&schema.fields, &convert_options.clone().include_columns); - let fields = schema.clone().fields; let fields_subset = projection_indices .iter() - .map(|i| fields.get(*i).unwrap().into()) + .map(|i| schema.fields.get(*i).unwrap().into()) .collect::>(); let read_schema = Arc::new(Schema::new(fields_subset)?); let read_daft_fields = Arc::new( @@ -373,7 +373,6 @@ pub async fn stream_csv_local( .and_then(|opt| opt.chunk_size.or_else(|| opt.buffer_size.map(|bs| bs / 8))) .unwrap_or(DEFAULT_CHUNK_SIZE); let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; - let num_fields = schema.fields.len(); // We produce `file_len / SLABSIZE` number of file slabs, each of which might be split into two, so the maximum number // of tasks we might spawn is 2 times this number. @@ -399,7 +398,7 @@ pub async fn stream_csv_local( projection_indices, read_daft_fields, read_schema, - fields, + schema.fields, include_columns, predicate, limit, @@ -1165,7 +1164,6 @@ fn parse_csv_chunk( where R: std::io::Read, { - // let mut chunk_buffer = &mut csv_buffer.buffer; let mut tables = vec![]; loop { let (rows_read, has_more) = local_read_rows(&mut reader, csv_buffer.buffer.as_mut_slice()) From b87caba7c6ce952b34d6857f41fdccef34724e6b Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Tue, 22 Oct 2024 16:01:39 -0700 Subject: [PATCH 23/38] Clean up --- src/daft-csv/src/local.rs | 96 +++++++++++++++++++++------------------ 1 file changed, 51 insertions(+), 45 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index a1e2857d56..1d079c6112 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -151,46 +151,6 @@ struct FileSlabPool { slabs: Mutex>>, } -#[derive(Debug, Default)] -struct FileSlabState { - buffer: Box<[u8]>, - valid_bytes: usize, -} - -impl FileSlabState { - fn find_newline(&self, offset: usize) -> Option { - newline_position(&self.buffer[offset..self.valid_bytes]) - } - - fn validate_record(&self, validator: &mut CsvValidator, start: usize) -> Option { - validator.validate_record(&mut self.buffer[start..self.valid_bytes].iter()) - } -} - -/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. -#[derive(Debug)] -struct FileSlab { - state: RwLock, - pool: Weak, -} - -impl FileSlab { - fn find_newline(&self, offset: usize) -> Option { - let guard = self.state.read().unwrap(); - guard.find_newline(offset) - } -} - -// Modify the Drop method for FileSlabs so that their states are returned to their parent slab pool. -impl Drop for FileSlab { - fn drop(&mut self) { - if let Some(pool) = self.pool.upgrade() { - let file_slab_state = std::mem::take(&mut self.state); - pool.return_slab(file_slab_state); - } - } -} - impl FileSlabPool { fn new() -> Arc { let slabs: Vec> = (0..SLABPOOL_DEFAULT_SIZE) @@ -235,6 +195,49 @@ impl FileSlabPool { } } +/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. +#[derive(Debug)] +struct FileSlab { + state: RwLock, + pool: Weak, +} + +impl FileSlab { + fn find_newline(&self, offset: usize) -> Option { + let guard = self.state.read().unwrap(); + guard.find_newline(offset) + } +} + +// Modify the Drop method for FileSlabs so that their states are returned to their parent slab pool. +impl Drop for FileSlab { + fn drop(&mut self) { + if let Some(pool) = self.pool.upgrade() { + let file_slab_state = std::mem::take(&mut self.state); + pool.return_slab(file_slab_state); + } + } +} + +#[derive(Debug, Default)] +struct FileSlabState { + buffer: Box<[u8]>, + valid_bytes: usize, +} + +impl FileSlabState { + /// Helper function that find the first \n char in the file slab state's buffer starting from `offset.` + fn find_newline(&self, offset: usize) -> Option { + newline_position(&self.buffer[offset..self.valid_bytes]) + } + + /// Validate the CSV record in the file slab state's buffer starting from `start`. `validator` is a + /// state machine that might need to process multiple buffers to validate CSV records. + fn validate_record(&self, validator: &mut CsvValidator, start: usize) -> Option { + validator.validate_record(&mut self.buffer[start..self.valid_bytes].iter()) + } +} + /// Reads a single local CSV file in a non-streaming fashion. pub async fn read_csv_local( uri: &str, @@ -411,7 +414,7 @@ pub async fn stream_csv_local( Ok(stream) } -/// Helper function that reads up to 1 MiB of the CSV file to estimate stats and/or infer the schema of the file. +/// Helper function that reads up to 1 MiB of the CSV file to estimate stats and/or infer the schema of the file. async fn get_schema_and_estimators( uri: &str, convert_options: &CsvConvertOptions, @@ -453,6 +456,8 @@ async fn get_schema_and_estimators( )) } +/// A helper iterator that takes in a File and FileSlabPool and produces an iterator of FileSlabs +/// over the given file. struct SlabIterator { file: std::fs::File, slabpool: Arc, @@ -643,7 +648,6 @@ struct ChunkStateHolder { curr_byte_read_idx: usize, curr_byte_read_offset: usize, valid_chunk: bool, - // current_reader: Option>, } impl ChunkStateHolder { @@ -656,14 +660,15 @@ impl ChunkStateHolder { curr_byte_read_idx: 0, curr_byte_read_offset: 0, valid_chunk: false, - // current_reader: None, } } + /// Creates an empty ChunkStateHolder. fn empty() -> Self { Self::new(vec![]) } + /// Checks if the current ChunkStateHolder is empty. fn is_empty(&self) -> bool { self.states.is_empty() } @@ -882,8 +887,7 @@ impl ChunkState { fn find_newline(&self, offset: usize) -> Option { match self { Self::Continue { slab, .. } => slab.find_newline(offset), - // This function is not needed for non-Continue chunk states. - _ => None, + _ => panic!("find_newline should never be called on non-Continue chunk states"), } } @@ -898,6 +902,7 @@ impl ChunkState { } } +/// A helper struct that implements `std::io::Read` over a slice of ChunkStates' buffers. struct MultiSliceReader<'a> { states: &'a [ChunkState], curr_read_idx: usize, @@ -971,6 +976,7 @@ fn newline_position(buffer: &[u8]) -> Option { memchr::memchr(NEWLINE, buffer) } +/// State machine that validates CSV records. struct CsvValidator { state: CsvState, num_fields: usize, From 1cac1a06b461c536507da70441060299d849a762 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Tue, 22 Oct 2024 21:59:54 -0700 Subject: [PATCH 24/38] minor cleanup --- src/daft-csv/src/local.rs | 703 ++++++++++++++------------------------ 1 file changed, 261 insertions(+), 442 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 1d079c6112..78b7c2010a 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -2,6 +2,7 @@ use core::str; use std::{ io::Read, num::NonZeroUsize, + path::Display, sync::{ atomic::{AtomicUsize, Ordering}, Arc, Mutex, RwLock, Weak, @@ -379,38 +380,38 @@ pub async fn stream_csv_local( // We produce `file_len / SLABSIZE` number of file slabs, each of which might be split into two, so the maximum number // of tasks we might spawn is 2 times this number. - let (sender, receiver) = crossbeam_channel::bounded( - max_chunks_in_flight - .unwrap_or_else(|| 2 * (file_len as f64 / SLABSIZE as f64).ceil() as usize), - ); + // let (sender, receiver) = crossbeam_channel::bounded( + // max_chunks_in_flight + // .unwrap_or_else(|| 2 * (file_len as f64 / SLABSIZE as f64).ceil() as usize), + // ); // Consume the CSV file asynchronously. - rayon::spawn(move || { - // TODO(desmond): We might consider creating per-process buffer pools and slab pools. - let buffer_pool = Arc::new(CsvBufferPool::new( - record_buffer_size, - num_fields, - chunk_size_rows, - n_threads * 2, - )); - consume_slab_iterator( - file, - buffer_pool, - num_fields, - parse_options, - projection_indices, - read_daft_fields, - read_schema, - schema.fields, - include_columns, - predicate, - limit, - sender, - ); - }); - - let flattened_receiver = receiver.into_iter().flat_map(|rx| rx.into_iter()); - let stream = futures::stream::iter(flattened_receiver); + // rayon::spawn(move || { + // TODO(desmond): We might consider creating per-process buffer pools and slab pools. + let buffer_pool = Arc::new(CsvBufferPool::new( + record_buffer_size, + num_fields, + chunk_size_rows, + n_threads * 2, + )); + let tables = consume_slab_iterator( + file, + buffer_pool, + num_fields, + &parse_options, + projection_indices, + read_daft_fields, + read_schema, + schema.fields, + include_columns, + predicate, + limit, + // sender, + )?; + // }); + + // let flattened_receiver = receiver.into_iter().flat_map(|rx| rx.into_iter()); + let stream = futures::stream::iter(tables.into_iter().map(Ok)); Ok(stream) } @@ -474,23 +475,114 @@ impl SlabIterator { } } +type SlabRow = (Arc, usize); + impl Iterator for SlabIterator { - type Item = (Arc, usize, bool); + type Item = SlabRow; fn next(&mut self) -> Option { let slab = self.slabpool.get_slab(); - let (bytes_read, filled_buffer) = { + let bytes_read = { let mut writer = slab.state.write().unwrap(); let bytes_read = self.file.read(&mut writer.buffer).unwrap(); if bytes_read == 0 { return None; } - let filled_buffer = bytes_read == writer.buffer.len(); self.total_bytes_read += bytes_read; writer.valid_bytes = bytes_read; - (bytes_read, filled_buffer) + bytes_read }; - Some((slab, bytes_read, filled_buffer)) + Some((slab, bytes_read)) + } +} + +struct ChunkyIterator { + slab_iter: I, + last_chunk: Option, + validator: CsvValidator, +} + +impl ChunkyIterator +where + I: Iterator, +{ + fn new(slab_iter: I, validator: CsvValidator) -> Self { + Self { + slab_iter, + last_chunk: None, + validator, + } + } +} + +impl Iterator for ChunkyIterator +where + I: Iterator, +{ + type Item = ChunkState; + fn next(&mut self) -> Option { + let curr_chunk = match &self.last_chunk { + Some(ChunkState::Start { .. } | ChunkState::Continue { .. }) => { + if let Some((slab, valid_bytes)) = self.slab_iter.next() { + let mut curr_pos = 0; + let mut chunk_state: Option = None; + while chunk_state.is_none() + && let Some(pos) = slab.find_newline(curr_pos) + && curr_pos < valid_bytes + { + let offset = curr_pos + pos; + let guard = slab.state.read().unwrap(); + chunk_state = match guard.validate_record(&mut self.validator, offset + 1) { + Some(true) => Some(ChunkState::Final { + slab: slab.clone(), + end: offset, + valid_bytes, + }), + None => Some(ChunkState::Continue { + slab: slab.clone(), + end: valid_bytes, + }), + Some(false) => { + curr_pos = offset + 1; + None + } + } + } + if let Some(chunk_state) = chunk_state { + Some(chunk_state) + } else { + Some(ChunkState::Continue { + slab: slab.clone(), + end: valid_bytes, + }) + } + } else { + None + } + } + Some(ChunkState::Final { + slab, + end, + valid_bytes, + }) => Some(ChunkState::Start { + slab: slab.clone(), + start: end + 1, + end: *valid_bytes, + }), + None => { + if let Some((slab, valid_bytes)) = self.slab_iter.next() { + Some(ChunkState::Start { + slab: slab, + start: 0, + end: valid_bytes, + }) + } else { + None + } + } + }; + self.last_chunk = curr_chunk.clone(); + curr_chunk } } @@ -499,7 +591,7 @@ fn consume_slab_iterator( file: std::fs::File, buffer_pool: Arc, num_fields: usize, - parse_options: CsvParseOptions, + parse_options: &CsvParseOptions, projection_indices: Arc>, read_daft_fields: Arc>>, read_schema: Arc, @@ -507,356 +599,124 @@ fn consume_slab_iterator( include_columns: Option>, predicate: Option>, limit: Option, - sender: Sender>>, -) { +) -> DaftResult> { // Create slab pool for file reads. let slabpool = FileSlabPool::new(); let rows_read = Arc::new(AtomicUsize::new(0)); - let mut slab_iterator = SlabIterator::new(file, slabpool); - let mut has_header = parse_options.has_header; - let field_delimiter = parse_options.delimiter; - let escape_char = parse_options.escape_char; - let quote_char = parse_options.quote; - let double_quote_escape_allowed = parse_options.double_quote; - - let mut curr_chunk = ChunkStateHolder::empty(); - let mut csv_validator = CsvValidator::new( + let slab_iterator = SlabIterator::new(file, slabpool); + + let csv_validator = CsvValidator::new( num_fields, - quote_char, - field_delimiter, - escape_char, - double_quote_escape_allowed, + parse_options.quote, + parse_options.delimiter, + parse_options.escape_char, + parse_options.double_quote, ); - loop { - // Check limit. - let limit_reached = limit.map_or(false, |limit| { - let current_rows_read = rows_read.load(Ordering::Relaxed); - current_rows_read >= limit - }); - if limit_reached { - break; - } - // Grab a starting file slab if the current CSV chunk is empty. - if curr_chunk.is_empty() { - if let Some((next, bytes_read, _)) = slab_iterator.next() { - curr_chunk.states.push(ChunkState::Start { - slab: next, - start: 0, - end: bytes_read, - }); - curr_chunk.reset(); - } else { - // EOF. - break; - } - continue; - } - // Grab file slabs until we find a valid CSV chunk. - loop { - if let Some((next, bytes_read, filled_buffer)) = slab_iterator.next() { - // If the next buffer is not completely filled, we take this to mean that we've reached EOF. - if !filled_buffer { - curr_chunk.states.push(ChunkState::Final { - end: bytes_read, - slab: next, - }); - break; - } - curr_chunk.states.push(ChunkState::Continue { - slab: next, - end: bytes_read, - }); - while curr_chunk.goto_next_newline() { - if curr_chunk.validate_csv_record(&mut csv_validator, &mut slab_iterator) { - break; - } - } - if curr_chunk.is_valid() { - break; - } - } else { - // If there is no next file slab, turn the last ChunkState into a final ChunkState. - if let Some(last_state) = curr_chunk.states.pop() { - match last_state { - ChunkState::Start { slab, start, end } => { - curr_chunk - .states - .push(ChunkState::StartAndFinal { slab, start, end }); - } - ChunkState::Continue { slab, end } => { - curr_chunk.states.push(ChunkState::Final { slab, end }); - } - _ => panic!("There should be no final CSV chunk states at this point."), - } - } else { - panic!("There should be at least one CSV chunk state at this point.") - } - break; - } - } - let states_to_read = curr_chunk.states; - curr_chunk.states = std::mem::take(&mut curr_chunk.next_states); - curr_chunk.reset(); - let parse_options = parse_options.clone(); - let projection_indices = projection_indices.clone(); - let fields = fields.clone(); - let read_daft_fields = read_daft_fields.clone(); - let read_schema = read_schema.clone(); - let include_columns = include_columns.clone(); - let predicate = predicate.clone(); - let stream_sender = sender.clone(); - let rows_read = Arc::clone(&rows_read); - let mut csv_buffer = buffer_pool.get_buffer(); - // We produce roughly `SLABSIZE / DEFAULT_CHUNK_SIZE` tables per state, with maybe an overflow of 1 table per state. - let (tx, rx) = crossbeam_channel::bounded( - 2 * (states_to_read.len() as f64 * SLABSIZE as f64 / DEFAULT_CHUNK_SIZE as f64).ceil() - as usize, - ); - rayon::spawn(move || { - if let Some(state) = states_to_read.last() { - assert!(state.is_final()); - } else { - return; - } - let multi_slice_reader = MultiSliceReader::new(states_to_read.as_slice()); - dispatch_to_parse_csv( - has_header, - parse_options, - multi_slice_reader, - projection_indices, - fields, - read_daft_fields, - read_schema, - &mut csv_buffer, - include_columns, - predicate, - tx, - rows_read, - ); - }); - has_header = false; - - let _ = stream_sender.send(rx); - } -} -struct ChunkStateHolder { - states: Vec, - next_states: Vec, - curr_newline_idx: usize, - curr_newline_offset: usize, - curr_byte_read_idx: usize, - curr_byte_read_offset: usize, - valid_chunk: bool, -} + let chunk_iterator = ChunkyIterator::new(slab_iterator, csv_validator); -impl ChunkStateHolder { - fn new(states: Vec) -> Self { - Self { - states, - next_states: vec![], - curr_newline_idx: 1, - curr_newline_offset: 0, - curr_byte_read_idx: 0, - curr_byte_read_offset: 0, - valid_chunk: false, - } - } + let has_header = parse_options.has_header; - /// Creates an empty ChunkStateHolder. - fn empty() -> Self { - Self::new(vec![]) - } + let mut chunks = vec![]; - /// Checks if the current ChunkStateHolder is empty. - fn is_empty(&self) -> bool { - self.states.is_empty() - } + let mut all_tables = vec![]; - #[allow(clippy::doc_lazy_continuation)] - /// `goto_next_line` and `validate_csv_record` are two helper function that determines what chunk of - /// data to parse given a starting position within the file, and the desired initial chunk size. - /// - /// Given a starting position, we use our chunk size to compute a preliminary start and stop - /// position. For example, we can visualize all preliminary chunks in a file as follows. - /// - /// Chunk 1 Chunk 2 Chunk 3 Chunk N - /// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ - /// │ │ │\n │ │ \n │ │ \n │ - /// │ │ │ │ │ │ │ │ - /// │ │ │ \n │ │ │ │ │ - /// │ \n │ │ │ │ \n │ │ │ - /// │ │ │ │ │ │ ... │ \n │ - /// │ │ │ \n │ │ │ │ │ - /// │ \n │ │ │ │ │ │ │ - /// │ │ │ │ │ \n │ │ \n │ - /// └──────────┘ └──────────┘ └──────────┘ └──────────┘ - /// - /// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary - /// chunk boundaries. So we adjust each preliminary chunk as follows: - /// - Find the first record terminator from the chunk's start. This is the new starting position. - /// - Find the first record terminator from the chunk's end. This is the new ending position. - /// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. - /// - /// For example: - /// - /// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N - /// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ - /// │ \n││ \n│ │ \n│ \n │ │ - /// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ - /// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ - /// │ \n │ │ │ │ \n │ │ │ - /// │ │ │ │ │ │ ... │ \n │ - /// │ │ │ \n │ │ │ │ │ - /// │ \n │ │ │ │ │ │ │ - /// │ │ │ │ │ \n │ │ \n │ - /// └──────────┘ └──────────┘ └──────────┘ └──────────┘ - /// - /// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do - /// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can - /// now happen in parallel. - /// - /// This is the same method as described in: - /// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). - /// - /// Another observation is that seeing a pure \n character is not necessarily indicative of a record - /// terminator. We need to consider whether the \n character was seen within a quoted field, since the - /// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following - /// algorithm: - /// 1. Find a \n character. - /// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse - /// as valid CSV, and does it produce the same number of fields as our schema. - /// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. - /// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next - /// \n character and go back to 2. - fn goto_next_newline(&mut self) -> bool { - self.valid_chunk = false; - loop { - if self.curr_newline_idx >= self.states.len() { - return false; - } - if self.curr_newline_offset >= self.states[self.curr_newline_idx].len() { - self.curr_newline_offset = 0; - self.curr_newline_idx += 1; - continue; - } - if let Some(pos) = - self.states[self.curr_newline_idx].find_newline(self.curr_newline_offset) - { - self.curr_newline_offset = pos + 1; - if self.curr_newline_offset >= self.states[self.curr_newline_idx].len() { - self.curr_newline_offset = 0; - self.curr_newline_idx += 1; - continue; - } - self.curr_byte_read_idx = self.curr_newline_idx; - self.curr_byte_read_offset = self.curr_newline_offset; - return true; - } else { - self.curr_newline_offset = 0; - self.curr_newline_idx += 1; - continue; - } - } - } + let mut flush = |chunks: &mut Vec<_>| { + let reader = MultiSliceReader::new(&chunks); + let mut buffer = buffer_pool.get_buffer(); + let tables = dispatch_to_parse_csv( + has_header, + parse_options, + reader, + projection_indices.clone(), + fields.clone(), + read_daft_fields.clone(), + read_schema.clone(), + &mut buffer, + include_columns.clone(), + predicate.clone(), + ) + .unwrap(); + all_tables.extend(tables.into_iter()); + chunks.clear(); + }; - fn validate_csv_record( - &mut self, - validator: &mut CsvValidator, - slab_iter: &mut impl Iterator, usize, bool)>, - ) -> bool { - validator.reset(); - loop { - // Run the CSV state machine to see if we're currently at a valid record boundary. - if self.curr_byte_read_idx < self.states.len() { - let validity = { - let state = &self.states[self.curr_byte_read_idx]; - // Only the first slab that we're validating might not start from offset 0 (it starts from the offset where we found a newline). - if self.curr_byte_read_offset >= state.len() { - self.curr_byte_read_idx += 1; - self.curr_byte_read_offset = 0; - continue; - } - let guard = state.get_slab().state.read().unwrap(); - guard.validate_record(validator, self.curr_byte_read_offset) - }; - if let Some(valid) = validity { - self.valid_chunk = valid; - if valid { - let (ending_idx, ending_offset) = if self.curr_byte_read_offset == 0 { - ( - self.curr_byte_read_idx - 1, - self.states[self.curr_byte_read_idx - 1].len(), - ) - } else { - (self.curr_byte_read_idx, self.curr_byte_read_offset) - }; - self.next_states = self.states.split_off(ending_idx); - if let Some(front_of_next) = self.next_states.get_mut(0) { - self.states.push(ChunkState::Final { - slab: Arc::clone(front_of_next.get_slab()), - end: ending_offset, - }); - *front_of_next = ChunkState::Start { - slab: Arc::clone(front_of_next.get_slab()), - start: ending_offset, - end: front_of_next.len(), - }; - } else { - panic!("There should be at least one chunk state that's split off."); - } - } - return valid; - } - } - // We ran out of bytes while running the CSV state machine. Read another file slab then - // continue running the state machine. - if let Some((next, bytes_read, filled_buffer)) = slab_iter.next() { - if !filled_buffer { - // EOF. Make this chunk state holder valid and exit. - self.states.push(ChunkState::Final { - slab: next, - end: bytes_read, - }); - self.valid_chunk = true; - return true; - } - self.states.push(ChunkState::Continue { - slab: next, - end: bytes_read, - }); - } else { - // EOF. Make this chunk state holder valid and exit. - self.valid_chunk = true; - return true; + for chunk in chunk_iterator { + chunks.push(chunk); + match chunks.last().unwrap() { + ChunkState::Final { .. } => { + flush(&mut chunks); } + _ => {} } } - - fn is_valid(&self) -> bool { - self.valid_chunk - } - - fn reset(&mut self) { - self.curr_newline_idx = 1; - self.curr_newline_offset = 0; - self.curr_byte_read_idx = 0; - self.curr_byte_read_offset = 0; - } + flush(&mut chunks); + Ok(all_tables) } -#[derive(Debug)] +/// `goto_next_line` and `validate_csv_record` are two helper function that determines what chunk of +/// data to parse given a starting position within the file, and the desired initial chunk size. +/// +/// Given a starting position, we use our chunk size to compute a preliminary start and stop +/// position. For example, we can visualize all preliminary chunks in a file as follows. +/// +/// Chunk 1 Chunk 2 Chunk 3 Chunk N +/// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ +/// │ │ │\n │ │ \n │ │ \n │ +/// │ │ │ │ │ │ │ │ +/// │ │ │ \n │ │ │ │ │ +/// │ \n │ │ │ │ \n │ │ │ +/// │ │ │ │ │ │ ... │ \n │ +/// │ │ │ \n │ │ │ │ │ +/// │ \n │ │ │ │ │ │ │ +/// │ │ │ │ │ \n │ │ \n │ +/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +/// +/// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary +/// chunk boundaries. So we adjust each preliminary chunk as follows: +/// - Find the first record terminator from the chunk's start. This is the new starting position. +/// - Find the first record terminator from the chunk's end. This is the new ending position. +/// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. +/// +/// For example: +/// +/// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N +/// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ +/// │ \n││ \n│ │ \n│ \n │ │ +/// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ +/// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ +/// │ \n │ │ │ │ \n │ │ │ +/// │ │ │ │ │ │ ... │ \n │ +/// │ │ │ \n │ │ │ │ │ +/// │ \n │ │ │ │ │ │ │ +/// │ │ │ │ │ \n │ │ \n │ +/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +/// +/// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do +/// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can +/// now happen in parallel. +/// +/// This is the same method as described in: +/// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). +/// +/// Another observation is that seeing a pure \n character is not necessarily indicative of a record +/// terminator. We need to consider whether the \n character was seen within a quoted field, since the +/// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following +/// algorithm: +/// 1. Find a \n character. +/// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse +/// as valid CSV, and does it produce the same number of fields as our schema. +/// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. +/// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next +/// \n character and go back to 2. + +#[derive(Debug, Clone)] enum ChunkState { Start { slab: Arc, start: usize, end: usize, }, - StartAndFinal { - slab: Arc, - start: usize, - end: usize, - }, Continue { slab: Arc, end: usize, @@ -864,44 +724,27 @@ enum ChunkState { Final { slab: Arc, end: usize, + valid_bytes: usize, }, } -impl ChunkState { - #[inline] - fn is_final(&self) -> bool { - matches!(self, Self::Final { .. } | Self::StartAndFinal { .. }) - } - - #[inline] - fn len(&self) -> usize { +impl std::fmt::Display for ChunkState { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - Self::Start { start, end, .. } => end - start, - Self::StartAndFinal { start, end, .. } => end - start, - Self::Continue { end, .. } => *end, - Self::Final { end, .. } => *end, - } - } - - #[inline] - fn find_newline(&self, offset: usize) -> Option { - match self { - Self::Continue { slab, .. } => slab.find_newline(offset), - _ => panic!("find_newline should never be called on non-Continue chunk states"), - } - } - - #[inline] - fn get_slab(&self) -> &Arc { - match self { - Self::Start { slab, .. } => slab, - Self::StartAndFinal { slab, .. } => slab, - Self::Continue { slab, .. } => slab, - Self::Final { slab, .. } => slab, + ChunkState::Start { + slab: _, + start, + end, + } => write!(f, "ChunkState::Start-{start}-{end}"), + ChunkState::Final { + slab: _, + end, + valid_bytes, + } => write!(f, "ChunkState::Final-{end}-{valid_bytes}"), + ChunkState::Continue { slab: _, end } => write!(f, "ChunkState::Continue-{end}"), } } } - /// A helper struct that implements `std::io::Read` over a slice of ChunkStates' buffers. struct MultiSliceReader<'a> { states: &'a [ChunkState], @@ -931,15 +774,11 @@ impl<'a> Read for MultiSliceReader<'a> { slab.state.read().unwrap(); (*start, *end, guard) } - ChunkState::StartAndFinal { slab, start, end } => { - let guard = slab.state.read().unwrap(); - (*start, *end, guard) - } ChunkState::Continue { slab, end } => { let guard = slab.state.read().unwrap(); (0, *end, guard) } - ChunkState::Final { slab, end } => { + ChunkState::Final { slab, end, .. } => { let guard = slab.state.read().unwrap(); (0, *end, guard) } @@ -1071,11 +910,6 @@ impl CsvValidator { self.transition_table[CsvState::Escape as usize] = [CsvState::QuotedField; 256]; } - fn reset(&mut self) { - self.num_fields_seen = 1; - self.state = CsvState::FieldStart; - } - fn validate_record<'a>(&mut self, iter: &mut impl Iterator) -> Option { for &byte in iter { let next_state = self.transition_table[self.state as usize][byte as usize]; @@ -1106,7 +940,7 @@ impl CsvValidator { #[allow(clippy::too_many_arguments)] fn dispatch_to_parse_csv( has_header: bool, - parse_options: CsvParseOptions, + parse_options: &CsvParseOptions, buffer_source: R, projection_indices: Arc>, fields: Vec, @@ -1115,44 +949,29 @@ fn dispatch_to_parse_csv( csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, - sender: crossbeam_channel::Sender>, - rows_read: Arc, -) where +) -> DaftResult> +where R: std::io::Read, { - let table_results = { - let rdr = ReaderBuilder::new() - .has_headers(has_header) - .delimiter(parse_options.delimiter) - .double_quote(parse_options.double_quote) - .quote(parse_options.quote) - .escape(parse_options.escape_char) - .comment(parse_options.comment) - .flexible(parse_options.allow_variable_columns) - .from_reader(buffer_source); - parse_csv_chunk( - rdr, - projection_indices, - fields, - read_daft_fields, - read_schema, - csv_buffer, - include_columns, - predicate, - ) - }; - match table_results { - Ok(tables) => { - for table in tables { - let table_len = table.len(); - let _ = sender.send(Ok(table)); - rows_read.fetch_add(table_len, Ordering::SeqCst); - } - } - Err(e) => { - let _ = sender.send(Err(e)); - } - } + let rdr = ReaderBuilder::new() + .has_headers(has_header) + .delimiter(parse_options.delimiter) + .double_quote(parse_options.double_quote) + .quote(parse_options.quote) + .escape(parse_options.escape_char) + .comment(parse_options.comment) + .flexible(parse_options.allow_variable_columns) + .from_reader(buffer_source); + parse_csv_chunk( + rdr, + projection_indices, + fields, + read_daft_fields, + read_schema, + csv_buffer, + include_columns, + predicate, + ) } /// Helper function that consumes a CSV reader and turns it into a vector of Daft tables. From 84e74ad6daf25d7959ff0492dfd7c44238944e35 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Tue, 22 Oct 2024 22:56:08 -0700 Subject: [PATCH 25/38] async works baby --- src/daft-csv/src/local.rs | 126 +++++++++++++++++++++++++------------- 1 file changed, 83 insertions(+), 43 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 78b7c2010a..e3aadec00e 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -26,7 +26,7 @@ use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; -use futures::Stream; +use futures::{stream::BoxStream, Stream, StreamExt, TryStreamExt}; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -394,11 +394,11 @@ pub async fn stream_csv_local( chunk_size_rows, n_threads * 2, )); - let tables = consume_slab_iterator( + let stream = consume_slab_iterator( file, buffer_pool, num_fields, - &parse_options, + parse_options, projection_indices, read_daft_fields, read_schema, @@ -411,8 +411,11 @@ pub async fn stream_csv_local( // }); // let flattened_receiver = receiver.into_iter().flat_map(|rx| rx.into_iter()); - let stream = futures::stream::iter(tables.into_iter().map(Ok)); - Ok(stream) + let val = stream.flat_map(|v| { + let value = v.unwrap(); + futures::stream::iter(value.into_iter().map(Ok)) + }); + Ok(val) } /// Helper function that reads up to 1 MiB of the CSV file to estimate stats and/or infer the schema of the file. @@ -586,12 +589,43 @@ where } } +struct ChunkWindowIterator { + chunk_iter: I, +} + +impl ChunkWindowIterator { + fn new(chunk_iter: I) -> Self { + Self { chunk_iter } + } +} + +impl Iterator for ChunkWindowIterator +where + I: Iterator, +{ + type Item = Vec; + fn next(&mut self) -> Option { + let mut chunks = Vec::with_capacity(2); + while let Some(chunk) = self.chunk_iter.next() { + chunks.push(chunk); + if let ChunkState::Final { .. } = chunks.last().unwrap() { + break; + } + } + if chunks.is_empty() { + None + } else { + Some(chunks) + } + } +} + #[allow(clippy::too_many_arguments)] fn consume_slab_iterator( file: std::fs::File, buffer_pool: Arc, num_fields: usize, - parse_options: &CsvParseOptions, + parse_options: CsvParseOptions, projection_indices: Arc>, read_daft_fields: Arc>>, read_schema: Arc, @@ -599,7 +633,7 @@ fn consume_slab_iterator( include_columns: Option>, predicate: Option>, limit: Option, -) -> DaftResult> { +) -> DaftResult>>> { // Create slab pool for file reads. let slabpool = FileSlabPool::new(); let rows_read = Arc::new(AtomicUsize::new(0)); @@ -614,44 +648,50 @@ fn consume_slab_iterator( ); let chunk_iterator = ChunkyIterator::new(slab_iterator, csv_validator); - + let chunk_window_iterator = ChunkWindowIterator::new(chunk_iterator); let has_header = parse_options.has_header; - - let mut chunks = vec![]; - - let mut all_tables = vec![]; - - let mut flush = |chunks: &mut Vec<_>| { - let reader = MultiSliceReader::new(&chunks); - let mut buffer = buffer_pool.get_buffer(); - let tables = dispatch_to_parse_csv( - has_header, - parse_options, - reader, - projection_indices.clone(), - fields.clone(), - read_daft_fields.clone(), - read_schema.clone(), - &mut buffer, - include_columns.clone(), - predicate.clone(), - ) - .unwrap(); - all_tables.extend(tables.into_iter()); - chunks.clear(); - }; - - for chunk in chunk_iterator { - chunks.push(chunk); - match chunks.last().unwrap() { - ChunkState::Final { .. } => { - flush(&mut chunks); + let parse_options = Arc::new(parse_options); + let stream = futures::stream::iter(chunk_window_iterator) + .map(move |w| { + println!("window of size {}", w.len()); + for c in w.iter() { + println!("\t {c}"); } - _ => {} - } - } - flush(&mut chunks); - Ok(all_tables) + let mut buffer = buffer_pool.get_buffer(); + let parse_options = parse_options.clone(); + let projection_indices = projection_indices.clone(); + let fields = fields.clone(); + let read_daft_fields = read_daft_fields.clone(); + let read_schema = read_schema.clone(); + let include_columns = include_columns.clone(); + let predicate = predicate.clone(); + tokio::spawn(async move { + let (tx, rx) = tokio::sync::oneshot::channel(); + rayon::spawn(move || { + let reader = MultiSliceReader::new(&w); + let tables = dispatch_to_parse_csv( + has_header, + &parse_options, + reader, + projection_indices, + fields, + read_daft_fields, + read_schema, + &mut buffer, + include_columns, + predicate, + ) + .unwrap(); + println!("generated {} tables", tables.len()); + tx.send(tables).unwrap(); + }); + rx.await + }) + }) + .buffered(10) + .map(|v| v.unwrap().context(super::OneShotRecvSnafu {})) + .map_err(|err| err.into()); + Ok(stream.boxed()) } /// `goto_next_line` and `validate_csv_record` are two helper function that determines what chunk of From 554744a301008314bd8a4a0ffe80b2fc62f0bfd8 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Tue, 22 Oct 2024 23:01:44 -0700 Subject: [PATCH 26/38] reset validation state --- src/daft-csv/src/local.rs | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index e3aadec00e..2cf1d1c92c 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -653,10 +653,10 @@ fn consume_slab_iterator( let parse_options = Arc::new(parse_options); let stream = futures::stream::iter(chunk_window_iterator) .map(move |w| { - println!("window of size {}", w.len()); - for c in w.iter() { - println!("\t {c}"); - } + // println!("window of size {}", w.len()); + // for c in w.iter() { + // println!("\t {c}"); + // } let mut buffer = buffer_pool.get_buffer(); let parse_options = parse_options.clone(); let projection_indices = projection_indices.clone(); @@ -682,7 +682,7 @@ fn consume_slab_iterator( predicate, ) .unwrap(); - println!("generated {} tables", tables.len()); + // println!("generated {} tables", tables.len()); tx.send(tables).unwrap(); }); rx.await @@ -951,6 +951,8 @@ impl CsvValidator { } fn validate_record<'a>(&mut self, iter: &mut impl Iterator) -> Option { + self.state = CsvState::FieldStart; + self.num_fields_seen = 1; for &byte in iter { let next_state = self.transition_table[self.state as usize][byte as usize]; From 105daacfa4af006685f516d998b663a8d7ef7a15 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Tue, 22 Oct 2024 23:23:46 -0700 Subject: [PATCH 27/38] remove box stream --- src/daft-csv/src/local.rs | 30 ++++++++++++++---------------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 2cf1d1c92c..36e7ca53be 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -176,10 +176,11 @@ impl FileSlabPool { let slab = slabs.pop(); match slab { Some(slab) => slab, - None => RwLock::new(FileSlabState { + None => { + RwLock::new(FileSlabState { buffer: unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, valid_bytes: 0, - }), + })}, } }; @@ -394,7 +395,7 @@ pub async fn stream_csv_local( chunk_size_rows, n_threads * 2, )); - let stream = consume_slab_iterator( + consume_slab_iterator( file, buffer_pool, num_fields, @@ -406,16 +407,7 @@ pub async fn stream_csv_local( include_columns, predicate, limit, - // sender, - )?; - // }); - - // let flattened_receiver = receiver.into_iter().flat_map(|rx| rx.into_iter()); - let val = stream.flat_map(|v| { - let value = v.unwrap(); - futures::stream::iter(value.into_iter().map(Ok)) - }); - Ok(val) + ) } /// Helper function that reads up to 1 MiB of the CSV file to estimate stats and/or infer the schema of the file. @@ -633,7 +625,7 @@ fn consume_slab_iterator( include_columns: Option>, predicate: Option>, limit: Option, -) -> DaftResult>>> { +) -> DaftResult> + Send> { // Create slab pool for file reads. let slabpool = FileSlabPool::new(); let rows_read = Arc::new(AtomicUsize::new(0)); @@ -688,10 +680,16 @@ fn consume_slab_iterator( rx.await }) }) - .buffered(10) + .buffered(32) .map(|v| v.unwrap().context(super::OneShotRecvSnafu {})) .map_err(|err| err.into()); - Ok(stream.boxed()) + + let flattened = stream.flat_map(|v: DaftResult>| { + let value = v.unwrap(); + futures::stream::iter(value.into_iter().map(Ok)) + }); + + Ok(flattened) } /// `goto_next_line` and `validate_csv_record` are two helper function that determines what chunk of From 2f04c6d9896bf68414af650baa6b1b4f0999ec86 Mon Sep 17 00:00:00 2001 From: Sammy Sidhu Date: Tue, 22 Oct 2024 23:38:45 -0700 Subject: [PATCH 28/38] error clean up --- src/daft-csv/src/local.rs | 37 +++++++++++++++---------------------- 1 file changed, 15 insertions(+), 22 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 36e7ca53be..54783f1a1d 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -36,7 +36,7 @@ use snafu::ResultExt; use crate::{ metadata::read_csv_schema_single, read::{fields_to_projection_indices, tables_concat}, - ArrowSnafu, CsvConvertOptions, CsvParseOptions, CsvReadOptions, + ArrowSnafu, CsvConvertOptions, CsvParseOptions, CsvReadOptions, JoinSnafu, }; // Our local CSV reader takes the following approach to reading CSV files: @@ -406,7 +406,7 @@ pub async fn stream_csv_local( schema.fields, include_columns, predicate, - limit, + n_threads, ) } @@ -600,7 +600,7 @@ where let mut chunks = Vec::with_capacity(2); while let Some(chunk) = self.chunk_iter.next() { chunks.push(chunk); - if let ChunkState::Final { .. } = chunks.last().unwrap() { + if let ChunkState::Final { .. } = chunks.last().expect("We just pushed a chunk") { break; } } @@ -624,11 +624,10 @@ fn consume_slab_iterator( fields: Vec, include_columns: Option>, predicate: Option>, - limit: Option, + n_threads: usize, ) -> DaftResult> + Send> { // Create slab pool for file reads. let slabpool = FileSlabPool::new(); - let rows_read = Arc::new(AtomicUsize::new(0)); let slab_iterator = SlabIterator::new(file, slabpool); let csv_validator = CsvValidator::new( @@ -643,12 +642,9 @@ fn consume_slab_iterator( let chunk_window_iterator = ChunkWindowIterator::new(chunk_iterator); let has_header = parse_options.has_header; let parse_options = Arc::new(parse_options); - let stream = futures::stream::iter(chunk_window_iterator) - .map(move |w| { - // println!("window of size {}", w.len()); - // for c in w.iter() { - // println!("\t {c}"); - // } + let stream = futures::stream::iter(chunk_window_iterator.enumerate()) + .map(move |(i, w)| { + let has_header = has_header && (i == 0); let mut buffer = buffer_pool.get_buffer(); let parse_options = parse_options.clone(); let projection_indices = projection_indices.clone(); @@ -672,22 +668,19 @@ fn consume_slab_iterator( &mut buffer, include_columns, predicate, - ) - .unwrap(); - // println!("generated {} tables", tables.len()); - tx.send(tables).unwrap(); + ); + tx.send(tables).expect("OneShot Channel should still be open"); }); rx.await }) }) - .buffered(32) - .map(|v| v.unwrap().context(super::OneShotRecvSnafu {})) + .buffered(n_threads) + .map(|v| v.context(JoinSnafu {})?.context(super::OneShotRecvSnafu {})? ) .map_err(|err| err.into()); - - let flattened = stream.flat_map(|v: DaftResult>| { - let value = v.unwrap(); - futures::stream::iter(value.into_iter().map(Ok)) - }); + let flattened = stream.map(|result: DaftResult>| { + let tables = result?; + DaftResult::Ok(futures::stream::iter(tables.into_iter().map(Ok))) + }).try_flatten(); Ok(flattened) } From efb91802e7b182aad9cd783bee6e6642ea73fd04 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 23 Oct 2024 11:57:53 -0700 Subject: [PATCH 29/38] Apply local limit; apply max chunks in flight; cleanup --- src/arrow2/src/io/csv/read_async/reader.rs | 4 + src/daft-csv/src/local.rs | 214 ++++++++++++--------- 2 files changed, 122 insertions(+), 96 deletions(-) diff --git a/src/arrow2/src/io/csv/read_async/reader.rs b/src/arrow2/src/io/csv/read_async/reader.rs index a95b88e5ad..7fdb0eace9 100644 --- a/src/arrow2/src/io/csv/read_async/reader.rs +++ b/src/arrow2/src/io/csv/read_async/reader.rs @@ -43,6 +43,7 @@ where pub fn local_read_rows( reader: &mut read::Reader, rows: &mut [read::ByteRecord], + limit: Option, ) -> Result<(usize, bool)> where R: std::io::Read, @@ -50,6 +51,9 @@ where let mut row_number = 0; let mut has_more = true; for row in rows.iter_mut() { + if matches!(limit, Some(limit) if row_number >= limit) { + break; + } has_more = reader .read_byte_record(row) .map_err(|e| Error::External(format!(" at line {}", row_number), Box::new(e)))?; diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 54783f1a1d..560297a893 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -2,11 +2,7 @@ use core::str; use std::{ io::Read, num::NonZeroUsize, - path::Display, - sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, Mutex, RwLock, Weak, - }, + sync::{Arc, Mutex, RwLock, Weak}, }; use arrow2::{ @@ -16,8 +12,7 @@ use arrow2::{ read_async::local_read_rows, }, }; -use common_error::{DaftError, DaftResult}; -use crossbeam_channel::Sender; +use common_error::DaftResult; use daft_core::{ prelude::{Schema, Series}, utils::arrow::cast_array_for_daft_if_needed, @@ -26,7 +21,7 @@ use daft_decoding::deserialize::deserialize_column; use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; -use futures::{stream::BoxStream, Stream, StreamExt, TryStreamExt}; +use futures::{Stream, StreamExt, TryStreamExt}; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -63,9 +58,82 @@ use crate::{ // │ | ┌───┐ ┌───┐ ┌────┐ ┬--─┐ ┌───┐ ┌───┐ ┌───┐ ┌───┐ // │ ─┼───►│ │ │ │ ──────► │ ┬┘┌─┘ ┬─┘ ───────► │ │ │ │ ──────────► │ │ │ │ // │ CSV File │ └───┘ └───┘ └───┴ └───┘ └───┘ └───┘ └───┘ └───┘ -// │ │ Chain of buffers Adjusted chunks Vectors of ByteRecords Stream of Daft tables +// │ │ Chain of slabs Adjusted chunks Vectors of ByteRecords Stream of Daft tables // │ │ // └──────────┘ +// +// +// The above flow is concretely implemented via a series of iterators: +// 1. SlabIterator provides an iterator of slabs populated with file data. +// 2. ChunkyIterator takes the SlabIterator and provides an iterator of `Start`, `Continue`, and +// `Final` chunks. Each chunk contains a reference to a file slab, and the start-end range of bytes of the +// slab that it is valid for. +// 3. ChunkWindowIterator takes a ChunkyIterator, and creates windows of Start-Continue*-Final adjusted chunks. +// An adjusted chunk (described in greater detail below), always starts with `ChunkState::Start`, +// ends with `ChunkState::Final`, and has any number of `ChunkState::Continue` chunks in between. +// +// We take items off the ChunkWindowIterator, and pass them to a CSV decoder to be turned into Daft tables in parallel. +// +// +// "Adjusted chunks" were mentioned a few times above and refer to chunks of complete CSV records. This is +// needed for us to process chunks in parallel without having to stitch together records that are split across +// multiple chunks. Here's a way to think about adjusted chunks: +// +// Given a starting position, we use our chunk size to compute a preliminary start and stop +// position. For example, we can visualize all preliminary chunks in a file as follows. +// +// Chunk 1 Chunk 2 Chunk 3 Chunk N +// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ +// │ │ │\n │ │ \n │ │ \n │ +// │ │ │ │ │ │ │ │ +// │ │ │ \n │ │ │ │ │ +// │ \n │ │ │ │ \n │ │ │ +// │ │ │ │ │ │ ... │ \n │ +// │ │ │ \n │ │ │ │ │ +// │ \n │ │ │ │ │ │ │ +// │ │ │ │ │ \n │ │ \n │ +// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +// +// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary +// chunk boundaries. So we adjust each preliminary chunk as follows: +// - Find the first record terminator from the chunk's start. This is the new starting position. +// - Find the first record terminator from the chunk's end. This is the new ending position. +// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. +// +// For example: +// +// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N +// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ +// │ \n││ \n│ │ \n│ \n │ │ +// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ +// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ +// │ \n │ │ │ │ \n │ │ │ +// │ │ │ │ │ │ ... │ \n │ +// │ │ │ \n │ │ │ │ │ +// │ \n │ │ │ │ │ │ │ +// │ │ │ │ │ \n │ │ \n │ +// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +// +// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do +// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can +// now happen in parallel. +// +// This is the same method as described in: +// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). +// +// Another observation is that seeing a pure \n character is not necessarily indicative of a record +// terminator. We need to consider whether the \n character was seen within a quoted field, since the +// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following +// algorithm: +// 1. Find a \n character. +// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse +// as valid CSV, and does it produce the same number of fields as our schema. +// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. +// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next +// \n character and go back to 2. +// 2c. If we hit the end of a chunk without coming to a decision on whether we found a valid CSV record, +// we simply move on to the next chunk of bytes and try to find a valid CSV record there. This is a +// simplification that makes the implementation a lot easier to maintain. /// A pool of ByteRecord slabs. Used for deserializing CSV. #[derive(Debug)] @@ -176,11 +244,10 @@ impl FileSlabPool { let slab = slabs.pop(); match slab { Some(slab) => slab, - None => { - RwLock::new(FileSlabState { + None => RwLock::new(FileSlabState { buffer: unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, valid_bytes: 0, - })}, + }), } }; @@ -319,7 +386,6 @@ pub async fn stream_csv_local( ) -> DaftResult> + Send> { let uri = uri.trim_start_matches("file://"); let file = std::fs::File::open(uri)?; - let file_len = file.metadata()?.len(); // Process the CSV convert options. let predicate = convert_options @@ -379,15 +445,6 @@ pub async fn stream_csv_local( .unwrap_or(DEFAULT_CHUNK_SIZE); let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; - // We produce `file_len / SLABSIZE` number of file slabs, each of which might be split into two, so the maximum number - // of tasks we might spawn is 2 times this number. - // let (sender, receiver) = crossbeam_channel::bounded( - // max_chunks_in_flight - // .unwrap_or_else(|| 2 * (file_len as f64 / SLABSIZE as f64).ceil() as usize), - // ); - - // Consume the CSV file asynchronously. - // rayon::spawn(move || { // TODO(desmond): We might consider creating per-process buffer pools and slab pools. let buffer_pool = Arc::new(CsvBufferPool::new( record_buffer_size, @@ -406,7 +463,8 @@ pub async fn stream_csv_local( schema.fields, include_columns, predicate, - n_threads, + limit, + max_chunks_in_flight.unwrap_or(n_threads), ) } @@ -567,7 +625,7 @@ where None => { if let Some((slab, valid_bytes)) = self.slab_iter.next() { Some(ChunkState::Start { - slab: slab, + slab, start: 0, end: valid_bytes, }) @@ -576,7 +634,7 @@ where } } }; - self.last_chunk = curr_chunk.clone(); + self.last_chunk.clone_from(&curr_chunk); curr_chunk } } @@ -598,7 +656,7 @@ where type Item = Vec; fn next(&mut self) -> Option { let mut chunks = Vec::with_capacity(2); - while let Some(chunk) = self.chunk_iter.next() { + for chunk in self.chunk_iter.by_ref() { chunks.push(chunk); if let ChunkState::Final { .. } = chunks.last().expect("We just pushed a chunk") { break; @@ -624,6 +682,7 @@ fn consume_slab_iterator( fields: Vec, include_columns: Option>, predicate: Option>, + limit: Option, n_threads: usize, ) -> DaftResult> + Send> { // Create slab pool for file reads. @@ -668,79 +727,29 @@ fn consume_slab_iterator( &mut buffer, include_columns, predicate, + limit, ); - tx.send(tables).expect("OneShot Channel should still be open"); + tx.send(tables) + .expect("OneShot Channel should still be open"); }); rx.await }) }) .buffered(n_threads) - .map(|v| v.context(JoinSnafu {})?.context(super::OneShotRecvSnafu {})? ) - .map_err(|err| err.into()); - let flattened = stream.map(|result: DaftResult>| { - let tables = result?; - DaftResult::Ok(futures::stream::iter(tables.into_iter().map(Ok))) - }).try_flatten(); - + .map(|v| { + v.context(JoinSnafu {})? + .context(super::OneShotRecvSnafu {})? + }); + let flattened = stream + .map(|result: DaftResult>| { + let tables = result?; + DaftResult::Ok(futures::stream::iter(tables.into_iter().map(Ok))) + }) + .try_flatten(); + Ok(flattened) } -/// `goto_next_line` and `validate_csv_record` are two helper function that determines what chunk of -/// data to parse given a starting position within the file, and the desired initial chunk size. -/// -/// Given a starting position, we use our chunk size to compute a preliminary start and stop -/// position. For example, we can visualize all preliminary chunks in a file as follows. -/// -/// Chunk 1 Chunk 2 Chunk 3 Chunk N -/// ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ -/// │ │ │\n │ │ \n │ │ \n │ -/// │ │ │ │ │ │ │ │ -/// │ │ │ \n │ │ │ │ │ -/// │ \n │ │ │ │ \n │ │ │ -/// │ │ │ │ │ │ ... │ \n │ -/// │ │ │ \n │ │ │ │ │ -/// │ \n │ │ │ │ │ │ │ -/// │ │ │ │ │ \n │ │ \n │ -/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ -/// -/// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary -/// chunk boundaries. So we adjust each preliminary chunk as follows: -/// - Find the first record terminator from the chunk's start. This is the new starting position. -/// - Find the first record terminator from the chunk's end. This is the new ending position. -/// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. -/// -/// For example: -/// -/// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N -/// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ -/// │ \n││ \n│ │ \n│ \n │ │ -/// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ -/// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ -/// │ \n │ │ │ │ \n │ │ │ -/// │ │ │ │ │ │ ... │ \n │ -/// │ │ │ \n │ │ │ │ │ -/// │ \n │ │ │ │ │ │ │ -/// │ │ │ │ │ \n │ │ \n │ -/// └──────────┘ └──────────┘ └──────────┘ └──────────┘ -/// -/// Using this method, we now have adjusted chunks that are aligned with record boundaries, that do -/// not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can -/// now happen in parallel. -/// -/// This is the same method as described in: -/// Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). -/// -/// Another observation is that seeing a pure \n character is not necessarily indicative of a record -/// terminator. We need to consider whether the \n character was seen within a quoted field, since the -/// string "some text \n some text" is a valid CSV string field. To do this, we carry out the following -/// algorithm: -/// 1. Find a \n character. -/// 2. Check if the CSV string immediately following this \n character is valid, i.e. does it parse -/// as valid CSV, and does it produce the same number of fields as our schema. -/// 2a. If there is a valid record at this point, then we assume that the \n we saw was a valid terminator. -/// 2b. If the record at this point is invalid, then this was likely a \n in a quoted field. Find the next -/// \n character and go back to 2. - #[derive(Debug, Clone)] enum ChunkState { Start { @@ -762,17 +771,17 @@ enum ChunkState { impl std::fmt::Display for ChunkState { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { - ChunkState::Start { + Self::Start { slab: _, start, end, } => write!(f, "ChunkState::Start-{start}-{end}"), - ChunkState::Final { + Self::Final { slab: _, end, valid_bytes, } => write!(f, "ChunkState::Final-{end}-{valid_bytes}"), - ChunkState::Continue { slab: _, end } => write!(f, "ChunkState::Continue-{end}"), + Self::Continue { slab: _, end } => write!(f, "ChunkState::Continue-{end}"), } } } @@ -982,6 +991,7 @@ fn dispatch_to_parse_csv( csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, + limit: Option, ) -> DaftResult> where R: std::io::Read, @@ -995,6 +1005,8 @@ where .comment(parse_options.comment) .flexible(parse_options.allow_variable_columns) .from_reader(buffer_source); + // The header should not count towards the limit. + let limit = limit.map(|limit| limit + (has_header as usize)); parse_csv_chunk( rdr, projection_indices, @@ -1004,6 +1016,7 @@ where csv_buffer, include_columns, predicate, + limit, ) } @@ -1018,14 +1031,17 @@ fn parse_csv_chunk( csv_buffer: &mut CsvBuffer, include_columns: Option>, predicate: Option>, + limit: Option, ) -> DaftResult> where R: std::io::Read, { let mut tables = vec![]; + let mut local_limit = limit; loop { - let (rows_read, has_more) = local_read_rows(&mut reader, csv_buffer.buffer.as_mut_slice()) - .context(ArrowSnafu {})?; + let (rows_read, has_more) = + local_read_rows(&mut reader, csv_buffer.buffer.as_mut_slice(), local_limit) + .context(ArrowSnafu {})?; let chunk = projection_indices .par_iter() .enumerate() @@ -1055,7 +1071,13 @@ where table }; tables.push(table); - + // Stop reading once we hit the local limit. + if let Some(local_limit) = &mut local_limit { + *local_limit -= num_rows; + if *local_limit == 0 { + break; + } + } // The number of record might exceed the number of byte records we've allocated. // Retry until all byte records in this chunk are read. if !has_more { From 05591429801e39ed49184954d9b8e7a884764884 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 23 Oct 2024 12:35:30 -0700 Subject: [PATCH 30/38] Address remaining comments --- Cargo.lock | 1 + src/daft-csv/Cargo.toml | 1 + src/daft-csv/src/lib.rs | 2 -- 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ade5517d93..514becbb94 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1772,6 +1772,7 @@ dependencies = [ "daft-table", "futures", "memchr", + "parking_lot", "pyo3", "rayon", "rstest", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index ca74b9c2ec..0795719556 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -14,6 +14,7 @@ daft-io = {path = "../daft-io", default-features = false} daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} memchr = "2.7.2" +parking_lot = "0.12.3" pyo3 = {workspace = true, optional = true} rayon = {workspace = true} serde = {workspace = true} diff --git a/src/daft-csv/src/lib.rs b/src/daft-csv/src/lib.rs index ce1a3615a9..4a738ab2a8 100644 --- a/src/daft-csv/src/lib.rs +++ b/src/daft-csv/src/lib.rs @@ -27,8 +27,6 @@ pub enum Error { #[snafu(display("{source}"))] IOError { source: daft_io::Error }, #[snafu(display("{source}"))] - StdIOError { source: std::io::Error }, - #[snafu(display("{source}"))] CSVError { source: csv_async::Error }, #[snafu(display("Invalid char: {}", val))] WrongChar { From 43c8a6a8736f9ae0d1f4c0e5f4d25aceb7569eb4 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 23 Oct 2024 12:42:23 -0700 Subject: [PATCH 31/38] Errr didn't git add some things --- src/daft-csv/src/local.rs | 139 ++++++++++++++++++++++---------------- 1 file changed, 80 insertions(+), 59 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 560297a893..d33369c2f7 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -2,7 +2,8 @@ use core::str; use std::{ io::Read, num::NonZeroUsize, - sync::{Arc, Mutex, RwLock, Weak}, + ops::{Deref, DerefMut}, + sync::{Arc, Weak}, }; use arrow2::{ @@ -22,6 +23,7 @@ use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; use futures::{Stream, StreamExt, TryStreamExt}; +use parking_lot::{Mutex, RwLock}; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -135,55 +137,71 @@ use crate::{ // we simply move on to the next chunk of bytes and try to find a valid CSV record there. This is a // simplification that makes the implementation a lot easier to maintain. +#[derive(Clone, Debug, Default)] +struct CsvSlab(Vec); + +impl CsvSlab { + fn new(record_size: usize, num_fields: usize, num_rows: usize) -> Self { + Self(vec![ + read::ByteRecord::with_capacity(record_size, num_fields); + num_rows + ]) + } +} + +impl Deref for CsvSlab { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl DerefMut for CsvSlab { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.0 + } +} + /// A pool of ByteRecord slabs. Used for deserializing CSV. #[derive(Debug)] struct CsvBufferPool { - buffers: Mutex>>, - buffer_size: usize, - record_buffer_size: usize, + buffers: Mutex>, + record_size: usize, num_fields: usize, + num_rows: usize, } /// A slab of ByteRecords. Used for deserializing CSV. struct CsvBuffer { - buffer: Vec, + buffer: CsvSlab, pool: Weak, } impl CsvBufferPool { pub fn new( - record_buffer_size: usize, + record_size: usize, num_fields: usize, - chunk_size_rows: usize, + num_rows: usize, initial_pool_size: usize, ) -> Self { - let chunk_buffers = vec![ - vec![ - read::ByteRecord::with_capacity(record_buffer_size, num_fields); - chunk_size_rows - ]; - initial_pool_size - ]; + let chunk_buffers = + vec![CsvSlab::new(record_size, num_fields, num_rows); initial_pool_size]; Self { buffers: Mutex::new(chunk_buffers), - buffer_size: chunk_size_rows, - record_buffer_size, + record_size, num_fields, + num_rows, } } pub fn get_buffer(self: &Arc) -> CsvBuffer { let buffer = { - let mut buffers = self.buffers.lock().unwrap(); + let mut buffers = self.buffers.lock(); let buffer = buffers.pop(); match buffer { Some(buffer) => buffer, - None => { - vec![ - read::ByteRecord::with_capacity(self.record_buffer_size, self.num_fields); - self.buffer_size - ] - } + None => CsvSlab::new(self.record_size, self.num_fields, self.num_rows), } }; @@ -193,10 +211,9 @@ impl CsvBufferPool { } } - fn return_buffer(&self, buffer: Vec) { - if let Ok(mut buffers) = self.buffers.lock() { - buffers.push(buffer); - } + fn return_buffer(&self, buffer: CsvSlab) { + let mut buffers = self.buffers.lock(); + buffers.push(buffer); } } @@ -226,12 +243,7 @@ impl FileSlabPool { // We get uninitialized buffers because we will always populate the buffers with a file read before use. .map(|_| Box::new_uninit_slice(SLABSIZE)) .map(|x| unsafe { x.assume_init() }) - .map(|buffer| { - RwLock::new(FileSlabState { - buffer, - valid_bytes: 0, - }) - }) + .map(|buffer| RwLock::new(FileSlabState::new(buffer, 0))) .collect(); Arc::new(Self { slabs: Mutex::new(slabs), @@ -240,14 +252,14 @@ impl FileSlabPool { fn get_slab(self: &Arc) -> Arc { let slab = { - let mut slabs = self.slabs.lock().unwrap(); + let mut slabs = self.slabs.lock(); let slab = slabs.pop(); match slab { Some(slab) => slab, - None => RwLock::new(FileSlabState { - buffer: unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, - valid_bytes: 0, - }), + None => RwLock::new(FileSlabState::new( + unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, + 0, + )), } }; @@ -258,9 +270,8 @@ impl FileSlabPool { } fn return_slab(&self, slab: RwLock) { - if let Ok(mut slabs) = self.slabs.lock() { - slabs.push(slab); - } + let mut slabs = self.slabs.lock(); + slabs.push(slab); } } @@ -272,9 +283,11 @@ struct FileSlab { } impl FileSlab { - fn find_newline(&self, offset: usize) -> Option { - let guard = self.state.read().unwrap(); - guard.find_newline(offset) + /// Given an offset into a FileSlab, finds the first \n char found in the FileSlabState's buffer, + /// then the returns the position relative to the given offset. + fn find_first_newline_from(&self, offset: usize) -> Option { + let guard = self.state.read(); + guard.find_first_newline_from(offset) } } @@ -295,8 +308,15 @@ struct FileSlabState { } impl FileSlabState { + fn new(buffer: Box<[u8]>, valid_bytes: usize) -> Self { + Self { + buffer, + valid_bytes, + } + } + /// Helper function that find the first \n char in the file slab state's buffer starting from `offset.` - fn find_newline(&self, offset: usize) -> Option { + fn find_first_newline_from(&self, offset: usize) -> Option { newline_position(&self.buffer[offset..self.valid_bytes]) } @@ -535,7 +555,7 @@ impl Iterator for SlabIterator { fn next(&mut self) -> Option { let slab = self.slabpool.get_slab(); let bytes_read = { - let mut writer = slab.state.write().unwrap(); + let mut writer = slab.state.write(); let bytes_read = self.file.read(&mut writer.buffer).unwrap(); if bytes_read == 0 { return None; @@ -580,11 +600,11 @@ where let mut curr_pos = 0; let mut chunk_state: Option = None; while chunk_state.is_none() - && let Some(pos) = slab.find_newline(curr_pos) + && let Some(pos) = slab.find_first_newline_from(curr_pos) && curr_pos < valid_bytes { let offset = curr_pos + pos; - let guard = slab.state.read().unwrap(); + let guard = slab.state.read(); chunk_state = match guard.validate_record(&mut self.validator, offset + 1) { Some(true) => Some(ChunkState::Final { slab: slab.clone(), @@ -716,7 +736,7 @@ fn consume_slab_iterator( let (tx, rx) = tokio::sync::oneshot::channel(); rayon::spawn(move || { let reader = MultiSliceReader::new(&w); - let tables = dispatch_to_parse_csv( + let tables = collect_tables( has_header, &parse_options, reader, @@ -785,7 +805,7 @@ impl std::fmt::Display for ChunkState { } } } -/// A helper struct that implements `std::io::Read` over a slice of ChunkStates' buffers. +/// A helper struct that implements `std::io::Read` over a slice of ChunkStates. struct MultiSliceReader<'a> { states: &'a [ChunkState], curr_read_idx: usize, @@ -810,16 +830,15 @@ impl<'a> Read for MultiSliceReader<'a> { let state = &self.states[self.curr_read_idx]; let (start, end, guard) = match state { ChunkState::Start { slab, start, end } => { - let guard: std::sync::RwLockReadGuard<'_, FileSlabState> = - slab.state.read().unwrap(); + let guard = slab.state.read(); (*start, *end, guard) } ChunkState::Continue { slab, end } => { - let guard = slab.state.read().unwrap(); + let guard = slab.state.read(); (0, *end, guard) } ChunkState::Final { slab, end, .. } => { - let guard = slab.state.read().unwrap(); + let guard = slab.state.read(); (0, *end, guard) } }; @@ -951,8 +970,10 @@ impl CsvValidator { } fn validate_record<'a>(&mut self, iter: &mut impl Iterator) -> Option { + // Reset state machine for each new validation attempt. self.state = CsvState::FieldStart; self.num_fields_seen = 1; + // Start running the state machine against each byte. for &byte in iter { let next_state = self.transition_table[self.state as usize][byte as usize]; @@ -977,13 +998,13 @@ impl CsvValidator { } } -/// Helper function that takes in a BufferSource, calls parse_csv() to extract table values from -/// the buffer source, then streams the results to `sender`. +/// Helper function that takes in a source of bytes, calls parse_csv() to extract table values from +/// the buffer source, then returns the vector of Daft tables. #[allow(clippy::too_many_arguments)] -fn dispatch_to_parse_csv( +fn collect_tables( has_header: bool, parse_options: &CsvParseOptions, - buffer_source: R, + byte_reader: R, projection_indices: Arc>, fields: Vec, read_daft_fields: Arc>>, @@ -1004,7 +1025,7 @@ where .escape(parse_options.escape_char) .comment(parse_options.comment) .flexible(parse_options.allow_variable_columns) - .from_reader(buffer_source); + .from_reader(byte_reader); // The header should not count towards the limit. let limit = limit.map(|limit| limit + (has_header as usize)); parse_csv_chunk( From eac91b2699db614ce4d1f27327685b1990fa59c9 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 23 Oct 2024 13:10:40 -0700 Subject: [PATCH 32/38] Cleanup --- src/daft-csv/src/local.rs | 186 +++++++++++++++++++++----------------- 1 file changed, 101 insertions(+), 85 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index d33369c2f7..4de486b67a 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -47,21 +47,21 @@ use crate::{ // // Slab Pool CSV Buffer Pool // ┌────────────────────┐ ┌────────────────────┐ -// │ 4MB Chunks │ │ CSV Buffers │ -// │┌───┐┌───┐┌───┐ │ │┌───┐┌───┐┌───┐ │ -// ││ ││ ││ │ ... │ ││ ││ ││ │ ... │ -// │└─┬─┘└─┬─┘└───┘ │ │└─┬─┘└─┬─┘└───┘ │ +// | 4MB Chunks | | CSV Buffers | +// |┌───┐┌───┐┌───┐ | |┌───┐┌───┐┌───┐ | +// || || || | ... | || || || | ... | +// |└─┬─┘└─┬─┘└───┘ | |└─┬─┘└─┬─┘└───┘ | // └──┼────┼────────────┘ └──┼────┼────────────┘ -// │ │ │ │ -// ───────┐ │ │ │ │ -// /│ │ │ │ │ │ -// /─┘ │ │ │ │ │ -// │ │ ▼ ▼ ▼ ▼ -// │ | ┌───┐ ┌───┐ ┌────┐ ┬--─┐ ┌───┐ ┌───┐ ┌───┐ ┌───┐ -// │ ─┼───►│ │ │ │ ──────► │ ┬┘┌─┘ ┬─┘ ───────► │ │ │ │ ──────────► │ │ │ │ -// │ CSV File │ └───┘ └───┘ └───┴ └───┘ └───┘ └───┘ └───┘ └───┘ -// │ │ Chain of slabs Adjusted chunks Vectors of ByteRecords Stream of Daft tables -// │ │ +// | | | | +// ───────┐ | | | | +// /| | | | | | +// /─┘ | | | | | +// | | ▼ ▼ ▼ ▼ +// | | ┌───┐ ┌───┐ ┌────┐ ┬--─┐ ┌───┐ ┌───┐ ┌───┐ ┌───┐ +// | ─┼───►| | | | ──────► | ┬┘┌─┘ ┬─┘ ───────► | | | | ──────────► | | | | +// | CSV File | └───┘ └───┘ └───┴ └───┘ └───┘ └───┘ └───┘ └───┘ +// | | Chain of slabs Adjusted chunks Vectors of ByteRecords Stream of Daft tables +// | | // └──────────┘ // // @@ -77,50 +77,52 @@ use crate::{ // We take items off the ChunkWindowIterator, and pass them to a CSV decoder to be turned into Daft tables in parallel. // // -// "Adjusted chunks" were mentioned a few times above and refer to chunks of complete CSV records. This is -// needed for us to process chunks in parallel without having to stitch together records that are split across -// multiple chunks. Here's a way to think about adjusted chunks: +// What we call "Chunk Windows" are also known as "Adjusted chunks" in the literature, and refers to contiguous +// bytes that consist of complete CSV records. This contiguous bytes are typically made up of one or more file slabs. +// This is abstraction is needed for us to process file slabs in parallel without having to stitch together records that +// are split across multiple slabs. Here's a way to think about adjusted chunks: // -// Given a starting position, we use our chunk size to compute a preliminary start and stop -// position. For example, we can visualize all preliminary chunks in a file as follows. +// Given a starting position, we use our slab size to compute a preliminary start and stop +// position. For example, we can visualize all slabs in a file as follows. // -// Chunk 1 Chunk 2 Chunk 3 Chunk N +// Slab 1 Slab 2 Slab 3 Slab N // ┌──────────┐ ┌──────────┐ ┌──────────┐ ┌──────────┐ -// │ │ │\n │ │ \n │ │ \n │ -// │ │ │ │ │ │ │ │ -// │ │ │ \n │ │ │ │ │ -// │ \n │ │ │ │ \n │ │ │ -// │ │ │ │ │ │ ... │ \n │ -// │ │ │ \n │ │ │ │ │ -// │ \n │ │ │ │ │ │ │ -// │ │ │ │ │ \n │ │ \n │ +// | | |\n | | | \n | \n | +// | | | | | | | | +// | | | | | | | | +// | \n | | | | | | | +// | | | | | | ... | \n | +// | | | | | | | | +// | \n | | | | | | | +// | | | | | | | \n | // └──────────┘ └──────────┘ └──────────┘ └──────────┘ // -// However, record boundaries (i.e. the \n terminators) do not align nicely with these preliminary -// chunk boundaries. So we adjust each preliminary chunk as follows: -// - Find the first record terminator from the chunk's start. This is the new starting position. -// - Find the first record terminator from the chunk's end. This is the new ending position. -// - If a given preliminary chunk doesn't contain a record terminator, the adjusted chunk is empty. +// However, record boundaries (i.e. the \n terminators) do not align nicely with the boundaries of slabs. +// So we create "adjusted chunks" from slabs as follows: +// 1. Find the first record terminator from a slabs's start. This is the starting position of the adjusted chunk. +// 2. Find the first record terminator from the next slabs's start. This is the ending positiono of the adjusted chunk. +// 3. If a given slab doesn't contain a record terminator (that we can find), then we add the current slab wholly into +// the adjusted chunk, then repeat step 2. with the next slab. // // For example: // -// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk 3 Adj. Chunk N -// ┌──────────────────┐┌─────────────────┐ ┌────────┐ ┌─┐ -// │ \n││ \n│ │ \n│ \n │ │ -// │ ┌───────┘│ ┌──────────┘ │ ┌─────┘ │ │ -// │ │ ┌───┘ \n │ ┌───────┘ │ ┌────────┘ │ -// │ \n │ │ │ │ \n │ │ │ -// │ │ │ │ │ │ ... │ \n │ -// │ │ │ \n │ │ │ │ │ -// │ \n │ │ │ │ │ │ │ -// │ │ │ │ │ \n │ │ \n │ -// └──────────┘ └──────────┘ └──────────┘ └──────────┘ +// Adjusted Chunk 1 Adj. Chunk 2 Adj. Chunk N +// ┌──────────────────┐┌─────────────────────────────┐ ┌─┐ +// | \n|| \n| \n | | +// | ┌───────┘| ┌─────┘ | | +// | | ┌───┘ | ┌────────┘ | +// | \n | | | | | +// | | | | ... | \n | +// | | | | | | +// | \n | | | | | +// | | | | | \n | +// └──────────┘ └───────────────────────────┘ └──────────┘ // // Using this method, we now have adjusted chunks that are aligned with record boundaries, that do // not overlap, and that fully cover every byte in the CSV file. Parsing each adjusted chunk can // now happen in parallel. // -// This is the same method as described in: +// This is similar to the method described in: // Ge, Chang et al. “Speculative Distributed CSV Data Parsing for Big Data Analytics.” Proceedings of the 2019 International Conference on Management of Data (2019). // // Another observation is that seeing a pure \n character is not necessarily indicative of a record @@ -472,7 +474,7 @@ pub async fn stream_csv_local( chunk_size_rows, n_threads * 2, )); - consume_slab_iterator( + stream_csv_as_tables( file, buffer_pool, num_fields, @@ -530,7 +532,7 @@ async fn get_schema_and_estimators( )) } -/// A helper iterator that takes in a File and FileSlabPool and produces an iterator of FileSlabs +/// An iterator of FileSlabs that takes in a File and FileSlabPool and yields FileSlabs /// over the given file. struct SlabIterator { file: std::fs::File, @@ -569,6 +571,49 @@ impl Iterator for SlabIterator { } } +/// ChunkStates are a wrapper over slabs that dictate the position of a slab in a chunk window, +/// and which bytes of the slab should be used for parsing CSV records. +#[derive(Debug, Clone)] +enum ChunkState { + // Represents the first chunk in a chunk window. + Start { + slab: Arc, + start: usize, + end: usize, + }, + // Represents any number of chunks between the Start and Final chunk in a chunk window. + Continue { + slab: Arc, + end: usize, + }, + // Represents the last chunk in a chunk window. + Final { + slab: Arc, + end: usize, + valid_bytes: usize, + }, +} + +impl std::fmt::Display for ChunkState { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::Start { + slab: _, + start, + end, + } => write!(f, "ChunkState::Start-{start}-{end}"), + Self::Final { + slab: _, + end, + valid_bytes, + } => write!(f, "ChunkState::Final-{end}-{valid_bytes}"), + Self::Continue { slab: _, end } => write!(f, "ChunkState::Continue-{end}"), + } + } +} + +/// An iterator of ChunkStates that takes in a SlabIterator and yields Start, Continue, and Final +/// ChunkStates over the given slabs. struct ChunkyIterator { slab_iter: I, last_chunk: Option, @@ -659,6 +704,8 @@ where } } +/// An iterator of ChunkWindows that takes in aa ChunkyIterator and yields vectors of ChunkStates +/// that contain Start-Continue*-Final chunks that are valid for CSV parsing. struct ChunkWindowIterator { chunk_iter: I, } @@ -691,7 +738,7 @@ where } #[allow(clippy::too_many_arguments)] -fn consume_slab_iterator( +fn stream_csv_as_tables( file: std::fs::File, buffer_pool: Arc, num_fields: usize, @@ -705,10 +752,11 @@ fn consume_slab_iterator( limit: Option, n_threads: usize, ) -> DaftResult> + Send> { - // Create slab pool for file reads. + // Create a slab iterator over the file. let slabpool = FileSlabPool::new(); let slab_iterator = SlabIterator::new(file, slabpool); + // Create a chunk iterator over the slab iterator. let csv_validator = CsvValidator::new( num_fields, parse_options.quote, @@ -716,9 +764,12 @@ fn consume_slab_iterator( parse_options.escape_char, parse_options.double_quote, ); - let chunk_iterator = ChunkyIterator::new(slab_iterator, csv_validator); + + // Create a chunk window iterator over the chunk iterator. let chunk_window_iterator = ChunkWindowIterator::new(chunk_iterator); + + // Stream tables from each chunk window. let has_header = parse_options.has_header; let parse_options = Arc::new(parse_options); let stream = futures::stream::iter(chunk_window_iterator.enumerate()) @@ -770,41 +821,6 @@ fn consume_slab_iterator( Ok(flattened) } -#[derive(Debug, Clone)] -enum ChunkState { - Start { - slab: Arc, - start: usize, - end: usize, - }, - Continue { - slab: Arc, - end: usize, - }, - Final { - slab: Arc, - end: usize, - valid_bytes: usize, - }, -} - -impl std::fmt::Display for ChunkState { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Self::Start { - slab: _, - start, - end, - } => write!(f, "ChunkState::Start-{start}-{end}"), - Self::Final { - slab: _, - end, - valid_bytes, - } => write!(f, "ChunkState::Final-{end}-{valid_bytes}"), - Self::Continue { slab: _, end } => write!(f, "ChunkState::Continue-{end}"), - } - } -} /// A helper struct that implements `std::io::Read` over a slice of ChunkStates. struct MultiSliceReader<'a> { states: &'a [ChunkState], From 7006a47b9b70a19f3a5a2807881fef0cb6ef47f7 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 23 Oct 2024 13:13:43 -0700 Subject: [PATCH 33/38] Remove crossbeam dep --- Cargo.lock | 1 - src/daft-csv/Cargo.toml | 1 - 2 files changed, 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 514becbb94..ff9277b012 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1762,7 +1762,6 @@ dependencies = [ "async-stream", "common-error", "common-py-serde", - "crossbeam-channel", "csv-async", "daft-compression", "daft-core", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index 0795719556..a916d1d7b8 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -4,7 +4,6 @@ async-compat = {workspace = true} async-stream = {workspace = true} common-error = {path = "../common/error", default-features = false} common-py-serde = {path = "../common/py-serde", default-features = false} -crossbeam-channel = "0.5.1" csv-async = "1.3.0" daft-compression = {path = "../daft-compression", default-features = false} daft-core = {path = "../daft-core", default-features = false} From 9d35c1b441562a986b7203820d7f508d8f9f5f29 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Wed, 23 Oct 2024 14:57:13 -0700 Subject: [PATCH 34/38] Move pool stuff into their own mod --- src/daft-csv/src/local.rs | 231 +++------------------------------ src/daft-csv/src/local/pool.rs | 221 +++++++++++++++++++++++++++++++ 2 files changed, 236 insertions(+), 216 deletions(-) create mode 100644 src/daft-csv/src/local/pool.rs diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 4de486b67a..fd555a46d2 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -1,15 +1,10 @@ use core::str; -use std::{ - io::Read, - num::NonZeroUsize, - ops::{Deref, DerefMut}, - sync::{Arc, Weak}, -}; +use std::{io::Read, num::NonZeroUsize, sync::Arc}; use arrow2::{ datatypes::Field, io::csv::{ - read::{self, Reader, ReaderBuilder}, + read::{Reader, ReaderBuilder}, read_async::local_read_rows, }, }; @@ -23,7 +18,6 @@ use daft_dsl::{optimization::get_required_columns, Expr}; use daft_io::{IOClient, IOStatsRef}; use daft_table::Table; use futures::{Stream, StreamExt, TryStreamExt}; -use parking_lot::{Mutex, RwLock}; use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, @@ -36,6 +30,9 @@ use crate::{ ArrowSnafu, CsvConvertOptions, CsvParseOptions, CsvReadOptions, JoinSnafu, }; +mod pool; +use pool::{CsvBuffer, CsvBufferPool, FileSlab, FileSlabPool, SLABSIZE}; + // Our local CSV reader takes the following approach to reading CSV files: // 1. Read the CSV file in 4MB chunks from a slab pool. // 2. Adjust the chunks so that chunks are contiguous and contain complete @@ -139,195 +136,8 @@ use crate::{ // we simply move on to the next chunk of bytes and try to find a valid CSV record there. This is a // simplification that makes the implementation a lot easier to maintain. -#[derive(Clone, Debug, Default)] -struct CsvSlab(Vec); - -impl CsvSlab { - fn new(record_size: usize, num_fields: usize, num_rows: usize) -> Self { - Self(vec![ - read::ByteRecord::with_capacity(record_size, num_fields); - num_rows - ]) - } -} - -impl Deref for CsvSlab { - type Target = Vec; - - fn deref(&self) -> &Self::Target { - &self.0 - } -} - -impl DerefMut for CsvSlab { - fn deref_mut(&mut self) -> &mut Self::Target { - &mut self.0 - } -} - -/// A pool of ByteRecord slabs. Used for deserializing CSV. -#[derive(Debug)] -struct CsvBufferPool { - buffers: Mutex>, - record_size: usize, - num_fields: usize, - num_rows: usize, -} - -/// A slab of ByteRecords. Used for deserializing CSV. -struct CsvBuffer { - buffer: CsvSlab, - pool: Weak, -} - -impl CsvBufferPool { - pub fn new( - record_size: usize, - num_fields: usize, - num_rows: usize, - initial_pool_size: usize, - ) -> Self { - let chunk_buffers = - vec![CsvSlab::new(record_size, num_fields, num_rows); initial_pool_size]; - Self { - buffers: Mutex::new(chunk_buffers), - record_size, - num_fields, - num_rows, - } - } - - pub fn get_buffer(self: &Arc) -> CsvBuffer { - let buffer = { - let mut buffers = self.buffers.lock(); - let buffer = buffers.pop(); - match buffer { - Some(buffer) => buffer, - None => CsvSlab::new(self.record_size, self.num_fields, self.num_rows), - } - }; - - CsvBuffer { - buffer, - pool: Arc::downgrade(self), - } - } - - fn return_buffer(&self, buffer: CsvSlab) { - let mut buffers = self.buffers.lock(); - buffers.push(buffer); - } -} - -impl Drop for CsvBuffer { - fn drop(&mut self) { - if let Some(pool) = self.pool.upgrade() { - let buffer = std::mem::take(&mut self.buffer); - pool.return_buffer(buffer); - } - } -} - -// The default size of a slab used for reading CSV files in chunks. Currently set to 4 MiB. This can be tuned. -const SLABSIZE: usize = 4 * 1024 * 1024; -// The default number of slabs in a slab pool. With 20 slabs, we reserve a total of 80 MiB of memory for reading file data. -const SLABPOOL_DEFAULT_SIZE: usize = 20; - -/// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. -#[derive(Debug)] -struct FileSlabPool { - slabs: Mutex>>, -} - -impl FileSlabPool { - fn new() -> Arc { - let slabs: Vec> = (0..SLABPOOL_DEFAULT_SIZE) - // We get uninitialized buffers because we will always populate the buffers with a file read before use. - .map(|_| Box::new_uninit_slice(SLABSIZE)) - .map(|x| unsafe { x.assume_init() }) - .map(|buffer| RwLock::new(FileSlabState::new(buffer, 0))) - .collect(); - Arc::new(Self { - slabs: Mutex::new(slabs), - }) - } - - fn get_slab(self: &Arc) -> Arc { - let slab = { - let mut slabs = self.slabs.lock(); - let slab = slabs.pop(); - match slab { - Some(slab) => slab, - None => RwLock::new(FileSlabState::new( - unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, - 0, - )), - } - }; - - Arc::new(FileSlab { - state: slab, - pool: Arc::downgrade(self), - }) - } - - fn return_slab(&self, slab: RwLock) { - let mut slabs = self.slabs.lock(); - slabs.push(slab); - } -} - -/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. -#[derive(Debug)] -struct FileSlab { - state: RwLock, - pool: Weak, -} - -impl FileSlab { - /// Given an offset into a FileSlab, finds the first \n char found in the FileSlabState's buffer, - /// then the returns the position relative to the given offset. - fn find_first_newline_from(&self, offset: usize) -> Option { - let guard = self.state.read(); - guard.find_first_newline_from(offset) - } -} - -// Modify the Drop method for FileSlabs so that their states are returned to their parent slab pool. -impl Drop for FileSlab { - fn drop(&mut self) { - if let Some(pool) = self.pool.upgrade() { - let file_slab_state = std::mem::take(&mut self.state); - pool.return_slab(file_slab_state); - } - } -} - -#[derive(Debug, Default)] -struct FileSlabState { - buffer: Box<[u8]>, - valid_bytes: usize, -} - -impl FileSlabState { - fn new(buffer: Box<[u8]>, valid_bytes: usize) -> Self { - Self { - buffer, - valid_bytes, - } - } - - /// Helper function that find the first \n char in the file slab state's buffer starting from `offset.` - fn find_first_newline_from(&self, offset: usize) -> Option { - newline_position(&self.buffer[offset..self.valid_bytes]) - } - - /// Validate the CSV record in the file slab state's buffer starting from `start`. `validator` is a - /// state machine that might need to process multiple buffers to validate CSV records. - fn validate_record(&self, validator: &mut CsvValidator, start: usize) -> Option { - validator.validate_record(&mut self.buffer[start..self.valid_bytes].iter()) - } -} +// Default size for CSV buffers. +const DEFAULT_CSV_BUFFER_SIZE: usize = SLABSIZE; // 4MiB. Like SLABSIZE, this can be tuned. /// Reads a single local CSV file in a non-streaming fashion. pub async fn read_csv_local( @@ -464,7 +274,7 @@ pub async fn stream_csv_local( let chunk_size = read_options .as_ref() .and_then(|opt| opt.chunk_size.or_else(|| opt.buffer_size.map(|bs| bs / 8))) - .unwrap_or(DEFAULT_CHUNK_SIZE); + .unwrap_or(DEFAULT_CSV_BUFFER_SIZE); let chunk_size_rows = (chunk_size as f64 / record_buffer_size as f64).ceil() as usize; // TODO(desmond): We might consider creating per-process buffer pools and slab pools. @@ -557,13 +367,13 @@ impl Iterator for SlabIterator { fn next(&mut self) -> Option { let slab = self.slabpool.get_slab(); let bytes_read = { - let mut writer = slab.state.write(); - let bytes_read = self.file.read(&mut writer.buffer).unwrap(); + let mut guard = slab.write(); + let bytes_read = self.file.read(&mut guard.buffer).unwrap(); if bytes_read == 0 { return None; } self.total_bytes_read += bytes_read; - writer.valid_bytes = bytes_read; + guard.valid_bytes = bytes_read; bytes_read }; @@ -649,7 +459,7 @@ where && curr_pos < valid_bytes { let offset = curr_pos + pos; - let guard = slab.state.read(); + let guard = slab.read(); chunk_state = match guard.validate_record(&mut self.validator, offset + 1) { Some(true) => Some(ChunkState::Final { slab: slab.clone(), @@ -846,15 +656,15 @@ impl<'a> Read for MultiSliceReader<'a> { let state = &self.states[self.curr_read_idx]; let (start, end, guard) = match state { ChunkState::Start { slab, start, end } => { - let guard = slab.state.read(); + let guard = slab.read(); (*start, *end, guard) } ChunkState::Continue { slab, end } => { - let guard = slab.state.read(); + let guard = slab.read(); (0, *end, guard) } ChunkState::Final { slab, end, .. } => { - let guard = slab.state.read(); + let guard = slab.read(); (0, *end, guard) } }; @@ -876,19 +686,8 @@ impl<'a> Read for MultiSliceReader<'a> { } } -// Daft does not currently support non-\n record terminators (e.g. carriage return \r, which only -// matters for pre-Mac OS X). const NEWLINE: u8 = b'\n'; const DOUBLE_QUOTE: u8 = b'"'; -const DEFAULT_CHUNK_SIZE: usize = SLABSIZE; // 4MiB. Like SLABSIZE, this can be tuned. - -/// Helper function that finds the first new line character (\n) in the given byte slice. -fn newline_position(buffer: &[u8]) -> Option { - // Assuming we are searching for the ASCII `\n` character, we don't need to do any special - // handling for UTF-8, since a `\n` value always corresponds to an ASCII `\n`. - // For more details, see: https://en.wikipedia.org/wiki/UTF-8#Encoding - memchr::memchr(NEWLINE, buffer) -} /// State machine that validates CSV records. struct CsvValidator { diff --git a/src/daft-csv/src/local/pool.rs b/src/daft-csv/src/local/pool.rs new file mode 100644 index 0000000000..c07ebb14bf --- /dev/null +++ b/src/daft-csv/src/local/pool.rs @@ -0,0 +1,221 @@ +use std::{ + ops::{Deref, DerefMut}, + sync::{Arc, Weak}, +}; + +use arrow2::io::csv::read; +use parking_lot::{Mutex, RwLock}; + +// The default size of a slab used for reading CSV files in chunks. Currently set to 4 MiB. This can be tuned. +pub const SLABSIZE: usize = 4 * 1024 * 1024; +// The default number of slabs in a slab pool. With 20 slabs, we reserve a total of 80 MiB of memory for reading file data. +const SLABPOOL_DEFAULT_SIZE: usize = 20; + +#[derive(Clone, Debug, Default)] +pub struct CsvSlab(Vec); + +impl CsvSlab { + fn new(record_size: usize, num_fields: usize, num_rows: usize) -> Self { + Self(vec![ + read::ByteRecord::with_capacity(record_size, num_fields); + num_rows + ]) + } +} + +impl Deref for CsvSlab { + type Target = Vec; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl DerefMut for CsvSlab { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.0 + } +} + +/// A pool of ByteRecord slabs. Used for deserializing CSV. +#[derive(Debug)] +pub struct CsvBufferPool { + buffers: Mutex>, + record_size: usize, + num_fields: usize, + num_rows: usize, +} + +/// A slab of ByteRecords. Used for deserializing CSV. +pub struct CsvBuffer { + pub buffer: CsvSlab, + pool: Weak, +} + +impl CsvBufferPool { + pub fn new( + record_size: usize, + num_fields: usize, + num_rows: usize, + initial_pool_size: usize, + ) -> Self { + let chunk_buffers = + vec![CsvSlab::new(record_size, num_fields, num_rows); initial_pool_size]; + Self { + buffers: Mutex::new(chunk_buffers), + record_size, + num_fields, + num_rows, + } + } + + pub fn get_buffer(self: &Arc) -> CsvBuffer { + let buffer = { + let mut buffers = self.buffers.lock(); + let buffer = buffers.pop(); + match buffer { + Some(buffer) => buffer, + None => CsvSlab::new(self.record_size, self.num_fields, self.num_rows), + } + }; + + CsvBuffer { + buffer, + pool: Arc::downgrade(self), + } + } + + fn return_buffer(&self, buffer: CsvSlab) { + let mut buffers = self.buffers.lock(); + buffers.push(buffer); + } +} + +impl Drop for CsvBuffer { + fn drop(&mut self) { + if let Some(pool) = self.pool.upgrade() { + let buffer = std::mem::take(&mut self.buffer); + pool.return_buffer(buffer); + } + } +} + +/// A pool of slabs. Used for reading CSV files in SLABSIZE chunks. +#[derive(Debug)] +pub struct FileSlabPool { + slabs: Mutex>>, +} + +impl FileSlabPool { + pub fn new() -> Arc { + let slabs: Vec> = (0..SLABPOOL_DEFAULT_SIZE) + // We get uninitialized buffers because we will always populate the buffers with a file read before use. + .map(|_| Box::new_uninit_slice(SLABSIZE)) + .map(|x| unsafe { x.assume_init() }) + .map(|buffer| RwLock::new(FileSlabState::new(buffer, 0))) + .collect(); + Arc::new(Self { + slabs: Mutex::new(slabs), + }) + } + + pub fn get_slab(self: &Arc) -> Arc { + let slab = { + let mut slabs = self.slabs.lock(); + let slab = slabs.pop(); + match slab { + Some(slab) => slab, + None => RwLock::new(FileSlabState::new( + unsafe { Box::new_uninit_slice(SLABSIZE).assume_init() }, + 0, + )), + } + }; + + Arc::new(FileSlab { + state: slab, + pool: Arc::downgrade(self), + }) + } + + fn return_slab(&self, slab: RwLock) { + let mut slabs = self.slabs.lock(); + slabs.push(slab); + } +} + +/// A slab of bytes. Used for reading CSV files in SLABSIZE chunks. +#[derive(Debug)] +pub struct FileSlab { + state: RwLock, + pool: Weak, +} + +impl FileSlab { + /// Given an offset into a FileSlab, finds the first \n char found in the FileSlabState's buffer, + /// then the returns the position relative to the given offset. + pub fn find_first_newline_from(&self, offset: usize) -> Option { + let guard = self.state.read(); + guard.find_first_newline_from(offset) + } +} + +impl Deref for FileSlab { + type Target = RwLock; + + fn deref(&self) -> &Self::Target { + &self.state + } +} + +// Modify the Drop method for FileSlabs so that their states are returned to their parent slab pool. +impl Drop for FileSlab { + fn drop(&mut self) { + if let Some(pool) = self.pool.upgrade() { + let file_slab_state = std::mem::take(&mut self.state); + pool.return_slab(file_slab_state); + } + } +} + +#[derive(Debug, Default)] +pub struct FileSlabState { + pub buffer: Box<[u8]>, + pub valid_bytes: usize, +} + +impl FileSlabState { + fn new(buffer: Box<[u8]>, valid_bytes: usize) -> Self { + Self { + buffer, + valid_bytes, + } + } + + /// Helper function that find the first \n char in the file slab state's buffer starting from `offset.` + fn find_first_newline_from(&self, offset: usize) -> Option { + newline_position(&self.buffer[offset..self.valid_bytes]) + } + + /// Validate the CSV record in the file slab state's buffer starting from `start`. `validator` is a + /// state machine that might need to process multiple buffers to validate CSV records. + pub fn validate_record( + &self, + validator: &mut super::CsvValidator, + start: usize, + ) -> Option { + validator.validate_record(&mut self.buffer[start..self.valid_bytes].iter()) + } +} + +// Daft does not currently support non-\n record terminators (e.g. carriage return \r, which only +// matters for pre-Mac OS X). +const NEWLINE: u8 = b'\n'; + +/// Helper function that finds the first new line character (\n) in the given byte slice. +fn newline_position(buffer: &[u8]) -> Option { + // Assuming we are searching for the ASCII `\n` character, we don't need to do any special + // handling for UTF-8, since a `\n` value always corresponds to an ASCII `\n`. + // For more details, see: https://en.wikipedia.org/wiki/UTF-8#Encoding + memchr::memchr(NEWLINE, buffer) +} From 42ef62d2fd246b00f2c0bbca56395346e248b6da Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Thu, 24 Oct 2024 01:14:46 -0700 Subject: [PATCH 35/38] use smoll vec --- Cargo.lock | 1 + src/daft-csv/Cargo.toml | 1 + src/daft-csv/src/local.rs | 5 +++-- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ff9277b012..186915c646 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1776,6 +1776,7 @@ dependencies = [ "rayon", "rstest", "serde", + "smallvec", "snafu", "tokio", "tokio-util", diff --git a/src/daft-csv/Cargo.toml b/src/daft-csv/Cargo.toml index a916d1d7b8..c46a8dcc8e 100644 --- a/src/daft-csv/Cargo.toml +++ b/src/daft-csv/Cargo.toml @@ -17,6 +17,7 @@ parking_lot = "0.12.3" pyo3 = {workspace = true, optional = true} rayon = {workspace = true} serde = {workspace = true} +smallvec = "1.13.2" snafu = {workspace = true} tokio = {workspace = true} tokio-util = {workspace = true} diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index fd555a46d2..234168d56f 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -22,6 +22,7 @@ use rayon::{ iter::IndexedParallelIterator, prelude::{IntoParallelRefIterator, ParallelIterator}, }; +use smallvec::SmallVec; use snafu::ResultExt; use crate::{ @@ -530,9 +531,9 @@ impl Iterator for ChunkWindowIterator where I: Iterator, { - type Item = Vec; + type Item = SmallVec<[ChunkState; 2]>; fn next(&mut self) -> Option { - let mut chunks = Vec::with_capacity(2); + let mut chunks = SmallVec::with_capacity(2); for chunk in self.chunk_iter.by_ref() { chunks.push(chunk); if let ChunkState::Final { .. } = chunks.last().expect("We just pushed a chunk") { From 2343c06db0feebe8b359c4973ab19dfefe0d4c44 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Thu, 24 Oct 2024 02:21:17 -0700 Subject: [PATCH 36/38] Apply global limit within streaming path --- src/daft-csv/src/local.rs | 46 ++++++++++++++++++++------------------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index 234168d56f..d7565590e7 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -161,25 +161,7 @@ pub async fn read_csv_local( ) .await?; let tables = Box::pin(stream); - // Apply limit. - let limit = convert_options.as_ref().and_then(|opts| opts.limit); - let mut remaining_rows = limit.map(|limit| limit as i64); - use futures::TryStreamExt; let collected_tables = tables - .try_take_while(|result| { - match (result, remaining_rows) { - // Limit has been met, early-terminate. - (_, Some(rows_left)) if rows_left <= 0 => futures::future::ready(Ok(false)), - // Limit has not yet been met, update remaining limit slack and continue. - (table, Some(rows_left)) => { - remaining_rows = Some(rows_left - table.len() as i64); - futures::future::ready(Ok(true)) - } - // (1) No limit, never early-terminate. - // (2) Encountered error, propagate error to try_collect to allow it to short-circuit. - (_, None) => futures::future::ready(Ok(true)), - } - }) .try_collect::>() .await? .into_iter() @@ -197,10 +179,12 @@ pub async fn read_csv_local( return Table::empty(Some(Arc::new(Schema::try_from(&schema)?))); } let concated_table = tables_concat(collected_tables)?; + + // Apply head in case the last chunk went over limit. + let limit = convert_options.as_ref().and_then(|opts| opts.limit); if let Some(limit) = limit && concated_table.len() > limit { - // Apply head in case that last chunk went over limit. concated_table.head(limit) } else { Ok(concated_table) @@ -611,8 +595,9 @@ fn stream_csv_as_tables( predicate, limit, ); - tx.send(tables) - .expect("OneShot Channel should still be open"); + // We throw away the error because we might close the oneshot channel in the case where + // a limit is applied and we early-terminate. + let _ = tx.send(tables); }); rx.await }) @@ -629,7 +614,24 @@ fn stream_csv_as_tables( }) .try_flatten(); - Ok(flattened) + // Apply limit. + let mut remaining_rows = limit.map(|limit| limit as i64); + let limited = flattened.try_take_while(move |result| { + match (result, remaining_rows) { + // Limit has been met, early-terminate. + (_, Some(rows_left)) if rows_left <= 0 => futures::future::ready(Ok(false)), + // Limit has not yet been met, update remaining limit slack and continue. + (table, Some(rows_left)) => { + remaining_rows = Some(rows_left - table.len() as i64); + futures::future::ready(Ok(true)) + } + // (1) No limit, never early-terminate. + // (2) Encountered error, propagate error to try_collect to allow it to short-circuit. + (_, None) => futures::future::ready(Ok(true)), + } + }); + + Ok(limited) } /// A helper struct that implements `std::io::Read` over a slice of ChunkStates. From f5a80b471b7f94ccd512cd8585b99c66a6e9cab9 Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Thu, 24 Oct 2024 02:23:38 -0700 Subject: [PATCH 37/38] Add todo to modify read_csv to take in configurable record terminator --- src/daft-csv/src/local.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index d7565590e7..c363e8012b 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -689,6 +689,8 @@ impl<'a> Read for MultiSliceReader<'a> { } } +// TODO(desmond): The Daft read_csv API does not currently accept non-\n record terminators. We should +// make this user configurable. const NEWLINE: u8 = b'\n'; const DOUBLE_QUOTE: u8 = b'"'; From 8106072ae31ef470e02cb350259a91ac43cd841c Mon Sep 17 00:00:00 2001 From: desmondcheongzx Date: Thu, 24 Oct 2024 11:32:58 -0700 Subject: [PATCH 38/38] Fix limit with predicate --- src/daft-csv/src/local.rs | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/daft-csv/src/local.rs b/src/daft-csv/src/local.rs index c363e8012b..7a91fa5325 100644 --- a/src/daft-csv/src/local.rs +++ b/src/daft-csv/src/local.rs @@ -911,17 +911,18 @@ where } else { table }; + // Check the local limit. + let limit_reached = if let Some(local_limit) = &mut local_limit { + *local_limit -= table.len(); + *local_limit == 0 + } else { + false + }; tables.push(table); - // Stop reading once we hit the local limit. - if let Some(local_limit) = &mut local_limit { - *local_limit -= num_rows; - if *local_limit == 0 { - break; - } - } + // The number of record might exceed the number of byte records we've allocated. // Retry until all byte records in this chunk are read. - if !has_more { + if !has_more || limit_reached { break; } }