From 5198c8079712b66809d40f6ea44d7717558f1e30 Mon Sep 17 00:00:00 2001 From: Clement Rey Date: Thu, 25 Apr 2024 18:47:43 +0200 Subject: [PATCH] clusterkey-less datastore --- crates/re_data_store/benches/arrow2.rs | 19 ++- crates/re_data_store/benches/data_store.rs | 16 +- crates/re_data_store/benches/gc.rs | 14 +- crates/re_data_store/src/lib.rs | 3 +- crates/re_data_store/src/store.rs | 93 +---------- crates/re_data_store/src/store_arrow.rs | 61 +------ crates/re_data_store/src/store_dump.rs | 29 +--- crates/re_data_store/src/store_event.rs | 76 ++------- crates/re_data_store/src/store_format.rs | 7 - crates/re_data_store/src/store_gc.rs | 88 ++-------- crates/re_data_store/src/store_helpers.rs | 2 - crates/re_data_store/src/store_read.rs | 42 +---- crates/re_data_store/src/store_sanity.rs | 33 +--- crates/re_data_store/src/store_stats.rs | 28 +--- crates/re_data_store/src/store_subscriber.rs | 9 +- crates/re_data_store/src/store_write.rs | 159 ++----------------- crates/re_data_store/src/test_util.rs | 9 +- crates/re_data_store/tests/correctness.rs | 104 +++--------- crates/re_data_store/tests/data_store.rs | 122 ++++++-------- crates/re_data_store/tests/dump.rs | 36 ++--- crates/re_data_store/tests/internals.rs | 17 +- crates/re_data_store/tests/memory_test.rs | 6 +- 22 files changed, 178 insertions(+), 795 deletions(-) diff --git a/crates/re_data_store/benches/arrow2.rs b/crates/re_data_store/benches/arrow2.rs index dee0628802d02..236e3db60c3fc 100644 --- a/crates/re_data_store/benches/arrow2.rs +++ b/crates/re_data_store/benches/arrow2.rs @@ -9,10 +9,11 @@ use arrow2::array::{Array, FixedSizeListArray, PrimitiveArray, StructArray}; use criterion::Criterion; use itertools::Itertools; +use re_log_types::example_components::MyIndex; use re_log_types::DataCell; -use re_types::datagen::{build_some_instances, build_some_positions2d}; +use re_types::datagen::build_some_positions2d; use re_types::{ - components::{InstanceKey, Position2D}, + components::Position2D, testing::{build_some_large_structs, LargeStruct}, }; use re_types_core::{Component, SizeBytes}; @@ -40,7 +41,7 @@ const NUM_INSTANCES: usize = 1; #[derive(Debug, Clone, Copy)] enum ArrayKind { - /// E.g. an array of `InstanceKey`. + /// E.g. an array of `MyIndex`. Primitive, /// E.g. an array of `Position2D`. @@ -79,7 +80,7 @@ fn erased_clone(c: &mut Criterion) { match kind { ArrayKind::Primitive => { - let data = build_some_instances(NUM_INSTANCES); + let data = MyIndex::from_iter(0..NUM_INSTANCES as _); bench_arrow(&mut group, &data); bench_native(&mut group, &data); } @@ -198,7 +199,9 @@ fn estimated_size_bytes(c: &mut Criterion) { fn generate_cells(kind: ArrayKind) -> Vec { match kind { ArrayKind::Primitive => (0..NUM_ROWS) - .map(|_| DataCell::from_native(build_some_instances(NUM_INSTANCES).as_slice())) + .map(|_| { + DataCell::from_native(MyIndex::from_iter(0..NUM_INSTANCES as _).as_slice()) + }) .collect(), ArrayKind::Struct => (0..NUM_ROWS) .map(|_| { @@ -312,9 +315,9 @@ fn estimated_size_bytes(c: &mut Criterion) { .collect() } - fn generate_keys() -> Vec> { + fn generate_indices() -> Vec> { (0..NUM_ROWS) - .map(|_| build_some_instances(NUM_INSTANCES)) + .map(|_| MyIndex::from_iter(0..NUM_INSTANCES as _)) .collect() } @@ -325,7 +328,7 @@ fn estimated_size_bytes(c: &mut Criterion) { } match kind { - ArrayKind::Primitive => bench_std(&mut group, generate_keys()), + ArrayKind::Primitive => bench_std(&mut group, generate_indices()), ArrayKind::Struct => bench_std(&mut group, generate_positions()), ArrayKind::StructLarge => bench_std(&mut group, generate_rects()), } diff --git a/crates/re_data_store/benches/data_store.rs b/crates/re_data_store/benches/data_store.rs index 8b95671180a3a..4d6b81cb33331 100644 --- a/crates/re_data_store/benches/data_store.rs +++ b/crates/re_data_store/benches/data_store.rs @@ -9,14 +9,10 @@ use re_data_store::{ RangeQuery, TimeInt, TimeRange, }; use re_log_types::{ - build_frame_nr, DataCell, DataRow, DataTable, EntityPath, RowId, TableId, TimePoint, TimeType, - Timeline, -}; -use re_types::datagen::build_some_instances; -use re_types::{ - components::InstanceKey, - testing::{build_some_large_structs, LargeStruct}, + build_frame_nr, example_components::MyIndex, DataCell, DataRow, DataTable, EntityPath, RowId, + TableId, TimePoint, TimeType, Timeline, }; +use re_types::testing::{build_some_large_structs, LargeStruct}; use re_types_core::{ComponentName, Loggable as _}; criterion_group!( @@ -123,7 +119,6 @@ fn insert_same_time_point(c: &mut Criterion) { b.iter(|| { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig::default(), ); @@ -398,9 +393,8 @@ fn build_rows_ex( RowId::new(), "large_structs", time_point(frame_idx), - num_instances as _, ( - build_some_instances(num_instances), + MyIndex::from_iter(0..num_instances as _), build_some_large_structs(num_instances), ), ) @@ -432,10 +426,8 @@ fn build_rows_ex( } fn insert_rows(config: DataStoreConfig, rows: &[DataRow]) -> DataStore { - let cluster_key = InstanceKey::name(); let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - cluster_key, config, ); for row in rows { diff --git a/crates/re_data_store/benches/gc.rs b/crates/re_data_store/benches/gc.rs index ea1aaab316edb..3e840aac1c676 100644 --- a/crates/re_data_store/benches/gc.rs +++ b/crates/re_data_store/benches/gc.rs @@ -10,8 +10,7 @@ use re_data_store::{ use re_log_types::{ build_frame_nr, build_log_time, DataRow, DataTable, EntityPath, RowId, TableId, Time, TimePoint, }; -use re_types::components::InstanceKey; -use re_types_core::{AsComponents, ComponentBatch, ComponentName, Loggable as _}; +use re_types_core::{AsComponents, ComponentBatch}; criterion_group!(benches, plotting_dashboard); criterion_main!(benches); @@ -85,13 +84,7 @@ fn plotting_dashboard(c: &mut Criterion) { // Default config group.bench_function("default", |b| { - let store = build_store( - Default::default(), - InstanceKey::name(), - false, - &mut timegen, - &mut datagen, - ); + let store = build_store(Default::default(), false, &mut timegen, &mut datagen); b.iter_batched( || store.clone(), |mut store| { @@ -117,7 +110,6 @@ fn plotting_dashboard(c: &mut Criterion) { indexed_bucket_num_rows: num_rows_per_bucket, ..Default::default() }, - InstanceKey::name(), false, &mut timegen, &mut datagen, @@ -142,7 +134,6 @@ fn plotting_dashboard(c: &mut Criterion) { fn build_store( config: DataStoreConfig, - cluster_key: ComponentName, packed: bool, timegen: &mut FT, datagen: &mut FD, @@ -153,7 +144,6 @@ where { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - cluster_key, config, ); diff --git a/crates/re_data_store/src/lib.rs b/crates/re_data_store/src/lib.rs index 07227c54a0f1d..19da79f88717b 100644 --- a/crates/re_data_store/src/lib.rs +++ b/crates/re_data_store/src/lib.rs @@ -41,8 +41,7 @@ pub use self::store_subscriber::{StoreSubscriber, StoreSubscriberHandle}; pub use self::store_write::{WriteError, WriteResult}; pub(crate) use self::store::{ - ClusterCellCache, IndexedBucket, IndexedBucketInner, IndexedTable, MetadataRegistry, - StaticCell, StaticTable, + IndexedBucket, IndexedBucketInner, IndexedTable, MetadataRegistry, StaticCell, StaticTable, }; // Re-exports diff --git a/crates/re_data_store/src/store.rs b/crates/re_data_store/src/store.rs index fbf0515b8fcc2..528018ee9ba0c 100644 --- a/crates/re_data_store/src/store.rs +++ b/crates/re_data_store/src/store.rs @@ -5,8 +5,8 @@ use arrow2::datatypes::DataType; use nohash_hasher::IntMap; use parking_lot::RwLock; use re_log_types::{ - DataCell, DataCellColumn, EntityPath, EntityPathHash, ErasedTimeVec, NumInstances, - NumInstancesVec, RowId, RowIdVec, StoreId, TimeInt, TimePoint, TimeRange, Timeline, + DataCell, DataCellColumn, EntityPath, EntityPathHash, ErasedTimeVec, RowId, RowIdVec, StoreId, + TimeInt, TimePoint, TimeRange, Timeline, }; use re_types_core::{ComponentName, ComponentNameSet, SizeBytes}; @@ -123,27 +123,6 @@ impl std::ops::DerefMut for MetadataRegistry { } } -/// Used to cache auto-generated cluster cells (`[0]`, `[0, 1]`, `[0, 1, 2]`, …) so that they -/// can be properly deduplicated on insertion. -#[derive(Debug, Default, Clone)] -pub struct ClusterCellCache(pub IntMap); - -impl std::ops::Deref for ClusterCellCache { - type Target = IntMap; - - #[inline] - fn deref(&self) -> &Self::Target { - &self.0 - } -} - -impl std::ops::DerefMut for ClusterCellCache { - #[inline] - fn deref_mut(&mut self) -> &mut Self::Target { - &mut self.0 - } -} - // --- /// Incremented on each edit. @@ -164,20 +143,6 @@ pub struct StoreGeneration { pub struct DataStore { pub(crate) id: StoreId, - /// The cluster key specifies a column/component that is guaranteed to always be present for - /// every single row of data within the store. - /// - /// In addition to always being present, the payload of the cluster key..: - /// - is always increasingly sorted, - /// - is always dense (no validity bitmap), - /// - and never contains duplicate entries. - /// - /// This makes the cluster key a perfect candidate for joining query results together, and - /// doing so as efficiently as possible. - /// - /// See [`Self::insert_row`] for more information. - pub(crate) cluster_key: ComponentName, - /// The configuration of the data store (e.g. bucket sizes). pub(crate) config: DataStoreConfig, @@ -192,10 +157,6 @@ pub struct DataStore { /// Keeps track of arbitrary per-row metadata. pub(crate) metadata_registry: MetadataRegistry<(TimePoint, EntityPathHash)>, - /// Used to cache auto-generated cluster cells (`[0]`, `[0, 1]`, `[0, 1, 2]`, …) - /// so that they can be properly deduplicated on insertion. - pub(crate) cluster_cell_cache: ClusterCellCache, - /// All temporal [`IndexedTable`]s for all entities on all timelines. /// /// See also [`Self::static_tables`]. @@ -227,11 +188,9 @@ impl Clone for DataStore { fn clone(&self) -> Self { Self { id: self.id.clone(), - cluster_key: self.cluster_key, config: self.config.clone(), type_registry: self.type_registry.clone(), metadata_registry: self.metadata_registry.clone(), - cluster_cell_cache: self.cluster_cell_cache.clone(), tables: self.tables.clone(), static_tables: self.static_tables.clone(), insert_id: Default::default(), @@ -243,13 +202,10 @@ impl Clone for DataStore { } impl DataStore { - /// See [`Self::cluster_key`] for more information about the cluster key. - pub fn new(id: StoreId, cluster_key: ComponentName, config: DataStoreConfig) -> Self { + pub fn new(id: StoreId, config: DataStoreConfig) -> Self { Self { id, - cluster_key, config, - cluster_cell_cache: Default::default(), type_registry: Default::default(), metadata_registry: Default::default(), tables: Default::default(), @@ -283,11 +239,6 @@ impl DataStore { } } - /// See [`Self::cluster_key`] for more information about the cluster key. - pub fn cluster_key(&self) -> ComponentName { - self.cluster_key - } - /// See [`DataStoreConfig`] for more information about configuration. pub fn config(&self) -> &DataStoreConfig { &self.config @@ -355,10 +306,6 @@ pub struct IndexedTable { /// The entity this table is related to, for debugging purposes. pub entity_path: EntityPath, - /// Carrying the cluster key around to help with assertions and sanity checks all over the - /// place. - pub cluster_key: ComponentName, - /// The actual buckets, where the data is stored. /// /// The keys of this `BTreeMap` represent the lower bounds of the time-ranges covered by @@ -388,14 +335,13 @@ pub struct IndexedTable { } impl IndexedTable { - pub fn new(cluster_key: ComponentName, timeline: Timeline, entity_path: EntityPath) -> Self { - let bucket = IndexedBucket::new(cluster_key, timeline); + pub fn new(timeline: Timeline, entity_path: EntityPath) -> Self { + let bucket = IndexedBucket::new(timeline); let buckets_size_bytes = bucket.total_size_bytes(); Self { timeline, entity_path, buckets: [(TimeInt::MIN, bucket)].into(), - cluster_key, all_components: Default::default(), buckets_num_rows: 0, buckets_size_bytes, @@ -412,14 +358,13 @@ impl IndexedTable { let Self { timeline, entity_path: _, - cluster_key, buckets, all_components: _, // keep the history on purpose buckets_num_rows, buckets_size_bytes, } = self; - let bucket = IndexedBucket::new(*cluster_key, *timeline); + let bucket = IndexedBucket::new(*timeline); let size_bytes = bucket.total_size_bytes(); *buckets = [(TimeInt::MIN, bucket)].into(); @@ -441,10 +386,6 @@ pub struct IndexedBucket { /// The timeline the bucket's parent table operates in, for debugging purposes. pub timeline: Timeline, - /// Carrying the cluster key around to help with assertions and sanity checks all over the - /// place. - pub cluster_key: ComponentName, - // To simplify interior mutability. pub inner: RwLock, } @@ -453,18 +394,16 @@ impl Clone for IndexedBucket { fn clone(&self) -> Self { Self { timeline: self.timeline, - cluster_key: self.cluster_key, inner: RwLock::new(self.inner.read().clone()), } } } impl IndexedBucket { - pub(crate) fn new(cluster_key: ComponentName, timeline: Timeline) -> Self { + pub(crate) fn new(timeline: Timeline) -> Self { Self { timeline, inner: RwLock::new(IndexedBucketInner::default()), - cluster_key, } } } @@ -503,11 +442,6 @@ pub struct IndexedBucketInner { /// `RowId::ZERO` for empty buckets. pub max_row_id: RowId, - /// The entire column of `num_instances`. - /// - /// Keeps track of the expected number of instances in each row. - pub col_num_instances: NumInstancesVec, - /// All the rows for all the component columns. /// /// The cells are optional since not all rows will have data for every single component @@ -533,7 +467,6 @@ impl Default for IndexedBucketInner { col_insert_id: Default::default(), col_row_id: Default::default(), max_row_id: RowId::ZERO, - col_num_instances: Default::default(), columns: Default::default(), size_bytes: 0, // NOTE: computed below }; @@ -550,10 +483,6 @@ pub struct StaticTable { /// The entity this table is related to, for debugging purposes. pub entity_path: EntityPath, - /// Carrying the cluster key around to help with assertions and sanity checks all over the - /// place. - pub cluster_key: ComponentName, - /// Keeps track of one and only one [`StaticCell`] per component. /// /// Last-write-wins semantics apply, where ordering is defined by `RowId`. @@ -562,10 +491,9 @@ pub struct StaticTable { impl StaticTable { #[inline] - pub fn new(cluster_key: ComponentName, entity_path: EntityPath) -> Self { + pub fn new(entity_path: EntityPath) -> Self { Self { entity_path, - cluster_key, cells: Default::default(), } } @@ -577,10 +505,5 @@ pub struct StaticCell { pub insert_id: Option, pub row_id: RowId, - pub num_instances: NumInstances, pub cell: DataCell, - - // TODO(#5303): We keep track of cluster keys for each static cell for backwards - // compatibility with the legacy instance-key model. This will go away next. - pub cluster_key: DataCell, } diff --git a/crates/re_data_store/src/store_arrow.rs b/crates/re_data_store/src/store_arrow.rs index 60a792fada5c3..eb398f1155f77 100644 --- a/crates/re_data_store/src/store_arrow.rs +++ b/crates/re_data_store/src/store_arrow.rs @@ -2,7 +2,7 @@ use std::collections::{BTreeMap, VecDeque}; use arrow2::{array::Array, chunk::Chunk, datatypes::Schema}; use nohash_hasher::IntMap; -use re_log_types::{DataCellColumn, DataTable, DataTableResult, NumInstances, RowId, Timeline}; +use re_log_types::{DataCellColumn, DataTable, DataTableResult, RowId, Timeline}; use re_types_core::ComponentName; use crate::{ @@ -19,17 +19,11 @@ impl IndexedBucket { /// - `insert_id` /// - `row_id` /// - `time` - /// - `num_instances` - /// - `$cluster_key` /// - rest of component columns in ascending lexical order pub fn serialize(&self) -> DataTableResult<(Schema, Chunk>)> { re_tracing::profile_function!(); - let Self { - timeline, - cluster_key, - inner, - } = self; + let Self { timeline, inner } = self; let IndexedBucketInner { is_sorted: _, @@ -38,17 +32,14 @@ impl IndexedBucket { col_insert_id, col_row_id, max_row_id: _, - col_num_instances, columns, size_bytes: _, } = &*inner.read(); serialize( - cluster_key, Some((*timeline, col_time)), col_insert_id, col_row_id, - col_num_instances, columns, ) } @@ -61,15 +52,10 @@ impl StaticTable { /// - `insert_id` /// - `row_id` /// - `time` - /// - `num_instances` - /// - `$cluster_key` /// - rest of component columns in ascending lexical order pub fn serialize(&self) -> DataTableResult<(Schema, Chunk>)> { re_tracing::profile_function!(); - // NOTE: cannot fail, the cluster key _has_ to be there by definition - let cluster_keys = &self.cells[&self.cluster_key]; - let mut cells_per_row_id: BTreeMap> = Default::default(); for static_cell in self.cells.values() { cells_per_row_id @@ -77,9 +63,6 @@ impl StaticTable { .or_default() .push(static_cell.clone()); } - for cells in cells_per_row_id.values_mut() { - cells.push(cluster_keys.clone()); - } let col_insert_id = cells_per_row_id .values() @@ -88,11 +71,6 @@ impl StaticTable { let col_row_id = cells_per_row_id.keys().copied().collect(); - let col_num_instances = cells_per_row_id - .values() - .filter_map(|cells| cells.first().map(|cell| cell.num_instances)) - .collect(); - let component_names: Vec<_> = self .cells .values() @@ -113,25 +91,16 @@ impl StaticTable { } } - serialize( - &self.cluster_key, - None, - &col_insert_id, - &col_row_id, - &col_num_instances, - &columns, - ) + serialize(None, &col_insert_id, &col_row_id, &columns) } } // --- fn serialize( - cluster_key: &ComponentName, col_time: Option<(Timeline, &VecDeque)>, col_insert_id: &VecDeque, col_row_id: &VecDeque, - col_num_instances: &VecDeque, table: &IntMap, ) -> DataTableResult<(Schema, Chunk>)> { re_tracing::profile_function!(); @@ -146,14 +115,14 @@ fn serialize( { let (control_schema, control_columns) = - serialize_control_columns(col_time, col_insert_id, col_row_id, col_num_instances)?; + serialize_control_columns(col_time, col_insert_id, col_row_id)?; schema.fields.extend(control_schema.fields); schema.metadata.extend(control_schema.metadata); columns.extend(control_columns); } { - let (data_schema, data_columns) = serialize_data_columns(cluster_key, table)?; + let (data_schema, data_columns) = serialize_data_columns(table)?; schema.fields.extend(data_schema.fields); schema.metadata.extend(data_schema.metadata); columns.extend(data_columns); @@ -166,7 +135,6 @@ fn serialize_control_columns( col_time: Option<(Timeline, &VecDeque)>, col_insert_id: &VecDeque, col_row_id: &VecDeque, - col_num_instances: &VecDeque, ) -> DataTableResult<(Schema, Vec>)> { re_tracing::profile_function!(); @@ -177,7 +145,6 @@ fn serialize_control_columns( // - insert_id // - row_id // - time - // - num_instances // NOTE: Optional column, so make sure it's actually there: if !col_insert_id.is_empty() { @@ -204,16 +171,10 @@ fn serialize_control_columns( columns.push(time_column); } - let (num_instances_field, num_instances_column) = - DataTable::serialize_control_column(col_num_instances)?; - schema.fields.push(num_instances_field); - columns.push(num_instances_column); - Ok((schema, columns)) } fn serialize_data_columns( - cluster_key: &ComponentName, table: &IntMap, ) -> DataTableResult<(Schema, Vec>)> { re_tracing::profile_function!(); @@ -222,17 +183,7 @@ fn serialize_data_columns( let mut columns = Vec::new(); // NOTE: ordering is taken into account! - let mut table: BTreeMap<_, _> = table.iter().collect(); - - // Cluster column first and foremost! - // - // NOTE: cannot fail, the cluster key _has_ to be there by definition - let cluster_column = table.remove(&cluster_key).unwrap(); - { - let (field, column) = DataTable::serialize_data_column(cluster_key, cluster_column)?; - schema.fields.push(field); - columns.push(column); - } + let table: BTreeMap<_, _> = table.iter().collect(); for (component, column) in table { // NOTE: Don't serialize columns with only null values. diff --git a/crates/re_data_store/src/store_dump.rs b/crates/re_data_store/src/store_dump.rs index f44b31b351282..2e1e24fb9ffc3 100644 --- a/crates/re_data_store/src/store_dump.rs +++ b/crates/re_data_store/src/store_dump.rs @@ -63,7 +63,6 @@ impl DataStore { /// Serializes the entire datastore into an iterator of [`DataTable`]s, where each table /// corresponds 1-to-1 to an internal bucket. - // TODO(#1793): This shouldn't dump cluster keys that were autogenerated. pub fn to_data_tables( &self, time_filter: Option<(Timeline, TimeRange)>, @@ -95,14 +94,7 @@ impl DataStore { row_id, TimePoint::default(), static_table.entity_path.clone(), - static_cells - .iter() - .map(|cell| cell.num_instances.0) - .max() - .unwrap_or_default(), - static_cells - .into_iter() - .flat_map(|static_cell| [static_cell.cell, static_cell.cluster_key]), + static_cells.into_iter().map(|static_cell| static_cell.cell), ) .ok() }); @@ -120,11 +112,7 @@ impl DataStore { bucket.sort_indices_if_needed(); - let IndexedBucket { - timeline, - cluster_key: _, - inner, - } = bucket; + let IndexedBucket { timeline, inner } = bucket; let IndexedBucketInner { is_sorted: _, @@ -133,7 +121,6 @@ impl DataStore { col_insert_id: _, col_row_id, max_row_id: _, - col_num_instances, columns, size_bytes: _, } = &*inner.read(); @@ -146,7 +133,6 @@ impl DataStore { col_entity_path: std::iter::repeat_with(|| table.entity_path.clone()) .take(col_row_id.len()) .collect(), - col_num_instances: col_num_instances.clone(), columns: columns.clone().into_iter().collect(), // shallow } }) @@ -171,11 +157,7 @@ impl DataStore { bucket.sort_indices_if_needed(); - let IndexedBucket { - timeline, - cluster_key: _, - inner, - } = bucket; + let IndexedBucket { timeline, inner } = bucket; let IndexedBucketInner { is_sorted: _, @@ -184,7 +166,6 @@ impl DataStore { col_insert_id: _, col_row_id, max_row_id: _, - col_num_instances, columns, size_bytes: _, } = &*inner.read(); @@ -215,9 +196,6 @@ impl DataStore { .take(col_row_id.len()) .collect(); - let col_num_instances = - filter_column(col_time, col_num_instances.iter(), time_filter).collect(); - let mut columns2 = BTreeMap::default(); for (component, column) in columns { let column = filter_column(col_time, column.iter(), time_filter).collect(); @@ -229,7 +207,6 @@ impl DataStore { col_row_id, col_timelines, col_entity_path, - col_num_instances, columns: columns2, }) })) diff --git a/crates/re_data_store/src/store_event.rs b/crates/re_data_store/src/store_event.rs index b4e2b6de4b656..971c47a829bb1 100644 --- a/crates/re_data_store/src/store_event.rs +++ b/crates/re_data_store/src/store_event.rs @@ -211,10 +211,10 @@ mod tests { use std::collections::BTreeMap; use re_log_types::{ - example_components::{MyColor, MyPoint}, + example_components::{MyColor, MyIndex, MyPoint}, DataRow, RowId, TimePoint, Timeline, }; - use re_types_core::{components::InstanceKey, Loggable as _}; + use re_types_core::Loggable as _; use crate::{DataStore, GarbageCollectionOptions}; @@ -285,7 +285,6 @@ mod tests { fn store_events() -> anyhow::Result<()> { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -306,7 +305,7 @@ mod tests { row_id1, timepoint1.clone(), entity_path1.clone(), - [&InstanceKey::from_iter(0..10) as _], + [&MyIndex::from_iter(0..10) as _], )?; view.on_events(&[store.insert_row(&row1)?]); @@ -325,7 +324,7 @@ mod tests { (entity_path1.clone(), 1), // ], [ - (InstanceKey::name(), 1), // + (MyIndex::name(), 1), // ], [ (42.try_into().unwrap(), 1), // @@ -375,9 +374,9 @@ mod tests { (entity_path2.clone(), 1), // ], [ - (InstanceKey::name(), 1), // autogenerated, doesn't change - (MyPoint::name(), 1), // - (MyColor::name(), 1), // + (MyIndex::name(), 1), // autogenerated, doesn't change + (MyPoint::name(), 1), // + (MyColor::name(), 1), // ], [ (42.try_into().unwrap(), 2), // @@ -399,7 +398,7 @@ mod tests { timepoint3.clone(), entity_path2.clone(), [ - &InstanceKey::from_iter(0..num_instances as _) as _, + &MyIndex::from_iter(0..num_instances as _) as _, &colors as _, ], )? @@ -424,9 +423,9 @@ mod tests { (entity_path2.clone(), 2), // ], [ - (InstanceKey::name(), 2), // - (MyPoint::name(), 1), // - (MyColor::name(), 2), // + (MyIndex::name(), 2), // + (MyPoint::name(), 1), // + (MyColor::name(), 2), // ], [ (42.try_into().unwrap(), 2), // @@ -458,9 +457,9 @@ mod tests { (entity_path2.clone(), 1), // static -- no gc ], [ - (InstanceKey::name(), 1), // static -- no gc - (MyPoint::name(), 0), // - (MyColor::name(), 1), // static -- no gc + (MyIndex::name(), 1), // static -- no gc + (MyPoint::name(), 0), // + (MyColor::name(), 1), // static -- no gc ], [ (42.try_into().unwrap(), 0), // @@ -474,51 +473,4 @@ mod tests { Ok(()) } - - #[test] - fn autogenerated_cluster_keys() -> anyhow::Result<()> { - let mut store = DataStore::new( - re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), - Default::default(), - ); - - let timeline_frame = Timeline::new_sequence("frame"); - - let row1 = DataRow::from_component_batches( - RowId::new(), - TimePoint::from_iter([(timeline_frame, 42)]), - "entity_a".into(), - [&InstanceKey::from_iter(0..10) as _], - )?; - - // Not autogenerated, should fire. - { - let event = store.insert_row(&row1)?; - assert!(event.cells.contains_key(&store.cluster_key())); - - let (events, _) = store.gc(&GarbageCollectionOptions::gc_everything()); - assert_eq!(1, events.len()); - assert!(events[0].cells.contains_key(&store.cluster_key())); - } - - let row2 = DataRow::from_component_batches( - RowId::new(), - TimePoint::from_iter([(timeline_frame, 42)]), - "entity_b".into(), - [&[MyColor::from(0xAABBCCDD)] as _], - )?; - - // Autogenerated, should _not_ fire. - { - let event = store.insert_row(&row2)?; - assert!(!event.cells.contains_key(&store.cluster_key())); - - let (events, _) = store.gc(&GarbageCollectionOptions::gc_everything()); - assert_eq!(1, events.len()); - assert!(!events[0].cells.contains_key(&store.cluster_key())); - } - - Ok(()) - } } diff --git a/crates/re_data_store/src/store_format.rs b/crates/re_data_store/src/store_format.rs index b3eb183ceea5d..38a27854bd439 100644 --- a/crates/re_data_store/src/store_format.rs +++ b/crates/re_data_store/src/store_format.rs @@ -11,9 +11,7 @@ impl std::fmt::Display for DataStore { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let Self { id, - cluster_key, config, - cluster_cell_cache: _, type_registry: _, metadata_registry: _, tables, @@ -27,10 +25,6 @@ impl std::fmt::Display for DataStore { f.write_str("DataStore {\n")?; f.write_str(&indent::indent_all_by(4, format!("id: {id}\n")))?; - f.write_str(&indent::indent_all_by( - 4, - format!("cluster_key: {cluster_key:?}\n"), - ))?; f.write_str(&indent::indent_all_by(4, format!("config: {config:?}\n")))?; { @@ -85,7 +79,6 @@ impl std::fmt::Display for IndexedTable { timeline, entity_path, buckets, - cluster_key: _, all_components: _, buckets_num_rows: _, buckets_size_bytes: _, diff --git a/crates/re_data_store/src/store_gc.rs b/crates/re_data_store/src/store_gc.rs index b199b44e99916..a7331bb12fff4 100644 --- a/crates/re_data_store/src/store_gc.rs +++ b/crates/re_data_store/src/store_gc.rs @@ -10,7 +10,7 @@ use re_log_types::{ use re_types_core::{ComponentName, SizeBytes as _}; use crate::{ - store::{ClusterCellCache, IndexedBucketInner, IndexedTable}, + store::{IndexedBucketInner, IndexedTable}, DataStore, DataStoreStats, StoreDiff, StoreDiffKind, StoreEvent, }; @@ -232,9 +232,7 @@ impl DataStore { let mut batch_is_protected = false; let Self { - cluster_key, metadata_registry, - cluster_cell_cache, tables, .. } = self; @@ -254,8 +252,6 @@ impl DataStore { let dropped = Self::drop_batch( options, tables, - cluster_cell_cache, - *cluster_key, &mut num_bytes_to_drop, &batch, batch_is_protected, @@ -297,8 +293,6 @@ impl DataStore { let dropped = Self::drop_batch( options, tables, - cluster_cell_cache, - *cluster_key, &mut num_bytes_to_drop, &batch, batch_is_protected, @@ -341,8 +335,6 @@ impl DataStore { fn drop_batch( options: &GarbageCollectionOptions, tables: &mut BTreeMap<(EntityPathHash, Timeline), IndexedTable>, - cluster_cell_cache: &ClusterCellCache, - cluster_key: ComponentName, num_bytes_to_drop: &mut f64, batch: &[(TimePoint, (EntityPathHash, RowId))], batch_is_protected: bool, @@ -373,8 +365,7 @@ impl DataStore { // NOTE: We _must_ go through all tables no matter what, since the batch might contain // any number of distinct entities. for ((entity_path_hash, _), table) in &mut *tables { - let (removed, num_bytes_removed) = - table.try_drop_bucket(cluster_cell_cache, cluster_key, max_row_id); + let (removed, num_bytes_removed) = table.try_drop_bucket(max_row_id); *num_bytes_to_drop -= num_bytes_removed as f64; @@ -407,8 +398,7 @@ impl DataStore { // find all tables that could possibly contain this `RowId` for (&timeline, &time) in timepoint { if let Some(table) = tables.get_mut(&(*entity_path_hash, timeline)) { - let (removed, num_bytes_removed) = - table.try_drop_row(cluster_cell_cache, *row_id, time); + let (removed, num_bytes_removed) = table.try_drop_row(*row_id, time); if let Some(inner) = diff.as_mut() { if let Some(removed) = removed { inner.times.extend(removed.times); @@ -460,7 +450,6 @@ impl DataStore { let mut components_to_find: HashMap = table .all_components .iter() - .filter(|c| **c != table.cluster_key) .filter(|c| !dont_protect.contains(*c)) .map(|c| (*c, target_count)) .collect(); @@ -506,8 +495,11 @@ impl DataStore { // If any bucket has a non-empty component in any column, we keep it… for bucket in table.buckets.values() { let inner = bucket.inner.read(); - for num in &inner.col_num_instances { - if num.get() != 0 { + for column in inner.columns.values() { + if column + .iter() + .any(|cell| cell.as_ref().map_or(false, |cell| cell.num_instances() > 0)) + { return true; } } @@ -534,7 +526,7 @@ impl DataStore { for column in &mut inner.columns.values_mut() { let cell = column[i].take(); if let Some(cell) = cell { - diff.insert(self.cluster_key, &self.cluster_cell_cache, cell); + diff.insert(cell); } } } @@ -549,12 +541,7 @@ impl DataStore { impl IndexedTable { /// Try to drop an entire bucket at once if it doesn't contain any `RowId` greater than `max_row_id`. - fn try_drop_bucket( - &mut self, - cluster_cache: &ClusterCellCache, - cluster_key: ComponentName, - max_row_id: RowId, - ) -> (Vec, u64) { + fn try_drop_bucket(&mut self, max_row_id: RowId) -> (Vec, u64) { re_tracing::profile_function!(); let entity_path = self.entity_path.clone(); @@ -594,16 +581,6 @@ impl IndexedTable { for (component_name, column) in &mut columns { if let Some(cell) = column.pop_front().flatten() { - if cell.component_name() == cluster_key { - if let Some(cached_cell) = cluster_cache.get(&cell.num_instances()) { - if std::ptr::eq(cell.as_ptr(), cached_cell.as_ptr()) { - // We don't fire events when inserting autogenerated cluster cells, and - // therefore must not fire when removing them either. - continue; - } - } - } - diff.cells.insert(*component_name, cell); } } @@ -630,17 +607,11 @@ impl IndexedTable { /// specified `time`. /// /// Returns how many bytes were actually dropped, or zero if the row wasn't found. - fn try_drop_row( - &mut self, - cluster_cache: &ClusterCellCache, - row_id: RowId, - time: TimeInt, - ) -> (Option, u64) { + fn try_drop_row(&mut self, row_id: RowId, time: TimeInt) -> (Option, u64) { re_tracing::profile_function!(); let entity_path = self.entity_path.clone(); let timeline = self.timeline; - let cluster_key = self.cluster_key; let table_has_more_than_one_bucket = self.buckets.len() > 1; @@ -649,14 +620,7 @@ impl IndexedTable { let (diff, mut dropped_num_bytes) = { let inner = &mut *bucket.inner.write(); - inner.try_drop_row( - cluster_cache, - cluster_key, - row_id, - timeline, - &entity_path, - time, - ) + inner.try_drop_row(row_id, timeline, &entity_path, time) }; // NOTE: We always need to keep at least one bucket alive, otherwise we have @@ -688,8 +652,6 @@ impl IndexedBucketInner { /// Returns how many bytes were actually dropped, or zero if the row wasn't found. fn try_drop_row( &mut self, - cluster_cache: &ClusterCellCache, - cluster_key: ComponentName, row_id: RowId, timeline: Timeline, entity_path: &EntityPath, @@ -704,7 +666,6 @@ impl IndexedBucketInner { col_insert_id, col_row_id, max_row_id, - col_num_instances, columns, size_bytes, } = self; @@ -756,11 +717,6 @@ impl IndexedBucketInner { } } - // col_num_instances - if let Some(num_instances) = col_num_instances.swap_remove(row_index) { - dropped_num_bytes += num_instances.total_size_bytes(); - } - // each data column for column in columns.values_mut() { let cell = column.0.swap_remove(row_index).flatten(); @@ -771,11 +727,10 @@ impl IndexedBucketInner { if let Some(cell) = cell { if let Some(inner) = diff.as_mut() { - inner.insert(cluster_key, cluster_cache, cell); + inner.insert(cell); } else { let mut d = StoreDiff::deletion(removed_row_id, entity_path.clone()); d.at_timestamp(timeline, time); - d.insert(cluster_key, cluster_cache, cell); diff = Some(d); } } @@ -801,22 +756,7 @@ impl IndexedBucketInner { // --- impl StoreDiff { - fn insert( - &mut self, - cluster_key: ComponentName, - cluster_cache: &ClusterCellCache, - cell: DataCell, - ) { - if cell.component_name() == cluster_key { - if let Some(cached_cell) = cluster_cache.get(&cell.num_instances()) { - if std::ptr::eq(cell.as_ptr(), cached_cell.as_ptr()) { - // We don't fire events when inserting of autogenerated cluster cells, and - // therefore must not fire when removing them either. - return; - } - } - } - + fn insert(&mut self, cell: DataCell) { self.cells.insert(cell.component_name(), cell); } } diff --git a/crates/re_data_store/src/store_helpers.rs b/crates/re_data_store/src/store_helpers.rs index d8f07b842cd1e..f98b32c09a559 100644 --- a/crates/re_data_store/src/store_helpers.rs +++ b/crates/re_data_store/src/store_helpers.rs @@ -24,7 +24,6 @@ impl DataStore { RowId::new(), entity_path.clone(), timepoint.clone(), - 1, [component], ) { Ok(row) => row, @@ -64,7 +63,6 @@ impl DataStore { RowId::new(), entity_path.clone(), timepoint.clone(), - cell.num_instances(), cell, ) { Ok(row) => row, diff --git a/crates/re_data_store/src/store_read.rs b/crates/re_data_store/src/store_read.rs index ad83948579e43..392363fe85af0 100644 --- a/crates/re_data_store/src/store_read.rs +++ b/crates/re_data_store/src/store_read.rs @@ -215,10 +215,6 @@ impl DataStore { self.query_id.fetch_add(1, Ordering::Relaxed); let entity_path_hash = entity_path.hash(); - let cluster_comp_pos = component_names - .iter() - .find_position(|component_name| **component_name == self.cluster_key) - .map(|(pos, _)| pos); let Some(primary_comp_pos) = component_names .iter() .find_position(|component_name| **component_name == primary) @@ -233,12 +229,9 @@ impl DataStore { // querying for their temporal data. let mut component_names_opt = [(); N].map(|_| None); for (i, component_name) in component_names.iter().copied().enumerate() { - // TODO(#5303): We let the cluster key slip through for backwards compatibility with - // the legacy instance-key model. This will go away next. - let has_static_data = component_name != self.cluster_key - && static_table.map_or(false, |static_table| { - static_table.cells.contains_key(&component_name) - }); + let has_static_data = static_table.map_or(false, |static_table| { + static_table.cells.contains_key(&component_name) + }); component_names_opt[i] = (!has_static_data).then_some(component_name); } @@ -262,12 +255,6 @@ impl DataStore { if *component_name == primary { data_time = TimeInt::STATIC; max_row_id = RowId::max(max_row_id, static_cell.row_id); - - // TODO(#5303): We let the cluster key slip through for backwards compatibility with - // the legacy instance-key model. This will go away next. - if let Some(cluster_comp_pos) = cluster_comp_pos { - results[cluster_comp_pos] = Some(static_cell.cluster_key.clone()); - } } } } @@ -306,10 +293,6 @@ impl DataStore { self.query_id.fetch_add(1, Ordering::Relaxed); let entity_path_hash = entity_path.hash(); - let cluster_comp_pos = component_names - .iter() - .find_position(|component_name| **component_name == self.cluster_key) - .map(|(pos, _)| pos); let static_table = self.static_tables.get(&entity_path_hash); @@ -317,12 +300,9 @@ impl DataStore { // querying for their temporal data. let mut component_names_opt = [(); N].map(|_| None); for (i, component_name) in component_names.iter().copied().enumerate() { - // TODO(#5303): We let the cluster key slip through for backwards compatibility with - // the legacy instance-key model. This will go away next. - let has_static_data = component_name != self.cluster_key - && static_table.map_or(false, |static_table| { - static_table.cells.contains_key(&component_name) - }); + let has_static_data = static_table.map_or(false, |static_table| { + static_table.cells.contains_key(&component_name) + }); component_names_opt[i] = (!has_static_data).then_some(component_name); } @@ -340,12 +320,6 @@ impl DataStore { // the returned index. if static_cell.row_id > max_row_id { max_row_id = RowId::max(max_row_id, static_cell.row_id); - - // TODO(#5303): We let the cluster key slip through for backwards compatibility with - // the legacy instance-key model. This will go away next. - if let Some(cluster_comp_pos) = cluster_comp_pos { - results[cluster_comp_pos] = Some(static_cell.cluster_key.clone()); - } } } } @@ -602,7 +576,6 @@ impl IndexedBucket { col_insert_id: _, col_row_id, max_row_id: _, - col_num_instances: _, columns, size_bytes: _, } = &*self.inner.read(); @@ -726,7 +699,6 @@ impl IndexedBucket { col_insert_id: _, col_row_id, max_row_id: _, - col_num_instances: _, columns, size_bytes: _, } = &*self.inner.read(); @@ -845,7 +817,6 @@ impl IndexedBucketInner { col_insert_id, col_row_id, max_row_id: _, - col_num_instances, columns, size_bytes: _, } = self; @@ -898,7 +869,6 @@ impl IndexedBucketInner { reshuffle_control_column(col_insert_id, &swaps); } reshuffle_control_column(col_row_id, &swaps); - reshuffle_control_column(col_num_instances, &swaps); } { diff --git a/crates/re_data_store/src/store_sanity.rs b/crates/re_data_store/src/store_sanity.rs index ab11a3bf14a7a..19eab8653ebde 100644 --- a/crates/re_data_store/src/store_sanity.rs +++ b/crates/re_data_store/src/store_sanity.rs @@ -1,6 +1,4 @@ -use re_log_types::{ - DataCellColumn, NumInstances, RowId, TimeInt, TimeRange, VecDequeSortingExt as _, -}; +use re_log_types::{RowId, TimeInt, TimeRange, VecDequeSortingExt as _}; use re_types_core::{ComponentName, Loggable, SizeBytes as _}; use crate::{DataStore, IndexedBucket, IndexedBucketInner, IndexedTable}; @@ -43,12 +41,6 @@ pub enum SanityError { got: u64, }, - #[error("Couldn't find any column for the configured cluster key ('{cluster_key}')")] - ClusterColumnMissing { cluster_key: ComponentName }, - - #[error("The cluster column must be dense, found holes: {cluster_column:?}")] - ClusterColumnSparse { cluster_column: Box }, - #[error("Found overlapping indexed buckets: {t1_max_formatted} ({t1_max}) <-> {t2_max_formatted} ({t2_max})")] OverlappingBuckets { t1_max: i64, @@ -150,11 +142,7 @@ impl IndexedBucket { pub fn sanity_check(&self) -> SanityResult<()> { re_tracing::profile_function!(); - let Self { - timeline: _, - cluster_key, - inner, - } = self; + let Self { timeline: _, inner } = self; { let IndexedBucketInner { @@ -164,7 +152,6 @@ impl IndexedBucket { col_insert_id, col_row_id, max_row_id, - col_num_instances, columns, size_bytes: _, } = &*inner.read(); @@ -216,7 +203,6 @@ impl IndexedBucket { .then(|| (DataStore::insert_id_component_name(), col_insert_id.len())), // Some((COLUMN_TIMEPOINT.into(), col_time.len())), Some((RowId::name(), col_row_id.len())), - Some((NumInstances::name(), col_num_instances.len())), ] .into_iter() .flatten() @@ -237,21 +223,6 @@ impl IndexedBucket { } } } - - // The cluster column must be fully dense. - if self.num_rows() > 0 { - let cluster_column = - columns - .get(cluster_key) - .ok_or(SanityError::ClusterColumnMissing { - cluster_key: *cluster_key, - })?; - if !cluster_column.iter().all(|cell| cell.is_some()) { - return Err(SanityError::ClusterColumnSparse { - cluster_column: cluster_column.clone().into(), - }); - } - } } // Make sure size values aren't out of sync diff --git a/crates/re_data_store/src/store_stats.rs b/crates/re_data_store/src/store_stats.rs index 64dde26eac727..f960a4174d11c 100644 --- a/crates/re_data_store/src/store_stats.rs +++ b/crates/re_data_store/src/store_stats.rs @@ -1,10 +1,7 @@ use re_log_types::{EntityPathHash, TimePoint, TimeRange}; use re_types_core::SizeBytes; -use crate::{ - store::IndexedBucketInner, ClusterCellCache, DataStore, IndexedBucket, IndexedTable, - MetadataRegistry, -}; +use crate::{store::IndexedBucketInner, DataStore, IndexedBucket, IndexedTable, MetadataRegistry}; // --- @@ -44,8 +41,6 @@ pub struct DataStoreStats { pub type_registry: DataStoreRowStats, pub metadata_registry: DataStoreRowStats, - pub autogenerated: DataStoreRowStats, - /// `num_rows` is really `num_cells` in this case. pub static_tables: DataStoreRowStats, @@ -62,7 +57,6 @@ impl std::ops::Sub for DataStoreStats { Self { type_registry: self.type_registry - rhs.type_registry, metadata_registry: self.metadata_registry - rhs.metadata_registry, - autogenerated: self.autogenerated - rhs.autogenerated, static_tables: self.static_tables - rhs.static_tables, temporal: self.temporal - rhs.temporal, temporal_buckets: self.temporal_buckets - rhs.temporal_buckets, @@ -78,7 +72,6 @@ impl std::ops::Add for DataStoreStats { Self { type_registry: self.type_registry + rhs.type_registry, metadata_registry: self.metadata_registry + rhs.metadata_registry, - autogenerated: self.autogenerated + rhs.autogenerated, static_tables: self.static_tables + rhs.static_tables, temporal: self.temporal + rhs.temporal, temporal_buckets: self.temporal_buckets + rhs.temporal_buckets, @@ -107,14 +100,6 @@ impl DataStoreStats { } }; - let autogenerated = { - re_tracing::profile_scope!("autogenerated"); - DataStoreRowStats { - num_rows: store.cluster_cell_cache.len() as _, - num_bytes: store.cluster_cell_cache.total_size_bytes(), - } - }; - let static_tables = { re_tracing::profile_scope!("static data"); DataStoreRowStats { @@ -138,7 +123,6 @@ impl DataStoreStats { num_rows: static_tables.num_rows + temporal.num_rows, num_bytes: type_registry.num_bytes + metadata_registry.num_bytes - + autogenerated.num_bytes + static_tables.num_bytes + temporal.num_bytes, }; @@ -146,7 +130,6 @@ impl DataStoreStats { Self { type_registry, metadata_registry, - autogenerated, static_tables, temporal, temporal_buckets, @@ -175,13 +158,6 @@ impl SizeBytes for MetadataRegistry<(TimePoint, EntityPathHash)> { } } -impl SizeBytes for ClusterCellCache { - #[inline] - fn heap_size_bytes(&self) -> u64 { - self.0.heap_size_bytes() - } -} - impl SizeBytes for DataStore { #[inline] fn heap_size_bytes(&self) -> u64 { @@ -392,7 +368,6 @@ impl IndexedBucketInner { col_insert_id, col_row_id, max_row_id, - col_num_instances, columns, size_bytes, } = self; @@ -403,7 +378,6 @@ impl IndexedBucketInner { + col_insert_id.total_size_bytes() + col_row_id.total_size_bytes() + max_row_id.total_size_bytes() - + col_num_instances.total_size_bytes() + columns.total_size_bytes() + size_bytes.total_size_bytes(); diff --git a/crates/re_data_store/src/store_subscriber.rs b/crates/re_data_store/src/store_subscriber.rs index 51bba35288705..f56264164bfe0 100644 --- a/crates/re_data_store/src/store_subscriber.rs +++ b/crates/re_data_store/src/store_subscriber.rs @@ -156,10 +156,9 @@ mod tests { use ahash::HashSet; use re_log_types::{ - example_components::{MyColor, MyPoint}, + example_components::{MyColor, MyIndex, MyPoint}, DataRow, RowId, StoreId, TimePoint, Timeline, }; - use re_types_core::{components::InstanceKey, Loggable as _}; use crate::{DataStore, GarbageCollectionOptions, StoreSubscriber}; @@ -209,12 +208,10 @@ mod tests { fn store_subscriber() -> anyhow::Result<()> { let mut store1 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); let mut store2 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -235,7 +232,7 @@ mod tests { (timeline_yet_another, 1), // ]), "entity_a".into(), - [&InstanceKey::from_iter(0..10) as _], + [&MyIndex::from_iter(0..10) as _], )?; expected_events.extend(store1.insert_row(&row)); @@ -267,7 +264,7 @@ mod tests { TimePoint::default(), "entity_b".into(), [ - &InstanceKey::from_iter(0..num_instances as _) as _, + &MyIndex::from_iter(0..num_instances as _) as _, &colors as _, ], )? diff --git a/crates/re_data_store/src/store_write.rs b/crates/re_data_store/src/store_write.rs index b340d2f64b296..d1b3347f77e48 100644 --- a/crates/re_data_store/src/store_write.rs +++ b/crates/re_data_store/src/store_write.rs @@ -8,9 +8,7 @@ use re_log_types::{ DataCell, DataCellColumn, DataCellError, DataRow, EntityPathHash, RowId, TimeInt, TimePoint, TimeRange, VecDequeRemovalExt as _, }; -use re_types_core::{ - components::InstanceKey, ComponentName, ComponentNameSet, Loggable, SizeBytes as _, -}; +use re_types_core::{ComponentName, ComponentNameSet, SizeBytes as _}; use crate::{ DataStore, DataStoreConfig, IndexedBucket, IndexedBucketInner, IndexedTable, MetadataRegistry, @@ -27,15 +25,6 @@ pub enum WriteError { #[error("Error with one or more the underlying data cells")] DataCell(#[from] DataCellError), - #[error("The cluster component must be dense, got {0:?}")] - SparseClusteringComponent(DataCell), - - #[error( - "The cluster component must be increasingly sorted and not contain \ - any duplicates, got {0:?}" - )] - InvalidClusteringComponent(DataCell), - #[error("The inserted data must contain at least one cell")] Empty, @@ -56,10 +45,6 @@ pub type WriteResult = ::std::result::Result; impl DataStore { /// Inserts a [`DataRow`]'s worth of components into the datastore. - /// - /// If the bundle doesn't carry a payload for the cluster key, one will be auto-generated - /// based on the length of the components in the payload, in the form of an array of - /// monotonically increasing `u64`s going from `0` to `N-1`. pub fn insert_row(&mut self, row: &DataRow) -> WriteResult { // TODO(cmc): kind & insert_id need to somehow propagate through the span system. self.insert_id += 1; @@ -72,7 +57,6 @@ impl DataStore { row_id, timepoint, entity_path, - num_instances, cells, } = row; @@ -90,12 +74,10 @@ impl DataStore { } let entity_path_hash = entity_path.hash(); - let num_instances = *num_instances; trace!( kind = "insert", id = self.insert_id, - cluster_key = %self.cluster_key, timelines = ?timepoint.iter() .map(|(timeline, time)| (timeline.name(), timeline.typ().format_utc(*time))) .collect::>(), @@ -104,46 +86,14 @@ impl DataStore { "insertion started…" ); - let cluster_cell_pos = cells - .iter() - .find_position(|cell| cell.component_name() == self.cluster_key) - .map(|(pos, _)| pos); - - let generated_cluster_cell = if let Some(cluster_cell_pos) = cluster_cell_pos { - // We found a column with a name matching the cluster key's, let's make sure it's - // valid (dense, sorted, no duplicates) and use that if so. - - let cluster_cell = &cells[cluster_cell_pos]; - - // Clustering component must be dense. - if !cluster_cell.is_dense() { - return Err(WriteError::SparseClusteringComponent(cluster_cell.clone())); - } - // Clustering component must be sorted and not contain any duplicates. - if !cluster_cell.is_sorted_and_unique()? { - return Err(WriteError::InvalidClusteringComponent(cluster_cell.clone())); - } - - None - } else { - // The caller has not specified any cluster component, and so we'll have to generate - // one… unless we've already generated one of this exact length in the past, - // in which case we can simply re-use that cell. - - let (cell, _) = self.generate_cluster_cell(num_instances.into()); - - Some(cell) - }; - let insert_id = self.config.store_insert_ids.then_some(self.insert_id); let diff = if timepoint.is_static() { let static_table = self .static_tables .entry(entity_path_hash) - .or_insert_with(|| StaticTable::new(self.cluster_key, entity_path.clone())); + .or_insert_with(|| StaticTable::new(entity_path.clone())); - let cluster_key = self.cluster_key; let cells = row .cells() .iter() @@ -156,25 +106,13 @@ impl DataStore { }) .collect_vec(); - for cell in cells - .iter() - // TODO(#5303): We let the cluster key slip through for backwards compatibility with - // the legacy instance-key model. This will go away next. - .filter(|cell| cell.component_name() != cluster_key) - { + for cell in &cells { static_table.cells.insert( cell.component_name(), StaticCell { insert_id, row_id: *row_id, - num_instances, cell: (*cell).clone(), - // TODO(#5303): We keep track of cluster keys for each static cell for backwards - // compatibility with the legacy instance-key model. This will go away next. - cluster_key: generated_cluster_cell - .as_ref() - .unwrap_or_else(|| cells[cluster_cell_pos.unwrap()]) - .clone(), }, ); } @@ -188,15 +126,9 @@ impl DataStore { let index = self .tables .entry((entity_path_hash, *timeline)) - .or_insert_with(|| IndexedTable::new(self.cluster_key, *timeline, entity_path)); - - index.insert_row( - &self.config, - insert_id, - *time, - generated_cluster_cell.clone(), /* shallow */ - row, - ); + .or_insert_with(|| IndexedTable::new(*timeline, entity_path)); + + index.insert_row(&self.config, insert_id, *time, row); } let mut diff = StoreDiff::addition(*row_id, entity_path.clone()); @@ -205,11 +137,6 @@ impl DataStore { diff }; - // TODO(#4220): should we fire for auto-generated data? - // if let Some(cell) = generated_cluster_cell { - // diff = diff.with_cells([cell]); - // } - let event = StoreEvent { store_id: self.id.clone(), store_generation: self.generation(), @@ -233,35 +160,6 @@ impl DataStore { Ok(event) } - - /// Auto-generates an appropriate cluster cell for the specified number of instances and - /// transparently handles caching. - /// - /// Returns `true` if the cell was returned from cache. - // TODO(#1777): shared slices for auto generated keys - fn generate_cluster_cell(&mut self, num_instances: u32) -> (DataCell, bool) { - re_tracing::profile_function!(); - - if let Some(cell) = self.cluster_cell_cache.get(&num_instances) { - // Cache hit! - - (cell.clone(), true) - } else { - // Cache miss! Craft a new instance keys from the ground up. - - // …so we create it manually instead. - let values = - arrow2::array::UInt64Array::from_vec((0..num_instances as u64).collect_vec()) - .boxed(); - let mut cell = DataCell::from_arrow(InstanceKey::name(), values); - cell.compute_size_bytes(); - - self.cluster_cell_cache - .insert(num_instances, cell.clone() /* shallow */); - - (cell, false) - } - } } impl MetadataRegistry<(TimePoint, EntityPathHash)> { @@ -291,7 +189,6 @@ impl IndexedTable { config: &DataStoreConfig, insert_id: Option, time: TimeInt, - generated_cluster_cell: Option, row: &DataRow, ) { re_tracing::profile_function!(); @@ -326,7 +223,7 @@ impl IndexedTable { self.buckets_size_bytes -= bucket_size_before; self.buckets.insert(min, second_half); - return self.insert_row(config, insert_id, time, generated_cluster_cell, row); + return self.insert_row(config, insert_id, time, row); } // We couldn't split the bucket, either because it's already too small, or because it @@ -383,13 +280,12 @@ impl IndexedTable { TimeInt::new_temporal(new_time_bound), IndexedBucket { timeline, - cluster_key: self.cluster_key, inner: RwLock::new(inner), }, ); self.buckets_size_bytes += inner_size_bytes; - return self.insert_row(config, insert_id, time, generated_cluster_cell, row); + return self.insert_row(config, insert_id, time, row); } } @@ -420,8 +316,7 @@ impl IndexedTable { "inserted into indexed tables" ); - let size_bytes = - bucket.insert_row(insert_id, time, generated_cluster_cell, row, &components); + let size_bytes = bucket.insert_row(insert_id, time, row, &components); self.buckets_size_bytes += size_bytes; self.buckets_num_rows += 1; @@ -436,14 +331,13 @@ impl IndexedBucket { &mut self, insert_id: Option, time: TimeInt, - generated_cluster_cell: Option, row: &DataRow, components: &ComponentNameSet, ) -> u64 { re_tracing::profile_function!(); let mut size_bytes_added = 0u64; - let num_rows = self.num_rows() as usize; + let _num_rows = self.num_rows() as usize; let mut inner = self.inner.write(); let IndexedBucketInner { @@ -453,7 +347,6 @@ impl IndexedBucket { col_insert_id, col_row_id, max_row_id, - col_num_instances, columns, size_bytes, } = &mut *inner; @@ -477,21 +370,6 @@ impl IndexedBucket { col_row_id.push_back(row.row_id()); *max_row_id = RowId::max(*max_row_id, row.row_id()); size_bytes_added += row.row_id().total_size_bytes(); - col_num_instances.push_back(row.num_instances()); - size_bytes_added += row.num_instances().total_size_bytes(); - - // insert auto-generated cluster cell if present - if let Some(cluster_cell) = generated_cluster_cell { - let component_name = cluster_cell.component_name(); - let column = columns.entry(component_name).or_insert_with(|| { - let column = DataCellColumn::empty(num_rows); - size_bytes_added += component_name.total_size_bytes(); - size_bytes_added += column.total_size_bytes(); - column - }); - size_bytes_added += cluster_cell.total_size_bytes(); - column.0.push_back(Some(cluster_cell.clone())); - } // append components to their respective columns (2-way merge) @@ -512,11 +390,6 @@ impl IndexedBucket { // // fill unimpacted columns with null values for (component_name, column) in &mut *columns { - // The cluster key always gets added one way or another, don't try to force fill it! - if *component_name == self.cluster_key { - continue; - } - if !components.contains(component_name) { let none_cell: Option = None; size_bytes_added += none_cell.total_size_bytes(); @@ -559,11 +432,7 @@ impl IndexedBucket { /// cargo test -p re_data_store -- --nocapture datastore_internal_repr /// ``` fn split(&self) -> Option<(TimeInt, Self)> { - let Self { - timeline, - cluster_key: _, - inner, - } = self; + let Self { timeline, inner } = self; let mut inner1 = inner.write(); @@ -588,7 +457,6 @@ impl IndexedBucket { col_insert_id: col_insert_id1, col_row_id: col_row_id1, max_row_id: max_row_id1, - col_num_instances: col_num_instances1, columns: columns1, size_bytes: _, // NOTE: recomputed below } = &mut *inner1; @@ -605,7 +473,7 @@ impl IndexedBucket { }; let split_idx = find_split_index(times1).expect("must be splittable at this point"); - let (time_range2, col_time2, col_insert_id2, col_row_id2, col_num_instances2) = { + let (time_range2, col_time2, col_insert_id2, col_row_id2) = { re_tracing::profile_scope!("control"); // update everything _in place_! ( @@ -613,7 +481,6 @@ impl IndexedBucket { col_time1.split_off_or_default(split_idx), col_insert_id1.split_off_or_default(split_idx), col_row_id1.split_off_or_default(split_idx), - col_num_instances1.split_off_or_default(split_idx), ) }; // NOTE: We _have_ to fullscan here: the bucket is sorted by `(Time, RowId)`, there @@ -648,7 +515,6 @@ impl IndexedBucket { col_insert_id: col_insert_id2, col_row_id: col_row_id2, max_row_id: max_row_id2, - col_num_instances: col_num_instances2, columns: columns2, size_bytes: 0, // NOTE: computed below }; @@ -657,7 +523,6 @@ impl IndexedBucket { }; let bucket2 = Self { timeline, - cluster_key: self.cluster_key, inner: RwLock::new(inner2), }; diff --git a/crates/re_data_store/src/test_util.rs b/crates/re_data_store/src/test_util.rs index 50c568d4a943e..f928c4f455a6b 100644 --- a/crates/re_data_store/src/test_util.rs +++ b/crates/re_data_store/src/test_util.rs @@ -7,32 +7,29 @@ use crate::{DataStore, DataStoreConfig, WriteError}; #[doc(hidden)] #[macro_export] macro_rules! test_row { - ($entity:ident => $n:expr; [$c0:expr $(,)*]) => {{ + ($entity:ident => [$c0:expr $(,)*]) => {{ ::re_log_types::DataRow::from_cells1_sized( ::re_log_types::RowId::new(), $entity.clone(), ::re_log_types::TimePoint::default(), - $n, $c0, ) .unwrap() }}; - ($entity:ident @ $frames:tt => $n:expr; [$c0:expr $(,)*]) => {{ + ($entity:ident @ $frames:tt => [$c0:expr $(,)*]) => {{ ::re_log_types::DataRow::from_cells1_sized( ::re_log_types::RowId::new(), $entity.clone(), $frames, - $n, $c0, ) .unwrap() }}; - ($entity:ident @ $frames:tt => $n:expr; [$c0:expr, $c1:expr $(,)*]) => {{ + ($entity:ident @ $frames:tt => [$c0:expr, $c1:expr $(,)*]) => {{ ::re_log_types::DataRow::from_cells2_sized( ::re_log_types::RowId::new(), $entity.clone(), $frames, - $n, ($c0, $c1), ) .unwrap() diff --git a/crates/re_data_store/tests/correctness.rs b/crates/re_data_store/tests/correctness.rs index 2d75cf1a69c04..89119b663c15c 100644 --- a/crates/re_data_store/tests/correctness.rs +++ b/crates/re_data_store/tests/correctness.rs @@ -8,13 +8,12 @@ use re_data_store::{ test_row, test_util::sanity_unwrap, DataStore, DataStoreConfig, DataStoreStats, GarbageCollectionOptions, LatestAtQuery, WriteError, }; -use re_log_types::example_components::MyPoint; +use re_log_types::example_components::{MyIndex, MyPoint}; use re_log_types::{ - build_frame_nr, build_log_time, DataCell, DataRow, Duration, EntityPath, RowId, Time, TimeInt, - TimePoint, TimeType, Timeline, + build_frame_nr, build_log_time, DataRow, Duration, EntityPath, RowId, Time, TimeInt, TimePoint, + TimeType, Timeline, }; -use re_types::components::InstanceKey; -use re_types::datagen::{build_some_colors, build_some_instances, build_some_positions2d}; +use re_types::datagen::{build_some_colors, build_some_positions2d}; use re_types_core::Loggable as _; // --- @@ -54,7 +53,6 @@ fn row_id_ordering_semantics() -> anyhow::Result<()> { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -89,7 +87,6 @@ fn row_id_ordering_semantics() -> anyhow::Result<()> { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -120,7 +117,6 @@ fn row_id_ordering_semantics() -> anyhow::Result<()> { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -160,7 +156,6 @@ fn row_id_ordering_semantics() -> anyhow::Result<()> { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -202,73 +197,16 @@ fn write_errors() { let entity_path = EntityPath::from("this/that"); - { - pub fn build_sparse_instances() -> DataCell { - DataCell::from_component_sparse::([Some(1), None, Some(3)]) - } - - let mut store = DataStore::new( - re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), - Default::default(), - ); - let row = test_row!(entity_path @ - [build_frame_nr(32), build_log_time(Time::now())] => 3; [ - build_sparse_instances(), build_some_positions2d(3) - ]); - assert!(matches!( - store.insert_row(&row), - Err(WriteError::SparseClusteringComponent(_)), - )); - } - - { - pub fn build_unsorted_instances() -> DataCell { - DataCell::from_component::([1, 3, 2]) - } - - pub fn build_duped_instances() -> DataCell { - DataCell::from_component::([1, 2, 2]) - } - - let mut store = DataStore::new( - re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), - Default::default(), - ); - { - let row = test_row!(entity_path @ - [build_frame_nr(32), build_log_time(Time::now())] => 3; [ - build_unsorted_instances(), build_some_positions2d(3) - ]); - assert!(matches!( - store.insert_row(&row), - Err(WriteError::InvalidClusteringComponent(_)), - )); - } - { - let row = test_row!(entity_path @ - [build_frame_nr(32), build_log_time(Time::now())] => 3; [ - build_duped_instances(), build_some_positions2d(3) - ]); - assert!(matches!( - store.insert_row(&row), - Err(WriteError::InvalidClusteringComponent(_)), - )); - } - } - { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); let mut row = test_row!(entity_path @ [ build_frame_nr(1), build_log_time(Time::now()), - ] => 1; [ build_some_positions2d(1) ]); + ] => [ build_some_positions2d(1) ]); row.row_id = re_log_types::RowId::new(); store.insert_row(&row).unwrap(); @@ -298,7 +236,6 @@ fn latest_at_emptiness_edge_cases() { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); latest_at_emptiness_edge_cases_impl(&mut store); @@ -317,7 +254,7 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { store .insert_row(&test_row!(entity_path @ [ build_log_time(now), build_frame_nr(frame40), - ] => num_instances; [build_some_instances(num_instances as _)])) + ] => [MyIndex::from_iter(0..num_instances as _)])) .unwrap(); sanity_unwrap(store); @@ -332,8 +269,8 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_frame_nr, frame39), &entity_path, - InstanceKey::name(), - &[InstanceKey::name()], + MyIndex::name(), + &[MyIndex::name()], ); assert!(cells.is_none()); } @@ -343,8 +280,8 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_log_time, now_minus_1s_nanos), &entity_path, - InstanceKey::name(), - &[InstanceKey::name()], + MyIndex::name(), + &[MyIndex::name()], ); assert!(cells.is_none()); } @@ -354,8 +291,8 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_frame_nr, frame40), &EntityPath::from("does/not/exist"), - InstanceKey::name(), - &[InstanceKey::name()], + MyIndex::name(), + &[MyIndex::name()], ); assert!(cells.is_none()); } @@ -366,7 +303,7 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_frame_nr, frame40), &entity_path, - InstanceKey::name(), + MyIndex::name(), components, ); assert!(cells.is_none()); @@ -377,7 +314,7 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_frame_nr, frame40), &entity_path, - InstanceKey::name(), + MyIndex::name(), &[], ); assert!(cells.is_none()); @@ -388,8 +325,8 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_wrong_name, frame40), &EntityPath::from("does/not/exist"), - InstanceKey::name(), - &[InstanceKey::name()], + MyIndex::name(), + &[MyIndex::name()], ); assert!(cells.is_none()); } @@ -399,8 +336,8 @@ fn latest_at_emptiness_edge_cases_impl(store: &mut DataStore) { let cells = store.latest_at( &LatestAtQuery::new(timeline_wrong_kind, frame40), &EntityPath::from("does/not/exist"), - InstanceKey::name(), - &[InstanceKey::name()], + MyIndex::name(), + &[MyIndex::name()], ); assert!(cells.is_none()); } @@ -414,7 +351,6 @@ fn gc_correct() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig::default(), ); @@ -431,7 +367,7 @@ fn gc_correct() { let num_instances = rng.gen_range(0..=1_000); let row = test_row!(entity_path @ [ build_frame_nr(frame_nr), - ] => num_instances; [ + ] => [ build_some_colors(num_instances as _), ]); store.insert_row(&row).unwrap(); @@ -481,7 +417,6 @@ fn gc_metadata_size() -> anyhow::Result<()> { for enable_batching in [false, true] { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), Default::default(), ); @@ -529,7 +464,6 @@ fn entity_min_time_correct() -> anyhow::Result<()> { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); entity_min_time_correct_impl(&mut store)?; diff --git a/crates/re_data_store/tests/data_store.rs b/crates/re_data_store/tests/data_store.rs index 2a67aeb307ed4..27f8ccd1f4920 100644 --- a/crates/re_data_store/tests/data_store.rs +++ b/crates/re_data_store/tests/data_store.rs @@ -10,15 +10,16 @@ use re_data_store::{ DataStore, DataStoreConfig, DataStoreStats, GarbageCollectionOptions, GarbageCollectionTarget, LatestAtQuery, RangeQuery, TimeInt, TimeRange, }; -use re_log_types::{build_frame_nr, DataRow, DataTable, EntityPath, TableId, TimeType, Timeline}; +use re_log_types::{ + build_frame_nr, example_components::MyIndex, DataRow, DataTable, EntityPath, TableId, TimeType, + Timeline, +}; use re_types::{ - components::{Color, InstanceKey, Position2D}, + components::{Color, Position2D}, testing::{build_some_large_structs, LargeStruct}, }; use re_types::{ - datagen::{ - build_some_colors, build_some_instances, build_some_instances_from, build_some_positions2d, - }, + datagen::{build_some_colors, build_some_positions2d}, ComponentNameSet, }; use re_types_core::{ComponentName, Loggable as _}; @@ -41,7 +42,6 @@ fn all_components() { // Stress test save-to-disk & load-from-disk let mut store2 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - store.cluster_key(), store.config().clone(), ); for table in store.to_data_tables(None) { @@ -75,39 +75,35 @@ fn all_components() { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig { indexed_bucket_num_rows: u64::MAX, ..Default::default() }, ); - let cluster_key = store.cluster_key(); let components_a = &[ Color::name(), // added by test, static LargeStruct::name(), // added by test - cluster_key, // always here ]; let components_b = &[ Color::name(), // added by test, static Position2D::name(), // added by test LargeStruct::name(), // added by test - cluster_key, // always here ]; - let row = test_row!(entity_path => 2; [build_some_colors(2)]); + let row = test_row!(entity_path => [build_some_colors(2)]); store.insert_row(&row).unwrap(); let row = - test_row!(entity_path @ [build_frame_nr(frame1)] => 2; [build_some_large_structs(2)]); + test_row!(entity_path @ [build_frame_nr(frame1)] => [build_some_large_structs(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_a)); let row = test_row!(entity_path @ [ build_frame_nr(frame2), - ] => 2; [build_some_large_structs(2), build_some_positions2d(2)]); + ] => [build_some_large_structs(2), build_some_positions2d(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_b)); @@ -119,13 +115,11 @@ fn all_components() { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig { indexed_bucket_num_rows: 0, ..Default::default() }, ); - let cluster_key = store.cluster_key(); // ┌──────────┬─────────────┬────────┬───────────┬──────────┐ // │ frame_nr ┆ LargeStruct ┆ row_id ┆ insert_id ┆ instance │ @@ -143,32 +137,29 @@ fn all_components() { let components_a = &[ Color::name(), // added by test, static LargeStruct::name(), // added by test - cluster_key, // always here ]; let components_b = &[ Color::name(), // added by test, static LargeStruct::name(), // ⚠ inherited before the buckets got split apart! Position2D::name(), // added by test - cluster_key, // always here ]; - let row = test_row!(entity_path => 2; [build_some_colors(2)]); + let row = test_row!(entity_path => [build_some_colors(2)]); store.insert_row(&row).unwrap(); let row = - test_row!(entity_path @ [build_frame_nr(frame1)] => 2; [build_some_large_structs(2)]); + test_row!(entity_path @ [build_frame_nr(frame1)] => [build_some_large_structs(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_a)); - let row = test_row!(entity_path @ [build_frame_nr(frame2)] => 2; [build_some_instances(2)]); + let row = test_row!(entity_path @ [build_frame_nr(frame2)] => [MyIndex::from_iter(0..2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_a)); - let row = - test_row!(entity_path @ [build_frame_nr(frame3)] => 2; [build_some_positions2d(2)]); + let row = test_row!(entity_path @ [build_frame_nr(frame3)] => [build_some_positions2d(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_b)); @@ -181,13 +172,11 @@ fn all_components() { { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig { indexed_bucket_num_rows: 0, ..Default::default() }, ); - let cluster_key = store.cluster_key(); // ┌──────────┬─────────────┬─────────┬────────┬───────────┬──────────┐ // │ frame_nr ┆ LargeStruct ┆ point2d ┆ row_id ┆ insert_id ┆ instance │ @@ -207,39 +196,36 @@ fn all_components() { let components_a = &[ Color::name(), // added by test, static LargeStruct::name(), // added by test - cluster_key, // always here ]; let components_b = &[ Color::name(), // added by test, static Position2D::name(), // added by test but not contained in the second bucket LargeStruct::name(), // added by test - cluster_key, // always here ]; - let row = test_row!(entity_path => 2; [build_some_colors(2)]); + let row = test_row!(entity_path => [build_some_colors(2)]); store.insert_row(&row).unwrap(); let row = - test_row!(entity_path @ [build_frame_nr(frame2)] => 2; [build_some_large_structs(2)]); + test_row!(entity_path @ [build_frame_nr(frame2)] => [build_some_large_structs(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_a)); let row = - test_row!(entity_path @ [build_frame_nr(frame3)] => 2; [build_some_large_structs(2)]); + test_row!(entity_path @ [build_frame_nr(frame3)] => [build_some_large_structs(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_a)); let row = - test_row!(entity_path @ [build_frame_nr(frame4)] => 2; [build_some_large_structs(2)]); + test_row!(entity_path @ [build_frame_nr(frame4)] => [build_some_large_structs(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_a)); - let row = - test_row!(entity_path @ [build_frame_nr(frame1)] => 2; [build_some_positions2d(2)]); + let row = test_row!(entity_path @ [build_frame_nr(frame1)] => [build_some_positions2d(2)]); store.insert_row(&row).unwrap(); assert_latest_components_at(&mut store, &entity_path, Some(components_b)); @@ -257,7 +243,6 @@ fn latest_at() { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); latest_at_impl(&mut store); @@ -275,22 +260,21 @@ fn latest_at_impl(store: &mut DataStore) { let frame3 = TimeInt::new_temporal(3); let frame4 = TimeInt::new_temporal(4); - let (instances1, colors1) = (build_some_instances(3), build_some_colors(3)); - let row1 = - test_row!(entity_path @ [build_frame_nr(frame1)] => 3; [instances1.clone(), colors1]); + let (instances1, colors1) = (MyIndex::from_iter(0..3), build_some_colors(3)); + let row1 = test_row!(entity_path @ [build_frame_nr(frame1)] => [instances1.clone(), colors1]); let positions2 = build_some_positions2d(3); - let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => 3; [instances1, positions2]); + let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => [instances1, positions2]); let points3 = build_some_positions2d(10); - let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => 10; [points3]); + let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => [points3]); let colors4 = build_some_colors(5); - let row4 = test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [colors4]); + let row4 = test_row!(entity_path @ [build_frame_nr(frame4)] => [colors4]); // injecting some static colors let colors5 = build_some_colors(3); - let row5 = test_row!(entity_path => 5; [colors5]); + let row5 = test_row!(entity_path => [colors5]); insert_table_with_retries( store, @@ -309,7 +293,6 @@ fn latest_at_impl(store: &mut DataStore) { // Stress test save-to-disk & load-from-disk let mut store2 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - store.cluster_key(), store.config().clone(), ); for table in store.to_data_tables(None) { @@ -382,7 +365,6 @@ fn range() { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); range_impl(&mut store); @@ -400,39 +382,37 @@ fn range_impl(store: &mut DataStore) { let frame4 = TimeInt::new_temporal(4); let frame5 = TimeInt::new_temporal(5); - let insts1 = build_some_instances(3); + let insts1 = MyIndex::from_iter(0..3); let colors1 = build_some_colors(3); - let row1 = test_row!(entity_path @ [build_frame_nr(frame1)] => 3; [insts1.clone(), colors1]); + let row1 = test_row!(entity_path @ [build_frame_nr(frame1)] => [insts1.clone(), colors1]); let positions2 = build_some_positions2d(3); - let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => 3; [insts1, positions2]); + let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => [insts1, positions2]); let points3 = build_some_positions2d(10); - let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => 10; [points3]); + let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => [points3]); - let insts4_1 = build_some_instances_from(20..25); + let insts4_1 = MyIndex::from_iter(20..25); let colors4_1 = build_some_colors(5); - let row4_1 = test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [insts4_1, colors4_1]); + let row4_1 = test_row!(entity_path @ [build_frame_nr(frame4)] => [insts4_1, colors4_1]); - let insts4_2 = build_some_instances_from(25..30); + let insts4_2 = MyIndex::from_iter(25..30); let colors4_2 = build_some_colors(5); - let row4_2 = - test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [insts4_2.clone(), colors4_2]); + let row4_2 = test_row!(entity_path @ [build_frame_nr(frame4)] => [insts4_2.clone(), colors4_2]); let points4_25 = build_some_positions2d(5); - let row4_25 = test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [insts4_2, points4_25]); + let row4_25 = test_row!(entity_path @ [build_frame_nr(frame4)] => [insts4_2, points4_25]); - let insts4_3 = build_some_instances_from(30..35); + let insts4_3 = MyIndex::from_iter(30..35); let colors4_3 = build_some_colors(5); - let row4_3 = - test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [insts4_3.clone(), colors4_3]); + let row4_3 = test_row!(entity_path @ [build_frame_nr(frame4)] => [insts4_3.clone(), colors4_3]); let points4_4 = build_some_positions2d(5); - let row4_4 = test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [insts4_3, points4_4]); + let row4_4 = test_row!(entity_path @ [build_frame_nr(frame4)] => [insts4_3, points4_4]); // injecting some static colors let colors5 = build_some_colors(8); - let row5 = test_row!(entity_path => 8; [colors5]); + let row5 = test_row!(entity_path => [colors5]); insert_table_with_retries( store, @@ -466,7 +446,6 @@ fn range_impl(store: &mut DataStore) { // Stress test save-to-disk & load-from-disk let mut store2 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - store.cluster_key(), store.config().clone(), ); for table in store.to_data_tables(None) { @@ -590,7 +569,6 @@ fn gc() { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); gc_impl(&mut store); @@ -611,7 +589,7 @@ fn gc_impl(store: &mut DataStore) { let num_instances = rng.gen_range(0..=1_000); let row = test_row!(entity_path @ [ build_frame_nr(frame_nr) - ] => num_instances; [ + ] => [ build_some_large_structs(num_instances as _), ]); store.insert_row(&row).unwrap(); @@ -660,7 +638,6 @@ fn protected_gc() { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); protected_gc_impl(&mut store); @@ -677,18 +654,17 @@ fn protected_gc_impl(store: &mut DataStore) { let frame3 = TimeInt::new_temporal(3); let frame4 = TimeInt::new_temporal(4); - let (instances1, colors1) = (build_some_instances(3), build_some_colors(3)); - let row1 = - test_row!(entity_path @ [build_frame_nr(frame1)] => 3; [instances1.clone(), colors1]); + let (instances1, colors1) = (MyIndex::from_iter(0..3), build_some_colors(3)); + let row1 = test_row!(entity_path @ [build_frame_nr(frame1)] => [instances1.clone(), colors1]); let positions2 = build_some_positions2d(3); - let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => 3; [instances1, positions2]); + let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => [instances1, positions2]); let points3 = build_some_positions2d(10); - let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => 10; [points3]); + let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => [points3]); let colors4 = build_some_colors(5); - let row4 = test_row!(entity_path @ [build_frame_nr(frame4)] => 5; [colors4]); + let row4 = test_row!(entity_path @ [build_frame_nr(frame4)] => [colors4]); store.insert_row(&row1).unwrap(); store.insert_row(&row2).unwrap(); @@ -763,7 +739,6 @@ fn protected_gc_clear() { for config in re_data_store::test_util::all_configs() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); protected_gc_clear_impl(&mut store); @@ -781,18 +756,17 @@ fn protected_gc_clear_impl(store: &mut DataStore) { let frame3 = TimeInt::new_temporal(3); let frame4 = TimeInt::new_temporal(4); - let (instances1, colors1) = (build_some_instances(3), build_some_colors(3)); - let row1 = - test_row!(entity_path @ [build_frame_nr(frame1)] => 3; [instances1.clone(), colors1]); + let (instances1, colors1) = (MyIndex::from_iter(0..3), build_some_colors(3)); + let row1 = test_row!(entity_path @ [build_frame_nr(frame1)] => [instances1.clone(), colors1]); let positions2 = build_some_positions2d(3); - let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => 3; [instances1, positions2]); + let row2 = test_row!(entity_path @ [build_frame_nr(frame2)] => [instances1, positions2]); let colors2 = build_some_colors(0); - let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => 0; [colors2]); + let row3 = test_row!(entity_path @ [build_frame_nr(frame3)] => [colors2]); let points4 = build_some_positions2d(0); - let row4 = test_row!(entity_path @ [build_frame_nr(frame4)] => 0; [points4]); + let row4 = test_row!(entity_path @ [build_frame_nr(frame4)] => [points4]); // Insert the 3 rows as static let mut static_table = diff --git a/crates/re_data_store/tests/dump.rs b/crates/re_data_store/tests/dump.rs index e3e0b6c5063f9..00f1824f4f468 100644 --- a/crates/re_data_store/tests/dump.rs +++ b/crates/re_data_store/tests/dump.rs @@ -7,11 +7,10 @@ use re_data_store::{ DataStore, DataStoreStats, GarbageCollectionOptions, TimeInt, TimeRange, Timeline, }; use re_log_types::{ - build_frame_nr, build_log_time, DataRow, DataTable, EntityPath, RowId, TableId, + build_frame_nr, build_log_time, example_components::MyIndex, DataRow, DataTable, EntityPath, + RowId, TableId, }; -use re_types::components::InstanceKey; -use re_types::datagen::{build_some_colors, build_some_instances, build_some_positions2d}; -use re_types_core::Loggable as _; +use re_types::datagen::{build_some_colors, build_some_positions2d}; // --- @@ -47,7 +46,6 @@ impl RowSet { std::collections::hash_map::Entry::Occupied(mut entry) => { assert_eq!(entry.get().entity_path(), row.entity_path()); assert_eq!(entry.get().cells(), row.cells()); - assert_eq!(entry.get().num_instances(), row.num_instances()); for (timeline, time) in row.timepoint() { entry.get_mut().timepoint.insert(*timeline, *time); } @@ -79,17 +77,14 @@ fn data_store_dump() { let mut store1 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); let mut store2 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); let mut store3 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); @@ -184,12 +179,10 @@ fn data_store_dump_filtered() { let mut store1 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); let mut store2 = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config.clone(), ); @@ -281,25 +274,25 @@ fn create_insert_table(entity_path: impl Into) -> DataTable { let frame3 = TimeInt::new_temporal(3); let frame4 = TimeInt::new_temporal(4); - let (instances1, colors1) = (build_some_instances(3), build_some_colors(3)); + let (instances1, colors1) = (MyIndex::from_iter(0..3), build_some_colors(3)); let row1 = test_row!(entity_path @ [ build_frame_nr(frame1), - ] => 3; [instances1.clone(), colors1]); + ] => [instances1.clone(), colors1]); let positions2 = build_some_positions2d(3); let row2 = test_row!(entity_path @ [ build_frame_nr(frame2), - ] => 3; [instances1, positions2]); + ] => [instances1, positions2]); let positions3 = build_some_positions2d(10); let row3 = test_row!(entity_path @ [ build_log_time(frame3.into()) /* ! */, build_frame_nr(frame3), - ] => 10; [positions3]); + ] => [positions3]); let colors4 = build_some_colors(5); let row4 = test_row!(entity_path @ [ build_frame_nr(frame4), - ] => 5; [colors4]); + ] => [colors4]); let mut table = DataTable::from_rows(TableId::new(), [row1, row2, row3, row4]); table.compute_all_size_bytes(); @@ -321,7 +314,6 @@ fn data_store_dump_empty_column() { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), config, ); @@ -336,15 +328,15 @@ fn data_store_dump_empty_column_impl(store: &mut DataStore) { // Start by inserting a table with 2 rows, one with colors, and one with points. { - let (instances1, colors1) = (build_some_instances(3), build_some_colors(3)); + let (instances1, colors1) = (MyIndex::from_iter(0..3), build_some_colors(3)); let row1 = test_row!(entity_path @ [ build_frame_nr(frame1), - ] => 3; [instances1, colors1]); + ] => [instances1, colors1]); - let (instances2, positions2) = (build_some_instances(3), build_some_positions2d(3)); + let (instances2, positions2) = (MyIndex::from_iter(0..3), build_some_positions2d(3)); let row2 = test_row!(entity_path @ [ build_frame_nr(frame2), - ] => 3; [instances2, positions2]); + ] => [instances2, positions2]); let mut table = DataTable::from_rows(TableId::new(), [row1, row2]); table.compute_all_size_bytes(); insert_table_with_retries(store, &table); @@ -352,10 +344,10 @@ fn data_store_dump_empty_column_impl(store: &mut DataStore) { // Now insert another table with points only. { - let (instances3, positions3) = (build_some_instances(3), build_some_colors(3)); + let (instances3, positions3) = (MyIndex::from_iter(0..3), build_some_colors(3)); let row3 = test_row!(entity_path @ [ build_frame_nr(frame3), - ] => 3; [instances3, positions3]); + ] => [instances3, positions3]); let mut table = DataTable::from_rows(TableId::new(), [row3]); table.compute_all_size_bytes(); insert_table_with_retries(store, &table); diff --git a/crates/re_data_store/tests/internals.rs b/crates/re_data_store/tests/internals.rs index 5449c284626fc..b0cb6b5e351c5 100644 --- a/crates/re_data_store/tests/internals.rs +++ b/crates/re_data_store/tests/internals.rs @@ -3,9 +3,9 @@ //! They're awful, but sometimes you just have to… use re_data_store::{DataStore, DataStoreConfig}; -use re_log_types::{build_frame_nr, DataRow, EntityPath, RowId, TimePoint}; -use re_types::{components::InstanceKey, datagen::build_some_instances}; -use re_types_core::Loggable as _; +use re_log_types::{ + build_frame_nr, example_components::MyIndex, DataRow, EntityPath, RowId, TimePoint, +}; // --- Internals --- @@ -24,7 +24,6 @@ fn pathological_bucket_topology() { let mut store_forward = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig { indexed_bucket_num_rows: 10, ..Default::default() @@ -32,7 +31,6 @@ fn pathological_bucket_topology() { ); let mut store_backward = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig { indexed_bucket_num_rows: 10, ..Default::default() @@ -54,8 +52,7 @@ fn pathological_bucket_topology() { RowId::new(), entity_path.clone(), timepoint.clone(), - num_instances, - build_some_instances(num_instances as _), + MyIndex::from_iter(0..num_instances), ) .unwrap(); store_forward.insert_row(&row).unwrap(); @@ -64,8 +61,7 @@ fn pathological_bucket_topology() { RowId::new(), entity_path.clone(), timepoint.clone(), - num_instances, - build_some_instances(num_instances as _), + MyIndex::from_iter(0..num_instances), ) .unwrap(); store_backward.insert_row(&row).unwrap(); @@ -87,8 +83,7 @@ fn pathological_bucket_topology() { RowId::new(), entity_path.clone(), timepoint, - num_instances, - build_some_instances(num_instances as _), + MyIndex::from_iter(0..num_instances), ) .unwrap() }) diff --git a/crates/re_data_store/tests/memory_test.rs b/crates/re_data_store/tests/memory_test.rs index 2b9b1f5c409f4..42c34c088a17f 100644 --- a/crates/re_data_store/tests/memory_test.rs +++ b/crates/re_data_store/tests/memory_test.rs @@ -57,8 +57,7 @@ fn memory_use(run: impl Fn() -> R) -> usize { use re_data_store::{DataStore, DataStoreConfig}; use re_log_types::{DataRow, RowId, TimePoint, TimeType, Timeline}; -use re_types::components::{InstanceKey, Scalar}; -use re_types_core::Loggable as _; +use re_types::components::Scalar; /// The memory overhead of storing many scalars in the store. #[test] @@ -70,7 +69,6 @@ fn scalar_memory_overhead() { let total_mem_use = memory_use(|| { let mut store = DataStore::new( re_log_types::StoreId::random(re_log_types::StoreKind::Recording), - InstanceKey::name(), DataStoreConfig::default(), ); @@ -78,12 +76,10 @@ fn scalar_memory_overhead() { let entity_path = re_log_types::entity_path!("scalar"); let timepoint = TimePoint::default().with(Timeline::new("log_time", TimeType::Time), i as i64); - let num_instances = 1; let row = DataRow::from_cells1_sized( RowId::new(), entity_path, timepoint, - num_instances, vec![Scalar(i as f64)], ) .unwrap();