Skip to content
Open
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
312 changes: 195 additions & 117 deletions rust/lance/src/dataset/optimize.rs
Original file line number Diff line number Diff line change
Expand Up @@ -205,9 +205,157 @@ impl AddAssign for CompactionMetrics {
}
}

/// Trait for implementing custom compaction planning strategies.
///
/// This trait allows users to define their own compaction strategies by implementing
/// the `plan` method. The default implementation is provided by [`DefaultCompactionPlanner`].
#[async_trait::async_trait]
pub trait CompactionPlanner: Send + Sync {
// get all fragments by default
fn get_fragments(&self, dataset: &Dataset, _options: &CompactionOptions) -> Vec<FileFragment> {
// get_fragments should be returning fragments in sorted order (by id)
// and fragment ids should be unique
dataset.get_fragments()
}
Comment on lines +215 to +220
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Was already commented on, but do we need this? It seems like individual implementations can just call dataset.get_fragments() and then do whatever filtering they would like.


// no filter by default
async fn filter_fragments(
&self,
_dataset: &Dataset,
fragments: Vec<FileFragment>,
_options: &CompactionOptions,
) -> Result<Vec<FileFragment>> {
Ok(fragments)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do these really need to be trait methods? I think we can probably leave them out and just let individual implementations use them if they want to. It will keep the trait simpler.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removed.


async fn plan(&self, dataset: &Dataset, options: &CompactionOptions) -> Result<CompactionPlan>;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's document this method.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added docs

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will CompactionOptions be flexible enough for all possible strategies? Should we maybe accept options as a JSON string or a Map<String, String>? This way different strategies can expose their own custom options. That would leave the API a little less defined but it would be more flexible.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we even need to take CompactionOptions? Maybe it should be a argument to the constructor of the individual structs. That way each could have their own arguments but also be strongly typed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will CompactionOptions be flexible enough for all possible strategies? Should we maybe accept options as a JSON string or a Map<String, String>? This way different strategies can expose their own custom options. That would leave the API a little less defined but it would be more flexible.

Hi @westonpace Thanks a lot for your review. Added Map<String, String> for flexible.

Do we even need to take CompactionOptions? Maybe it should be a argument to the constructor of the individual structs. That way each could have their own arguments but also be strongly typed.

Hi @wjones127 Thanks a lot for your review.

I have tried several ways to eliminate the CompactionOptions parameter in the plan method, but none of them are perfect :( The main contradiction is that during users start planning compaction based on the built planner, they may dynamically adjust the options parameters on certain conditions, such as

https://github.com/lancedb/lance/blob/254a8217ac26666585983aa7ec8c4234f4c3f99f/rust/lance/src/dataset/optimize.rs#L225

If options are passed in when building the planner, then after modifying the options subsequently, it must also be ensured that the options in the planner can be seen. Therefore, we need Arc + mutex and cannot use clone.

On the contrary, it might be simpler and more flexible to pass in the desired options each time the plan method is called here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If options are passed in when building the planner, then after modifying the options subsequently, it must also be ensured that the options in the planner can be seen. Therefore, we need Arc + mutex and cannot use clone.

I don't understand this. The logic of validate() can live in the planner and be internal.

If I were to rewrite compact_files, I would do:

pub async fn compact_files(
    dataset: &mut Dataset,
    mut options: CompactionOptions,
    remap_options: Option<Arc<dyn IndexRemapperOptions>>, // These will be deprecated later
) -> Result<CompactionMetrics> {
    info!(target: TRACE_DATASET_EVENTS, event=DATASET_COMPACTING_EVENT, uri = &dataset.uri);
    // .validate() now happens inside of `from_options`
    let planner = DefaultCompactionPlanner::from_options(options);

    compact_files_with_planner(dataset, &planner, remap_options).await
}

}

/// Formulate a plan to compact the files in a dataset
///
/// The compaction plan will contain a list of tasks to execute. Each task
/// will contain approximately `target_rows_per_fragment` rows and will be
/// rewriting fragments that are adjacent in the dataset's fragment list. Some
/// tasks may contain a single fragment when that fragment has deletions that
/// are being materialized and doesn't have any neighbors that need to be
/// compacted.
#[derive(Debug, Clone, Default)]
pub struct DefaultCompactionPlanner;

#[async_trait::async_trait]
impl CompactionPlanner for DefaultCompactionPlanner {
async fn plan(&self, dataset: &Dataset, options: &CompactionOptions) -> Result<CompactionPlan> {
let fragments = self.get_fragments(dataset, options);
debug_assert!(
fragments.windows(2).all(|w| w[0].id() < w[1].id()),
"fragments in manifest are not sorted"
);
let mut fragment_metrics = futures::stream::iter(fragments)
.map(|fragment| async move {
match collect_metrics(&fragment).await {
Ok(metrics) => Ok((fragment.metadata, metrics)),
Err(e) => Err(e),
}
})
.buffered(dataset.object_store().io_parallelism());

let index_fragmaps = load_index_fragmaps(dataset).await?;
let indices_containing_frag = |frag_id: u32| {
index_fragmaps
.iter()
.enumerate()
.filter(|(_, bitmap)| bitmap.contains(frag_id))
.map(|(pos, _)| pos)
.collect::<Vec<_>>()
};

let mut candidate_bins: Vec<CandidateBin> = Vec::new();
let mut current_bin: Option<CandidateBin> = None;
let mut i = 0;

while let Some(res) = fragment_metrics.next().await {
let (fragment, metrics) = res?;

let candidacy = if options.materialize_deletions
&& metrics.deletion_percentage() > options.materialize_deletions_threshold
{
Some(CompactionCandidacy::CompactItself)
} else if metrics.physical_rows < options.target_rows_per_fragment {
// Only want to compact if their are neighbors to compact such that
// we can get a larger fragment.
Some(CompactionCandidacy::CompactWithNeighbors)
} else {
// Not a candidate
None
};

let indices = indices_containing_frag(fragment.id as u32);

match (candidacy, &mut current_bin) {
(None, None) => {} // keep searching
(Some(candidacy), None) => {
// Start a new bin
current_bin = Some(CandidateBin {
fragments: vec![fragment],
pos_range: i..(i + 1),
candidacy: vec![candidacy],
row_counts: vec![metrics.num_rows()],
indices,
});
}
(Some(candidacy), Some(bin)) => {
// We cannot mix "indexed" and "non-indexed" fragments and so we only consider
// the existing bin if it contains the same indices
if bin.indices == indices {
// Add to current bin
bin.fragments.push(fragment);
bin.pos_range.end += 1;
bin.candidacy.push(candidacy);
bin.row_counts.push(metrics.num_rows());
} else {
// Index set is different. Complete previous bin and start new one
candidate_bins.push(current_bin.take().unwrap());
current_bin = Some(CandidateBin {
fragments: vec![fragment],
pos_range: i..(i + 1),
candidacy: vec![candidacy],
row_counts: vec![metrics.num_rows()],
indices,
});
}
}
(None, Some(_)) => {
// Bin is complete
candidate_bins.push(current_bin.take().unwrap());
}
}

i += 1;
}

// Flush the last bin
if let Some(bin) = current_bin {
candidate_bins.push(bin);
}

let final_bins = candidate_bins
.into_iter()
.filter(|bin| !bin.is_noop())
.flat_map(|bin| bin.split_for_size(options.target_rows_per_fragment))
.map(|bin| TaskData {
fragments: bin.fragments,
});

let mut compaction_plan = CompactionPlan::new(dataset.manifest.version, options.clone());
compaction_plan.extend_tasks(final_bins);

Ok(compaction_plan)
}
}

/// Compacts the files in the dataset without reordering them.
///
/// This does a few things:
/// By default, his does a few things:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/// By default, his does a few things:
/// By default, this does a few things:

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed.

/// * Removes deleted rows from fragments.
/// * Removes dropped columns from fragments.
/// * Merges fragments that are too small.
Expand All @@ -216,14 +364,24 @@ impl AddAssign for CompactionMetrics {
///
/// If no compaction is needed, this method will not make a new version of the table.
pub async fn compact_files(
dataset: &mut Dataset,
options: CompactionOptions,
remap_options: Option<Arc<dyn IndexRemapperOptions>>, // These will be deprecated later
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

) -> Result<CompactionMetrics> {
let planner = DefaultCompactionPlanner;
compact_files_with_planner(dataset, options, remap_options, &planner).await
}

pub async fn compact_files_with_planner(
dataset: &mut Dataset,
mut options: CompactionOptions,
remap_options: Option<Arc<dyn IndexRemapperOptions>>, // These will be deprecated later
planner: &dyn CompactionPlanner,
) -> Result<CompactionMetrics> {
info!(target: TRACE_DATASET_EVENTS, event=DATASET_COMPACTING_EVENT, uri = &dataset.uri);
options.validate();

let compaction_plan: CompactionPlan = plan_compaction(dataset, &options).await?;
let compaction_plan: CompactionPlan = planner.plan(dataset, &options).await?;

// If nothing to compact, don't make a commit.
if compaction_plan.tasks().is_empty() {
Expand Down Expand Up @@ -457,125 +615,12 @@ async fn load_index_fragmaps(dataset: &Dataset) -> Result<Vec<RoaringBitmap>> {
Ok(index_fragmaps)
}

/// Formulate a plan to compact the files in a dataset
///
/// The compaction plan will contain a list of tasks to execute. Each task
/// will contain approximately `target_rows_per_fragment` rows and will be
/// rewriting fragments that are adjacent in the dataset's fragment list. Some
/// tasks may contain a single fragment when that fragment has deletions that
/// are being materialized and doesn't have any neighbors that need to be
/// compacted.
pub async fn plan_compaction(
dataset: &Dataset,
options: &CompactionOptions,
) -> Result<CompactionPlan> {
// get_fragments should be returning fragments in sorted order (by id)
// and fragment ids should be unique
let fragments = dataset.get_fragments();
debug_assert!(
fragments.windows(2).all(|w| w[0].id() < w[1].id()),
"fragments in manifest are not sorted"
);
let mut fragment_metrics = futures::stream::iter(fragments)
.map(|fragment| async move {
match collect_metrics(&fragment).await {
Ok(metrics) => Ok((fragment.metadata, metrics)),
Err(e) => Err(e),
}
})
.buffered(dataset.object_store().io_parallelism());

let index_fragmaps = load_index_fragmaps(dataset).await?;
let indices_containing_frag = |frag_id: u32| {
index_fragmaps
.iter()
.enumerate()
.filter(|(_, bitmap)| bitmap.contains(frag_id))
.map(|(pos, _)| pos)
.collect::<Vec<_>>()
};

let mut candidate_bins: Vec<CandidateBin> = Vec::new();
let mut current_bin: Option<CandidateBin> = None;
let mut i = 0;

while let Some(res) = fragment_metrics.next().await {
let (fragment, metrics) = res?;

let candidacy = if options.materialize_deletions
&& metrics.deletion_percentage() > options.materialize_deletions_threshold
{
Some(CompactionCandidacy::CompactItself)
} else if metrics.physical_rows < options.target_rows_per_fragment {
// Only want to compact if their are neighbors to compact such that
// we can get a larger fragment.
Some(CompactionCandidacy::CompactWithNeighbors)
} else {
// Not a candidate
None
};

let indices = indices_containing_frag(fragment.id as u32);

match (candidacy, &mut current_bin) {
(None, None) => {} // keep searching
(Some(candidacy), None) => {
// Start a new bin
current_bin = Some(CandidateBin {
fragments: vec![fragment],
pos_range: i..(i + 1),
candidacy: vec![candidacy],
row_counts: vec![metrics.num_rows()],
indices,
});
}
(Some(candidacy), Some(bin)) => {
// We cannot mix "indexed" and "non-indexed" fragments and so we only consider
// the existing bin if it contains the same indices
if bin.indices == indices {
// Add to current bin
bin.fragments.push(fragment);
bin.pos_range.end += 1;
bin.candidacy.push(candidacy);
bin.row_counts.push(metrics.num_rows());
} else {
// Index set is different. Complete previous bin and start new one
candidate_bins.push(current_bin.take().unwrap());
current_bin = Some(CandidateBin {
fragments: vec![fragment],
pos_range: i..(i + 1),
candidacy: vec![candidacy],
row_counts: vec![metrics.num_rows()],
indices,
});
}
}
(None, Some(_)) => {
// Bin is complete
candidate_bins.push(current_bin.take().unwrap());
}
}

i += 1;
}

// Flush the last bin
if let Some(bin) = current_bin {
candidate_bins.push(bin);
}

let final_bins = candidate_bins
.into_iter()
.filter(|bin| !bin.is_noop())
.flat_map(|bin| bin.split_for_size(options.target_rows_per_fragment))
.map(|bin| TaskData {
fragments: bin.fragments,
});

let mut compaction_plan = CompactionPlan::new(dataset.manifest.version, options.clone());
compaction_plan.extend_tasks(final_bins);

Ok(compaction_plan)
let planner = DefaultCompactionPlanner;
planner.plan(dataset, options).await
}

/// The result of a single compaction task.
Expand Down Expand Up @@ -3519,4 +3564,37 @@ mod tests {
plan
);
}

#[tokio::test]
async fn test_default_compaction_planner() {
let test_dir = TempStrDir::default();
let test_uri = &test_dir;

let data = sample_data();
let schema = data.schema();

// Create dataset with multiple small fragments
let reader = RecordBatchIterator::new(vec![Ok(data.clone())], schema.clone());
let write_params = WriteParams {
max_rows_per_file: 2000,
..Default::default()
};
let dataset = Dataset::write(reader, test_uri, Some(write_params))
.await
.unwrap();

assert_eq!(dataset.get_fragments().len(), 5);

// Test default planner
let planner = DefaultCompactionPlanner;
let options = CompactionOptions {
target_rows_per_fragment: 5000,
..Default::default()
};
let plan = planner.plan(&dataset, &options).await.unwrap();

// Should create tasks to compact small fragments
assert!(!plan.tasks.is_empty());
assert_eq!(plan.read_version, dataset.manifest.version);
}
}
Loading