Skip to content

Commit c7df611

Browse files
perf: Reduce memory consumption for WARC reads and improve estimates (#3935)
This PR makes the following changes for `read_warc`: - Reduce memory consumption - Adds `WARC-Identified-Payload-Type` as an extracted metadata column - Improve stats estimation for scan tasks that read WARC ## Reduced memory consumption When reading a single Common Crawl file, the file size is typically 1GB, which decompresses to 5GB of data. Before this Resident Set Size peaks at `5.15GB` while heap size peaks at `10.98GB`: ![8D239774-4801-4350-A560-05CEA1202CB5_1_201_a](https://github.com/user-attachments/assets/5931ec32-bdce-4b2b-8838-a72e5f7ad6f4) After this PR, Resident Set Size peaks at `4.3GB` while heap size peaks at `6.6GB`, which is more in line with expectations: ![D4676719-F780-4745-8E72-A269F07B6D05_1_201_a](https://github.com/user-attachments/assets/defc2a24-38a2-4adf-9d5f-529f0a895212) ## Additional `WARC-Identified-Payload-Type` metadata column For ease of filtering WARC records, we extract `WARC-Identified-Payload-Type` from the metadata as its own column. Since this is an optional column, it is often NULL. ## Stats estimation A single Common Crawl .warc.gz file is typically 1GB in size, but takes up ~5GB of memory once decompressed. For a .warc.gz file with `145,717` records, before this PR we would estimate: ``` Stats = { Approx num rows = 9,912,769, Approx size bytes = 914.63 MiB, Accumulated selectivity = 1.00 } ``` After this PR, we now estimate: ``` Stats = { Approx num rows = 167,773, Approx size bytes = 4.34 GiB, Accumulated selectivity = 1.00 } ``` which is much closer to reality. ### Estimations with pushdowns When doing `daft.read_warc("file.warc.gz").select("Content-Length")`, we estimate `1.32 MiB` and in reality store `1.13 MiB`. When doing `daft.read_warc("cc-original.warc.gz").select("warc_content")`, we estimate `4.39 GiB` and in reality store `3.82 GiB`.
1 parent 8163982 commit c7df611

File tree

10 files changed

+221
-87
lines changed

10 files changed

+221
-87
lines changed

Cargo.lock

+1-1
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

daft/daft/__init__.pyi

+7
Original file line numberDiff line numberDiff line change
@@ -1582,6 +1582,13 @@ class PyMicroPartition:
15821582
io_config: IOConfig | None = None,
15831583
multithreaded_io: bool | None = None,
15841584
): ...
1585+
@classmethod
1586+
def read_warc(
1587+
cls,
1588+
uri: str,
1589+
io_config: IOConfig | None = None,
1590+
multithreaded_io: bool | None = None,
1591+
): ...
15851592

15861593
class PhysicalPlanScheduler:
15871594
"""A work scheduler for physical query plans."""

daft/io/_warc.py

