Skip to content

Commit 4cbbd10

Browse files
adriangbclaude
andcommitted
hide cube-root math from public API
The hierarchical sampling strategy (cube-root file split, sqrt row-group/row split, single-axis fallback for small inputs) is an implementation detail and may evolve. Stop leaking it through: - `ParquetSampling.system_target_remaining` becomes `pub(crate)`. External callers can't see or set it; it stays a coordination channel between `try_push_sample` and the parquet opener. - `try_push_sample`'s rustdoc no longer describes the cube-root / sqrt math. It now documents *behaviour* (absorb a TABLESAMPLE request, drop files / row groups / rows in proportion) and points at the SQL reference for users who want to know how it's implemented today. The math survives as an inline comment for maintainers. Co-Authored-By: Claude Opus 4.7 (1M context) <noreply@anthropic.com>
1 parent 75edac6 commit 4cbbd10

1 file changed

Lines changed: 23 additions & 45 deletions

File tree

datafusion/datasource-parquet/src/source.rs

Lines changed: 23 additions & 45 deletions
Original file line numberDiff line numberDiff line change
@@ -346,26 +346,15 @@ pub struct ParquetSampling {
346346
/// `ceil(target / row_cluster_size)` windows distributed across
347347
/// the row group with a random offset within each stride.
348348
pub row_cluster_size: usize,
349-
/// SYSTEM-mode adaptive target: the fraction of rows the opener
350-
/// should retain *after* file-level filtering has already been
351-
/// applied (via `keep_files` from the `SamplePushdown` rule).
349+
/// Internal coordination channel between
350+
/// [`ParquetSource::try_push_sample`] and the parquet opener.
351+
/// Not part of the public sampling API — direct callers configure
352+
/// sampling via the per-axis builders above. See the
353+
/// [`TABLESAMPLE clause`] section of the SQL reference for the
354+
/// pushdown strategy this implements.
352355
///
353-
/// When set, the opener ignores `row_group_fraction` and
354-
/// `row_fraction` and instead chooses them adaptively based on
355-
/// the actual row-group count it sees per file:
356-
///
357-
/// * Multiple row groups: split as `q = sqrt(remaining)` at both
358-
/// row-group and row level so the product stays at `remaining`.
359-
/// * Single row group: skip row-group sampling and apply
360-
/// `remaining` directly at the row level — otherwise a 1-RG
361-
/// file would only achieve `sqrt(remaining)` of the target.
362-
///
363-
/// This is the field [`ParquetSource::try_push_sample`] sets when
364-
/// it absorbs a `TABLESAMPLE SYSTEM(p%)`. Direct callers of
365-
/// [`ParquetSource::with_row_group_sampling`] /
366-
/// [`ParquetSource::with_row_fraction`] leave this `None` and
367-
/// retain the explicit per-axis behaviour.
368-
pub system_target_remaining: Option<f64>,
356+
/// [`TABLESAMPLE clause`]: https://datafusion.apache.org/user-guide/sql/select.html#tablesample-clause
357+
pub(crate) system_target_remaining: Option<f64>,
369358
}
370359

371360
impl Default for ParquetSampling {
@@ -987,30 +976,17 @@ impl FileSource for ParquetSource {
987976
Ok(tnr)
988977
}
989978

990-
/// Absorb a TABLESAMPLE-shaped sample request.
979+
/// Absorb a `TABLESAMPLE`-shaped sample request into the parquet
980+
/// scan: drop files, row groups, and rows in proportion to the
981+
/// requested fraction, with no `SampleExec` left in the plan.
991982
///
992-
/// For SYSTEM sampling we use a hierarchical "cube-root" split
993-
/// across files, row groups, and rows. The ideal split for a
994-
/// requested fraction `p` is `q = p^(1/3)` at each axis so the
995-
/// product stays at `p`, but we adapt when an axis can't reduce:
983+
/// `SYSTEM` sampling is the only supported method. The sampling
984+
/// strategy (a hierarchical block-level reduction across files,
985+
/// row groups, and rows) is described in the [`TABLESAMPLE clause`]
986+
/// section of the SQL reference; it is intentionally not part of
987+
/// the public sampling API and may evolve.
996988
///
997-
/// * Single-file scan (`num_files <= 1`): the file axis is fixed
998-
/// at 1.0, so we hand the full `p` budget to the opener via
999-
/// [`ParquetSampling::system_target_remaining`]. The opener
1000-
/// then either splits row-group × row at `sqrt(p)` each (when
1001-
/// it sees ≥ 2 row groups) or applies the full `p` at the row
1002-
/// level alone (when it sees 1 row group).
1003-
/// * Multi-file scan: keep `target_files = ⌈n × p^(1/3)⌉` files
1004-
/// via a seeded shuffle, then ask the opener to retain
1005-
/// `remaining = p × num_files / target_files` of the rows it
1006-
/// sees. The opener again splits row-group × row adaptively.
1007-
///
1008-
/// The result: the expected output size is always close to
1009-
/// `p × N_total`, regardless of how many files / row groups the
1010-
/// scan happens to have. Without this adaptation, a single-file
1011-
/// `SYSTEM(10)` would read `cbrt(0.1) ≈ 46%` of the rows because
1012-
/// the file and row-group axes can't reduce. See the PR
1013-
/// description for the full math.
989+
/// [`TABLESAMPLE clause`]: https://datafusion.apache.org/user-guide/sql/select.html#tablesample-clause
1014990
fn try_push_sample(
1015991
&self,
1016992
spec: &datafusion_physical_plan::sample_pushdown::SampleSpec,
@@ -1020,6 +996,12 @@ impl FileSource for ParquetSource {
1020996
use datafusion_datasource::file::FileSourceSampleResult;
1021997
use datafusion_physical_plan::sample_pushdown::SampleMethod;
1022998

999+
// Implementation detail (not promised by the public API): the
1000+
// file axis uses ~p^(1/3) so the cube-root product across
1001+
// file × row-group × row stays at p; the opener handles the
1002+
// remaining row-group × row split adaptively based on the
1003+
// actual row-group count it sees per file.
1004+
10231005
match spec.method {
10241006
SampleMethod::System => {
10251007
let p = spec.fraction.clamp(0.0, 1.0);
@@ -1031,10 +1013,6 @@ impl FileSource for ParquetSource {
10311013
});
10321014
}
10331015

1034-
// File level: when there is more than one file, drop
1035-
// ~p^(1/3) of them. With a single file the axis can't
1036-
// reduce, so pass the full budget through to the
1037-
// opener.
10381016
let (keep_files, remaining_p) = if num_files <= 1 {
10391017
(None, p)
10401018
} else {

0 commit comments

Comments
 (0)