refactor: cache slices instead of vecs (#7989)
Immutable `Box<Vec<T>>`/`Arc<Vec<T>>` are better stored as `Box<[T]>`/`Arc<[T]>` because: - allocation always exact (no need for `shrink_to_fit`) - smaller (the fat pointer is just the memory address and the length, no capacity required) - less allocation (`Box`/`Arc` -> slice instead of `Box`/`Arc` -> `Vec` -> buffer); in fact the vector itself was offen missing in the accounting code Found while I was working on #7987. Co-authored-by: kodiakhq[bot] <49736102+kodiakhq[bot]@users.noreply.github.com>pull/24376/head
parent
b14d244a5d
commit
3e26567e05
|
@ -215,7 +215,7 @@ pub struct CachedTable {
|
|||
pub schema: Schema,
|
||||
pub column_id_map: HashMap<ColumnId, Arc<str>>,
|
||||
pub column_id_map_rev: HashMap<Arc<str>, ColumnId>,
|
||||
pub primary_key_column_ids: Vec<ColumnId>,
|
||||
pub primary_key_column_ids: Box<[ColumnId]>,
|
||||
}
|
||||
|
||||
impl CachedTable {
|
||||
|
@ -234,7 +234,7 @@ impl CachedTable {
|
|||
.keys()
|
||||
.map(|name| name.len())
|
||||
.sum::<usize>()
|
||||
+ (self.primary_key_column_ids.capacity() * size_of::<ColumnId>())
|
||||
+ (self.primary_key_column_ids.len() * size_of::<ColumnId>())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -259,7 +259,7 @@ impl From<TableSchema> for CachedTable {
|
|||
.collect();
|
||||
column_id_map_rev.shrink_to_fit();
|
||||
|
||||
let mut primary_key_column_ids: Vec<ColumnId> = schema
|
||||
let primary_key_column_ids = schema
|
||||
.primary_key()
|
||||
.into_iter()
|
||||
.map(|name| {
|
||||
|
@ -268,7 +268,6 @@ impl From<TableSchema> for CachedTable {
|
|||
.unwrap_or_else(|| panic!("primary key not known?!: {name}"))
|
||||
})
|
||||
.collect();
|
||||
primary_key_column_ids.shrink_to_fit();
|
||||
|
||||
Self {
|
||||
id,
|
||||
|
@ -394,7 +393,7 @@ mod tests {
|
|||
(Arc::from(col112.column.name.clone()), col112.column.id),
|
||||
(Arc::from(col113.column.name.clone()), col113.column.id),
|
||||
]),
|
||||
primary_key_column_ids: vec![col112.column.id, col113.column.id],
|
||||
primary_key_column_ids: [col112.column.id, col113.column.id].into(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
|
@ -415,7 +414,7 @@ mod tests {
|
|||
(Arc::from(col121.column.name.clone()), col121.column.id),
|
||||
(Arc::from(col122.column.name.clone()), col122.column.id),
|
||||
]),
|
||||
primary_key_column_ids: vec![col122.column.id],
|
||||
primary_key_column_ids: [col122.column.id].into(),
|
||||
}),
|
||||
),
|
||||
]),
|
||||
|
@ -447,7 +446,7 @@ mod tests {
|
|||
Arc::from(col211.column.name.clone()),
|
||||
col211.column.id,
|
||||
)]),
|
||||
primary_key_column_ids: vec![col211.column.id],
|
||||
primary_key_column_ids: [col211.column.id].into(),
|
||||
}),
|
||||
)]),
|
||||
};
|
||||
|
|
|
@ -38,13 +38,13 @@ pub enum Error {
|
|||
},
|
||||
}
|
||||
|
||||
type IngesterCounts = Option<Arc<Vec<(Uuid, u64)>>>;
|
||||
type IngesterCounts = Option<Arc<[(Uuid, u64)]>>;
|
||||
|
||||
/// Holds catalog information about a parquet file
|
||||
#[derive(Debug)]
|
||||
pub struct CachedParquetFiles {
|
||||
/// Parquet catalog information
|
||||
pub files: Arc<Vec<Arc<ParquetFile>>>,
|
||||
pub files: Arc<[Arc<ParquetFile>]>,
|
||||
|
||||
/// Number of persisted Parquet files per table ID per ingester UUID that ingesters have told
|
||||
/// us about. When a call to `get` includes a number of persisted Parquet files for this table
|
||||
|
@ -60,10 +60,10 @@ impl CachedParquetFiles {
|
|||
parquet_files: Vec<ParquetFile>,
|
||||
persisted_file_counts_from_ingesters: IngesterCounts,
|
||||
) -> Self {
|
||||
let files: Vec<_> = parquet_files.into_iter().map(Arc::new).collect();
|
||||
let files = parquet_files.into_iter().map(Arc::new).collect();
|
||||
|
||||
Self {
|
||||
files: Arc::new(files),
|
||||
files,
|
||||
persisted_file_counts_from_ingesters,
|
||||
}
|
||||
}
|
||||
|
@ -71,13 +71,13 @@ impl CachedParquetFiles {
|
|||
/// return the underlying files as a new Vec
|
||||
#[cfg(test)]
|
||||
fn vec(&self) -> Vec<Arc<ParquetFile>> {
|
||||
self.files.as_ref().clone()
|
||||
self.files.as_ref().to_vec()
|
||||
}
|
||||
|
||||
/// Estimate the memory consumption of this object and its contents
|
||||
fn size(&self) -> usize {
|
||||
// simplify accounting by ensuring len and capacity of vector are the same
|
||||
assert_eq!(self.files.len(), self.files.capacity());
|
||||
assert_eq!(self.files.len(), self.files.len());
|
||||
|
||||
// Note size_of_val is the size of the Arc
|
||||
// https://play.rust-lang.org/?version=stable&mode=debug&edition=2021&gist=ae8fee8b4f7f5f013dc01ea1fda165da
|
||||
|
@ -93,7 +93,7 @@ impl CachedParquetFiles {
|
|||
.as_ref()
|
||||
.map(|map| {
|
||||
std::mem::size_of_val(map.as_ref()) +
|
||||
map.capacity() * mem::size_of::<(Uuid, u64)>()
|
||||
map.len() * mem::size_of::<(Uuid, u64)>()
|
||||
}).unwrap_or_default()
|
||||
}
|
||||
}
|
||||
|
@ -227,8 +227,7 @@ impl ParquetFileCache {
|
|||
persisted_file_counts_by_ingester_uuid.map(|map| {
|
||||
let mut entries = map.into_iter().collect::<Vec<_>>();
|
||||
entries.sort();
|
||||
entries.shrink_to_fit();
|
||||
Arc::new(entries)
|
||||
entries.into()
|
||||
});
|
||||
let persisted_file_counts_by_ingester_uuid_captured =
|
||||
persisted_file_counts_by_ingester_uuid.clone();
|
||||
|
@ -246,7 +245,7 @@ impl ParquetFileCache {
|
|||
cached_file
|
||||
.persisted_file_counts_from_ingesters
|
||||
.as_ref()
|
||||
.map(|x| x.as_ref().as_ref()),
|
||||
.map(|x| x.as_ref()),
|
||||
ingester_counts,
|
||||
)
|
||||
} else {
|
||||
|
@ -361,7 +360,7 @@ mod tests {
|
|||
let table_id = table.table.id;
|
||||
|
||||
let single_file_size = 200;
|
||||
let two_file_size = 360;
|
||||
let two_file_size = 368;
|
||||
assert!(single_file_size < two_file_size);
|
||||
|
||||
let cache = make_cache(&catalog);
|
||||
|
|
|
@ -166,14 +166,14 @@ impl CachedPartition {
|
|||
pub struct PartitionSortKey {
|
||||
pub sort_key: Arc<SortKey>,
|
||||
pub column_set: HashSet<ColumnId>,
|
||||
pub column_order: Vec<ColumnId>,
|
||||
pub column_order: Box<[ColumnId]>,
|
||||
}
|
||||
|
||||
impl PartitionSortKey {
|
||||
fn new(sort_key: SortKey, column_id_map_rev: &HashMap<Arc<str>, ColumnId>) -> Self {
|
||||
let sort_key = Arc::new(sort_key);
|
||||
|
||||
let mut column_order: Vec<ColumnId> = sort_key
|
||||
let column_order: Box<[ColumnId]> = sort_key
|
||||
.iter()
|
||||
.map(|(name, _opts)| {
|
||||
*column_id_map_rev
|
||||
|
@ -181,7 +181,6 @@ impl PartitionSortKey {
|
|||
.unwrap_or_else(|| panic!("column_id_map_rev misses data: {name}"))
|
||||
})
|
||||
.collect();
|
||||
column_order.shrink_to_fit();
|
||||
|
||||
let mut column_set: HashSet<ColumnId> = column_order.iter().copied().collect();
|
||||
column_set.shrink_to_fit();
|
||||
|
@ -198,7 +197,7 @@ impl PartitionSortKey {
|
|||
size_of_val(self)
|
||||
+ self.sort_key.as_ref().size()
|
||||
+ (self.column_set.capacity() * size_of::<ColumnId>())
|
||||
+ (self.column_order.capacity() * size_of::<ColumnId>())
|
||||
+ (self.column_order.len() * size_of::<ColumnId>())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -239,7 +238,7 @@ mod tests {
|
|||
(Arc::from(c1.column.name.clone()), c1.column.id),
|
||||
(Arc::from(c2.column.name.clone()), c2.column.id),
|
||||
]),
|
||||
primary_key_column_ids: vec![c1.column.id, c2.column.id],
|
||||
primary_key_column_ids: [c1.column.id, c2.column.id].into(),
|
||||
});
|
||||
|
||||
let cache = PartitionCache::new(
|
||||
|
@ -259,7 +258,7 @@ mod tests {
|
|||
&PartitionSortKey {
|
||||
sort_key: Arc::new(p1.sort_key().unwrap()),
|
||||
column_set: HashSet::from([c1.column.id, c2.column.id]),
|
||||
column_order: vec![c1.column.id, c2.column.id],
|
||||
column_order: [c1.column.id, c2.column.id].into(),
|
||||
}
|
||||
);
|
||||
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 1);
|
||||
|
@ -320,7 +319,7 @@ mod tests {
|
|||
(Arc::from(c1.column.name.clone()), c1.column.id),
|
||||
(Arc::from(c2.column.name.clone()), c2.column.id),
|
||||
]),
|
||||
primary_key_column_ids: vec![c1.column.id, c2.column.id],
|
||||
primary_key_column_ids: [c1.column.id, c2.column.id].into(),
|
||||
});
|
||||
|
||||
let cache = PartitionCache::new(
|
||||
|
@ -370,7 +369,7 @@ mod tests {
|
|||
(Arc::from(c1.column.name.clone()), c1.column.id),
|
||||
(Arc::from(c2.column.name.clone()), c2.column.id),
|
||||
]),
|
||||
primary_key_column_ids: vec![c1.column.id, c2.column.id],
|
||||
primary_key_column_ids: [c1.column.id, c2.column.id].into(),
|
||||
});
|
||||
|
||||
let cache = PartitionCache::new(
|
||||
|
@ -421,7 +420,7 @@ mod tests {
|
|||
&PartitionSortKey {
|
||||
sort_key: Arc::new(p_sort_key.clone().unwrap()),
|
||||
column_set: HashSet::from([c1.column.id, c2.column.id]),
|
||||
column_order: vec![c1.column.id, c2.column.id],
|
||||
column_order: [c1.column.id, c2.column.id].into(),
|
||||
}
|
||||
);
|
||||
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 4);
|
||||
|
|
|
@ -29,7 +29,7 @@ const CACHE_ID: &str = "projected_schema";
|
|||
#[derive(Debug, Clone, Hash, PartialEq, Eq, PartialOrd, Ord)]
|
||||
struct CacheKey {
|
||||
table_id: TableId,
|
||||
projection: Vec<ColumnId>,
|
||||
projection: Box<[ColumnId]>,
|
||||
}
|
||||
|
||||
impl CacheKey {
|
||||
|
@ -40,18 +40,15 @@ impl CacheKey {
|
|||
// normalize column order
|
||||
projection.sort();
|
||||
|
||||
// ensure that cache key is as small as possible
|
||||
projection.shrink_to_fit();
|
||||
|
||||
Self {
|
||||
table_id,
|
||||
projection,
|
||||
projection: projection.into(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Size in of key including `Self`.
|
||||
fn size(&self) -> usize {
|
||||
size_of_val(self) + self.projection.capacity() * size_of::<ColumnId>()
|
||||
size_of_val(self) + self.projection.len() * size_of::<ColumnId>()
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -207,37 +204,40 @@ mod tests {
|
|||
schema: table_schema_a.clone(),
|
||||
column_id_map: column_id_map_a.clone(),
|
||||
column_id_map_rev: reverse_map(&column_id_map_a),
|
||||
primary_key_column_ids: vec![
|
||||
primary_key_column_ids: [
|
||||
ColumnId::new(1),
|
||||
ColumnId::new(2),
|
||||
ColumnId::new(3),
|
||||
ColumnId::new(4),
|
||||
],
|
||||
]
|
||||
.into(),
|
||||
});
|
||||
let table_1b = Arc::new(CachedTable {
|
||||
id: table_id_1,
|
||||
schema: table_schema_b.clone(),
|
||||
column_id_map: column_id_map_b.clone(),
|
||||
column_id_map_rev: reverse_map(&column_id_map_b),
|
||||
primary_key_column_ids: vec![
|
||||
primary_key_column_ids: [
|
||||
ColumnId::new(1),
|
||||
ColumnId::new(2),
|
||||
ColumnId::new(3),
|
||||
ColumnId::new(4),
|
||||
],
|
||||
]
|
||||
.into(),
|
||||
});
|
||||
let table_2a = Arc::new(CachedTable {
|
||||
id: table_id_2,
|
||||
schema: table_schema_a.clone(),
|
||||
column_id_map: column_id_map_a.clone(),
|
||||
column_id_map_rev: reverse_map(&column_id_map_a),
|
||||
primary_key_column_ids: vec![
|
||||
primary_key_column_ids: [
|
||||
ColumnId::new(1),
|
||||
ColumnId::new(2),
|
||||
ColumnId::new(3),
|
||||
ColumnId::new(4),
|
||||
ColumnId::new(5),
|
||||
],
|
||||
]
|
||||
.into(),
|
||||
});
|
||||
|
||||
// initial request
|
||||
|
|
|
@ -62,7 +62,7 @@ impl ChunkAdapter {
|
|||
pub(crate) async fn new_chunks(
|
||||
&self,
|
||||
cached_table: Arc<CachedTable>,
|
||||
files: Arc<Vec<Arc<ParquetFile>>>,
|
||||
files: Arc<[Arc<ParquetFile>]>,
|
||||
predicate: &Predicate,
|
||||
early_pruning_observer: MetricPruningObserver,
|
||||
span: Option<Span>,
|
||||
|
|
|
@ -240,7 +240,7 @@ pub mod tests {
|
|||
self.adapter
|
||||
.new_chunks(
|
||||
Arc::clone(cached_table),
|
||||
Arc::new(vec![Arc::clone(&self.parquet_file)]),
|
||||
vec![Arc::clone(&self.parquet_file)].into(),
|
||||
&Predicate::new(),
|
||||
MetricPruningObserver::new_unregistered(),
|
||||
None,
|
||||
|
|
Loading…
Reference in New Issue