+4-2
Original file line numberDiff line numberDiff line change
@@ -40,8 +40,9 @@ def read_warc(
4040
Defaults to None, which will let Daft decide based on the runner it is currently using.
4141
4242
returns:
43-
DataFrame: parsed DataFrame with mandatory metadata columns: "WARC-Record-ID", "WARC-Type", "WARC-Date", "Content-Length", one column "warc_content"
44-
with the raw byte content of the WARC record, and one column "warc_headers" with the remaining headers of the WARC record stored as a JSON string.
43+
DataFrame: parsed DataFrame with mandatory metadata columns ("WARC-Record-ID", "WARC-Type", "WARC-Date", "Content-Length"), one optional
44+
metadata column ("WARC-Identified-Payload-Type"), one column "warc_content" with the raw byte content of the WARC record,
45+
and one column "warc_headers" with the remaining headers of the WARC record stored as a JSON string.
4546
"""
4647
io_config = context.get_context().daft_planning_config.default_io_config if io_config is None else io_config
4748

@@ -60,6 +61,7 @@ def read_warc(
6061
"WARC-Type": DataType.string(),
6162
"WARC-Date": DataType.timestamp(TimeUnit.ns(), timezone="Etc/UTC"),
6263
"Content-Length": DataType.int64(),
64+
"WARC-Identified-Payload-Type": DataType.string(),
6365
"warc_content": DataType.binary(),
6466
"warc_headers": DataType.string(),
6567
}

daft/recordbatch/micropartition.py

+15
Original file line numberDiff line numberDiff line change
@@ -524,3 +524,18 @@ def read_json(
524524
multithreaded_io=multithreaded_io,
525525
)
526526
)
527+
528+
@classmethod
529+
def read_warc(
530+
cls,
531+
path: str,
532+
io_config: IOConfig | None = None,
533+
multithreaded_io: bool | None = None,
534+
) -> MicroPartition:
535+
return MicroPartition._from_pymicropartition(
536+
_PyMicroPartition.read_warc(
537+
uri=path,
538+
io_config=io_config,
539+
multithreaded_io=multithreaded_io,
540+
)
541+
)

src/daft-micropartition/src/micropartition.rs

+35
Original file line numberDiff line numberDiff line change
@@ -794,6 +794,41 @@ pub fn read_json_into_micropartition(
794794
}
795795
}
796796

797+
pub fn read_warc_into_micropartition(
798+
uris: &[&str],
799+
schema: SchemaRef,
800+
io_config: Arc<IOConfig>,
801+
multithreaded_io: bool,
802+
io_stats: Option<IOStatsRef>,
803+
) -> DaftResult<MicroPartition> {
804+
let io_client = daft_io::get_io_client(multithreaded_io, io_config)?;
805+
let convert_options = WarcConvertOptions {
806+
limit: None,
807+
include_columns: None,
808+
schema: schema.clone(),
809+
predicate: None,
810+
};
811+
812+
match uris {
813+
[] => Ok(MicroPartition::empty(None)),
814+
uris => {
815+
// Perform a bulk read of URIs, materializing a table per URI.
816+
let tables = daft_warc::read_warc_bulk(
817+
uris,
818+
convert_options,
819+
io_client,
820+
io_stats,
821+
multithreaded_io,
822+
None,
823+
8,
824+
)
825+
.context(DaftCoreComputeSnafu)?;
826+
827+
// Construct MicroPartition from tables and unioned schema
828+
Ok(MicroPartition::new_loaded(schema, Arc::new(tables), None))
829+
}
830+
}
831+
}
797832
fn get_file_column_names<'a>(
798833
columns: Option<&'a [&'a str]>,
799834
partition_spec: Option<&PartitionSpec>,

src/daft-micropartition/src/python.rs

+42
Original file line numberDiff line numberDiff line change
@@ -761,6 +761,48 @@ impl PyMicroPartition {
761761
Ok(mp.into())
762762
}
763763

764+
#[staticmethod]
765+
#[pyo3(signature = (
766+
uri,
767+
io_config=None,
768+
multithreaded_io=None
769+
))]
770+
pub fn read_warc(
771+
py: Python,
772+
uri: &str,
773+
io_config: Option<IOConfig>,
774+
multithreaded_io: Option<bool>,
775+
) -> PyResult<Self> {
776+
let schema = Arc::new(Schema::new(vec![
777+
Field::new("WARC-Record-ID", daft_core::prelude::DataType::Utf8),
778+
Field::new("WARC-Type", daft_core::prelude::DataType::Utf8),
779+
Field::new(
780+
"WARC-Date",
781+
daft_core::prelude::DataType::Timestamp(
782+
TimeUnit::Nanoseconds,
783+
Some("Etc/UTC".to_string()),
784+
),
785+
),
786+
Field::new("Content-Length", daft_core::prelude::DataType::Int64),
787+
Field::new(
788+
"WARC-Identified-Payload-Type",
789+
daft_core::prelude::DataType::Utf8,
790+
),
791+
Field::new("warc_content", daft_core::prelude::DataType::Binary),
792+
Field::new("warc_headers", daft_core::prelude::DataType::Utf8),
793+
])?);
794+
let mp = py.allow_threads(|| {
795+
crate::micropartition::read_warc_into_micropartition(
796+
&[uri],
797+
schema.into(),
798+
io_config.unwrap_or_default().config.into(),
799+
multithreaded_io.unwrap_or(true),
800+
None,
801+
)
802+
})?;
803+
Ok(mp.into())
804+
}
805+
764806
#[staticmethod]
765807
pub fn _from_unloaded_table_state(
766808
schema_bytes: &[u8],

src/daft-scan/Cargo.toml

+1
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ daft-stats = {path = "../daft-stats", default-features = false}
2222
futures = {workspace = true}
2323
indexmap = {workspace = true}
2424
itertools = {workspace = true}
25+
lazy_static = "1.5.0"
2526
parquet2 = {workspace = true}
2627
pyo3 = {workspace = true, optional = true}
2728
serde = {workspace = true}

src/daft-scan/src/lib.rs

+86-20
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
use std::{
44
any::Any,
55
borrow::Cow,
6+
collections::HashMap,
67
fmt::Debug,
78
hash::{Hash, Hasher},
89
sync::Arc,
@@ -15,6 +16,7 @@ use common_scan_info::{Pushdowns, ScanTaskLike, ScanTaskLikeRef};
1516
use daft_schema::schema::{Schema, SchemaRef};
1617
use daft_stats::{PartitionSpec, TableMetadata, TableStatistics};
1718
use itertools::Itertools;
19+
use lazy_static::lazy_static;
1820
use parquet2::metadata::FileMetaData;
1921
use serde::{Deserialize, Serialize};
2022

@@ -500,6 +502,21 @@ impl From<ScanTask> for ScanTaskLikeRef {
500502

501503
pub type ScanTaskRef = Arc<ScanTask>;
502504

505+
lazy_static! {
506+
static ref WARC_COLUMN_SIZES: HashMap<&'static str, usize> = {
507+
let mut m = HashMap::new();
508+
// Average sizes based on analysis of Common Crawl WARC files.
509+
m.insert("WARC-Record-ID", 36); // UUID-style identifiers.
510+
m.insert("WARC-Type", 8); // e.g. "response".
511+
m.insert("WARC-Date", 8); // Timestamp stored as i64 nanoseconds.
512+
m.insert("Content-Length", 8); // i64.
513+
m.insert("WARC-Identified-Payload-Type", 5); // e.g. "text/html". Typically null.
514+
m.insert("warc_content", 27282); // Average content size.
515+
m.insert("warc_headers", 350); // Average headers size.
516+
m
517+
};
518+
}
519+
503520
impl ScanTask {
504521
#[must_use]
505522
pub fn new(
@@ -671,15 +688,26 @@ impl ScanTask {
671688
FileFormatConfig::Csv(_) | FileFormatConfig::Json(_) => {
672689
config.csv_inflation_factor
673690
}
674-
// TODO(desmond): We can do a lot better here.
675-
FileFormatConfig::Warc(_) => 1.0,
691+
FileFormatConfig::Warc(_) => {
692+
if self.is_gzipped() {
693+
5.0
694+
} else {
695+
1.0
696+
}
697+
}
676698
#[cfg(feature = "python")]
677699
FileFormatConfig::Database(_) => 1.0,
678700
#[cfg(feature = "python")]
679701
FileFormatConfig::PythonFunction => 1.0,
680702
};
681703
let in_mem_size: f64 = (file_size as f64) * inflation_factor;
682-
let read_row_size = self.schema.estimate_row_size_bytes();
704+
let read_row_size = if self.is_warc() {
705+
// Across 100 Common Crawl WARC files, the average record size is 470 (metadata) + 27282 (content) bytes.
706+
// This is 27752 bytes per record.
707+
27752.0
708+
} else {
709+
self.schema.estimate_row_size_bytes()
710+
};
683711
in_mem_size / read_row_size
684712
})
685713
});
@@ -717,30 +745,68 @@ impl ScanTask {
717745
self.size_bytes_on_disk.map(|s| s as usize)
718746
}
719747

748+
fn is_warc(&self) -> bool {
749+
matches!(self.file_format_config.as_ref(), FileFormatConfig::Warc(_))
750+
}
751+
752+
fn is_gzipped(&self) -> bool {
753+
self.sources
754+
.first()
755+
.and_then(|s| match s {
756+
DataSource::File { path, .. } => {
757+
let filename = std::path::Path::new(path);
758+
Some(
759+
filename
760+
.extension()
761+
.is_some_and(|ext| ext.eq_ignore_ascii_case("gz")),
762+
)
763+
}
764+
_ => None,
765+
})
766+
.unwrap_or(false)
767+
}
768+
720769
#[must_use]
721770
pub fn estimate_in_memory_size_bytes(
722771
&self,
723772
config: Option<&DaftExecutionConfig>,
724773
) -> Option<usize> {
725-
let mat_schema = self.materialized_schema();
726-
self.statistics
727-
.as_ref()
728-
.and_then(|s| {
729-
// Derive in-memory size estimate from table stats.
730-
self.num_rows().and_then(|num_rows| {
731-
let row_size = s.estimate_row_size(Some(mat_schema.as_ref())).ok()?;
732-
let estimate = (num_rows as f64) * row_size;
733-
Some(estimate as usize)
774+
// WARC files that are gzipped are often 5x smaller than the uncompressed size.
775+
// For example, see this blog post by Common Crawl: https://commoncrawl.org/blog/february-2025-crawl-archive-now-available
776+
if self.is_warc() {
777+
let approx_num_rows = self.approx_num_rows(config)?;
778+
let mat_schema = self.materialized_schema();
779+
780+
// Calculate size based on materialized schema and WARC column sizes
781+
let row_size: usize = mat_schema
782+
.fields
783+
.iter()
784+
.map(|(name, _)| WARC_COLUMN_SIZES.get(name.as_str()).copied().unwrap_or(8))
785+
.sum();
786+
787+
let estimate = (approx_num_rows * row_size as f64) as usize;
788+
Some(estimate)
789+
} else {
790+
let mat_schema = self.materialized_schema();
791+
self.statistics
792+
.as_ref()
793+
.and_then(|s| {
794+
// Derive in-memory size estimate from table stats.
795+
self.num_rows().and_then(|num_rows| {
796+
let row_size = s.estimate_row_size(Some(mat_schema.as_ref())).ok()?;
797+
let estimate = (num_rows as f64) * row_size;
798+
Some(estimate as usize)
799+
})
734800
})
735-
})
736-
.or_else(|| {
737-
// use approximate number of rows multiplied by an approximate bytes-per-row
738-
self.approx_num_rows(config).map(|approx_num_rows| {
739-
let row_size = mat_schema.estimate_row_size_bytes();
740-
let estimate = approx_num_rows * row_size;
741-
estimate as usize
801+
.or_else(|| {
802+
// use approximate number of rows multiplied by an approximate bytes-per-row
803+
self.approx_num_rows(config).map(|approx_num_rows| {
804+
let row_size = mat_schema.estimate_row_size_bytes();
805+
let estimate = approx_num_rows * row_size;
806+
estimate as usize
807+
})
742808
})
743-
})
809+
}
744810
}
745811

746812
#[must_use]

src/daft-warc/Cargo.toml

-1
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@ daft-io = {path = "../daft-io", default-features = false}
1010
daft-recordbatch = {path = "../daft-recordbatch", default-features = false}
1111
flate2 = {version = "1.1", features = ["zlib-rs"], default-features = false}
1212
futures = {workspace = true}
13-
rayon = {workspace = true}
1413
serde_json = {workspace = true}
1514
snafu = {workspace = true}
1615
tokio = {workspace = true}

0 commit comments

Comments
 (0)