refactor: Rename Sequencer (and its entourage) to Shard

pull/24376/head
Jake Goulding 2022-08-19 14:06:47 -04:00 committed by Carol (Nichols || Goulding)
parent 05657ea068
commit 4abf21c724
No known key found for this signature in database
GPG Key ID: E907EE5A736F87D4
44 changed files with 1140 additions and 1139 deletions

View File

@ -3,7 +3,7 @@
use crate::handler::CompactorConfig;
use backoff::BackoffConfig;
use data_types::{
Namespace, NamespaceId, PartitionId, PartitionKey, PartitionParam, SequencerId, Table, TableId,
Namespace, NamespaceId, PartitionId, PartitionKey, PartitionParam, ShardId, Table, TableId,
TableSchema,
};
use iox_catalog::interface::{get_schema_by_id, Catalog};
@ -51,23 +51,23 @@ pub enum Error {
NamespaceNotFound { namespace_id: NamespaceId },
#[snafu(display(
"Error getting the most recent highest ingested throughput partitions for sequencer {}. {}",
sequencer_id,
"Error getting the most recent highest ingested throughput partitions for shard {}. {}",
shard_id,
source
))]
HighestThroughputPartitions {
source: iox_catalog::interface::Error,
sequencer_id: SequencerId,
shard_id: ShardId,
},
#[snafu(display(
"Error getting the most level 0 file partitions for sequencer {}. {}",
sequencer_id,
"Error getting the most level 0 file partitions for shard {}. {}",
shard_id,
source
))]
MostL0Partitions {
source: iox_catalog::interface::Error,
sequencer_id: SequencerId,
shard_id: ShardId,
},
}
@ -77,8 +77,8 @@ pub type Result<T, E = Error> = std::result::Result<T, E>;
/// Data points needed to run a compactor
#[derive(Debug)]
pub struct Compactor {
/// Sequencers assigned to this compactor
sequencers: Vec<SequencerId>,
/// Shards assigned to this compactor
shards: Vec<ShardId>,
/// Object store for reading and persistence of parquet files
pub(crate) store: ParquetStorage,
@ -120,8 +120,8 @@ pub struct Compactor {
/// Histogram for tracking time to select partition candidates to compact.
/// Even though we choose partitions to compact, we have to read parquet_file catalog
/// table to see which partitions have the most recent L0 files. This time is for tracking
/// reading that. This includes time to get candidates for all sequencers
/// this compactor manages and for each sequencer the process invokes
/// reading that. This includes time to get candidates for all shard
/// this compactor manages and for each shard the process invokes
/// at most 3 three different SQLs and at least one.
/// The expectation is small (a second or less) otherwise we have to improve it
pub(crate) candidate_selection_duration: Metric<DurationHistogram>,
@ -144,7 +144,7 @@ impl Compactor {
/// Initialize the Compactor Data
#[allow(clippy::too_many_arguments)]
pub fn new(
sequencers: Vec<SequencerId>,
shards: Vec<ShardId>,
catalog: Arc<dyn Catalog>,
store: ParquetStorage,
exec: Arc<Executor>,
@ -220,7 +220,7 @@ impl Compactor {
);
Self {
sequencers,
shards,
catalog,
store,
exec,
@ -247,25 +247,25 @@ impl Compactor {
/// with any new ingested files in the past.
///
/// * New ingested files means non-deleted L0 files
/// * In all cases above, for each sequencer, N partitions with the most new ingested files
/// * In all cases above, for each shard, N partitions with the most new ingested files
/// will be selected and the return list will include at most, P = N * S, partitions where S
/// is the number of sequencers this compactor handles.
/// is the number of shards this compactor handles.
pub async fn hot_partitions_to_compact(
&self,
// Max number of the most recent highest ingested throughput partitions
// per sequencer we want to read
max_num_partitions_per_sequencer: usize,
// per shard we want to read
max_num_partitions_per_shard: usize,
// Minimum number of the most recent writes per partition we want to count
// to prioritize partitions
min_recent_ingested_files: usize,
) -> Result<Vec<PartitionParam>> {
let mut candidates =
Vec::with_capacity(self.sequencers.len() * max_num_partitions_per_sequencer);
Vec::with_capacity(self.shards.len() * max_num_partitions_per_shard);
let mut repos = self.catalog.repositories().await;
for sequencer_id in &self.sequencers {
for shard_id in &self.shards {
let attributes = Attributes::from([
("sequencer_id", format!("{}", *sequencer_id).into()),
("shard_id", format!("{}", *shard_id).into()),
("partition_type", "hot".into()),
]);
@ -276,19 +276,19 @@ impl Compactor {
let mut partitions = repos
.parquet_files()
.recent_highest_throughput_partitions(
*sequencer_id,
*shard_id,
num_hours,
min_recent_ingested_files,
max_num_partitions_per_sequencer,
max_num_partitions_per_shard,
)
.await
.context(HighestThroughputPartitionsSnafu {
sequencer_id: *sequencer_id,
shard_id: *shard_id,
})?;
if !partitions.is_empty() {
debug!(
sequencer_id = sequencer_id.get(),
shard_id = shard_id.get(),
num_hours,
n = partitions.len(),
"found high-throughput partitions"
@ -299,9 +299,9 @@ impl Compactor {
}
}
// Record metric for candidates per sequencer
// Record metric for candidates per shard
debug!(
sequencer_id = sequencer_id.get(),
shard_id = shard_id.get(),
n = num_partitions,
"hot compaction candidates",
);
@ -319,33 +319,33 @@ impl Compactor {
/// - Have some level 0 parquet files that need to be upgraded or compacted
pub async fn cold_partitions_to_compact(
&self,
// Max number of cold partitions per sequencer we want to compact
max_num_partitions_per_sequencer: usize,
// Max number of cold partitions per shard we want to compact
max_num_partitions_per_shard: usize,
) -> Result<Vec<PartitionParam>> {
let mut candidates =
Vec::with_capacity(self.sequencers.len() * max_num_partitions_per_sequencer);
Vec::with_capacity(self.shards.len() * max_num_partitions_per_shard);
let mut repos = self.catalog.repositories().await;
for sequencer_id in &self.sequencers {
for shard_id in &self.shards {
let attributes = Attributes::from([
("sequencer_id", format!("{}", *sequencer_id).into()),
("shard_id", format!("{}", *shard_id).into()),
("partition_type", "cold".into()),
]);
let mut partitions = repos
.parquet_files()
.most_level_0_files_partitions(*sequencer_id, 24, max_num_partitions_per_sequencer)
.most_level_0_files_partitions(*shard_id, 24, max_num_partitions_per_shard)
.await
.context(MostL0PartitionsSnafu {
sequencer_id: *sequencer_id,
shard_id: *shard_id,
})?;
let num_partitions = partitions.len();
candidates.append(&mut partitions);
// Record metric for candidates per sequencer
// Record metric for candidates per shard
debug!(
sequencer_id = sequencer_id.get(),
shard_id = shard_id.get(),
n = num_partitions,
"cold compaction candidates",
);
@ -461,9 +461,9 @@ impl PartitionCompactionCandidateWithInfo {
self.candidate.partition_id
}
/// Partition sequencer ID
pub fn sequencer_id(&self) -> SequencerId {
self.candidate.sequencer_id
/// Partition shard ID
pub fn shard_id(&self) -> ShardId {
self.candidate.shard_id
}
/// Partition namespace ID
@ -493,7 +493,7 @@ mod tests {
async fn test_hot_partitions_to_compact() {
let catalog = TestCatalog::new();
// Create a db with 2 sequencers, one with 4 empty partitions and the other one with one
// Create a db with 2 shards, one with 4 empty partitions and the other one with one
// empty partition
let mut txn = catalog.catalog.start_transaction().await.unwrap();
@ -514,39 +514,39 @@ mod tests {
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = txn
.sequencers()
let shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
let partition1 = txn
.partitions()
.create_or_get("one".into(), sequencer.id, table.id)
.create_or_get("one".into(), shard.id, table.id)
.await
.unwrap();
let partition2 = txn
.partitions()
.create_or_get("two".into(), sequencer.id, table.id)
.create_or_get("two".into(), shard.id, table.id)
.await
.unwrap();
let partition3 = txn
.partitions()
.create_or_get("three".into(), sequencer.id, table.id)
.create_or_get("three".into(), shard.id, table.id)
.await
.unwrap();
let partition4 = txn
.partitions()
.create_or_get("four".into(), sequencer.id, table.id)
.create_or_get("four".into(), shard.id, table.id)
.await
.unwrap();
// other sequencer
// other shard
let another_table = txn
.tables()
.create_or_get("another_test_table", namespace.id)
.await
.unwrap();
let another_sequencer = txn
.sequencers()
let another_shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(2))
.await
.unwrap();
@ -554,7 +554,7 @@ mod tests {
.partitions()
.create_or_get(
"another_partition".into(),
another_sequencer.id,
another_shard.id,
another_table.id,
)
.await
@ -571,7 +571,7 @@ mod tests {
let time_provider = Arc::new(SystemProvider::new());
let config = make_compactor_config();
let compactor = Compactor::new(
vec![sequencer.id, another_sequencer.id],
vec![shard.id, another_shard.id], // XXXJPG mismatch!
Arc::clone(&catalog.catalog),
ParquetStorage::new(Arc::clone(&catalog.object_store)),
Arc::new(Executor::new(1)),
@ -595,7 +595,7 @@ mod tests {
// Basic parquet info
let p1 = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: table.id,
partition_id: partition1.id,
@ -708,13 +708,13 @@ mod tests {
assert_eq!(candidates[0].partition_id, partition3.id);
// --------------------------------------
// Case 6: has partition candidates for 2 sequencers
// Case 6: has partition candidates for 2 shards
//
// The another_sequencer now has non-deleted level-0 file ingested 5 hours ago
// The another_shard now has non-deleted level-0 file ingested 5 hours ago
let mut txn = catalog.catalog.start_transaction().await.unwrap();
let p6 = ParquetFileParams {
object_store_id: Uuid::new_v4(),
sequencer_id: another_sequencer.id,
shard_id: another_shard.id,
table_id: another_table.id,
partition_id: another_partition.id,
created_at: time_five_hour_ago,
@ -723,14 +723,14 @@ mod tests {
let _pf6 = txn.parquet_files().create(p6).await.unwrap();
txn.commit().await.unwrap();
//
// Will have 2 candidates, one for each sequencer
// Will have 2 candidates, one for each shard
let mut candidates = compactor.hot_partitions_to_compact(1, 1).await.unwrap();
candidates.sort();
assert_eq!(candidates.len(), 2);
assert_eq!(candidates[0].partition_id, partition3.id);
assert_eq!(candidates[0].sequencer_id, sequencer.id);
assert_eq!(candidates[0].shard_id, shard.id);
assert_eq!(candidates[1].partition_id, another_partition.id);
assert_eq!(candidates[1].sequencer_id, another_sequencer.id);
assert_eq!(candidates[1].shard_id, another_shard.id);
// Add info to partition
let partitions_with_info = compactor.add_info_to_partitions(&candidates).await.unwrap();
@ -762,7 +762,7 @@ mod tests {
let split_percentage = 80;
let max_concurrent_size_bytes = 100_000;
let max_cold_concurrent_size_bytes = 90_000;
let max_number_partitions_per_sequencer = 1;
let max_number_partitions_per_shard = 1;
let min_number_recent_ingested_per_partition = 1;
let input_size_threshold_bytes = 300 * 1024 * 1024;
let cold_input_size_threshold_bytes = 600 * 1024 * 1024;
@ -775,7 +775,7 @@ mod tests {
split_percentage,
max_concurrent_size_bytes,
max_cold_concurrent_size_bytes,
max_number_partitions_per_sequencer,
max_number_partitions_per_shard,
min_number_recent_ingested_per_partition,
input_size_threshold_bytes,
cold_input_size_threshold_bytes,
@ -789,7 +789,7 @@ mod tests {
async fn test_cold_partitions_to_compact() {
let catalog = TestCatalog::new();
// Create a db with 2 sequencers, one with 4 empty partitions and the other one with one
// Create a db with 2 shards, one with 4 empty partitions and the other one with one
// empty partition
let mut txn = catalog.catalog.start_transaction().await.unwrap();
@ -810,44 +810,44 @@ mod tests {
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = txn
.sequencers()
let shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
let partition1 = txn
.partitions()
.create_or_get("one".into(), sequencer.id, table.id)
.create_or_get("one".into(), shard.id, table.id)
.await
.unwrap();
let partition2 = txn
.partitions()
.create_or_get("two".into(), sequencer.id, table.id)
.create_or_get("two".into(), shard.id, table.id)
.await
.unwrap();
let partition3 = txn
.partitions()
.create_or_get("three".into(), sequencer.id, table.id)
.create_or_get("three".into(), shard.id, table.id)
.await
.unwrap();
let partition4 = txn
.partitions()
.create_or_get("four".into(), sequencer.id, table.id)
.create_or_get("four".into(), shard.id, table.id)
.await
.unwrap();
let partition5 = txn
.partitions()
.create_or_get("five".into(), sequencer.id, table.id)
.create_or_get("five".into(), shard.id, table.id)
.await
.unwrap();
// other sequencer
// other shard
let another_table = txn
.tables()
.create_or_get("another_test_table", namespace.id)
.await
.unwrap();
let another_sequencer = txn
.sequencers()
let another_shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(2))
.await
.unwrap();
@ -855,7 +855,7 @@ mod tests {
.partitions()
.create_or_get(
"another_partition".into(),
another_sequencer.id,
another_shard.id,
another_table.id,
)
.await
@ -872,7 +872,7 @@ mod tests {
let time_provider = Arc::new(SystemProvider::new());
let config = make_compactor_config();
let compactor = Compactor::new(
vec![sequencer.id, another_sequencer.id],
vec![shard.id, another_shard.id],
Arc::clone(&catalog.catalog),
ParquetStorage::new(Arc::clone(&catalog.object_store)),
Arc::new(Executor::new(1)),
@ -892,7 +892,7 @@ mod tests {
// Basic parquet info
let p1 = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: table.id,
partition_id: partition1.id,
@ -1017,26 +1017,26 @@ mod tests {
assert_eq!(candidates.len(), 1);
assert_eq!(candidates[0].partition_id, partition4.id);
// Ask for 2 partitions per sequencer; get partition4 and partition2
// Ask for 2 partitions per shard; get partition4 and partition2
let candidates = compactor.cold_partitions_to_compact(2).await.unwrap();
assert_eq!(candidates.len(), 2);
assert_eq!(candidates[0].partition_id, partition4.id);
assert_eq!(candidates[1].partition_id, partition2.id);
// Ask for 3 partitions per sequencer; still get only partition4 and partition2
// Ask for 3 partitions per shard; still get only partition4 and partition2
let candidates = compactor.cold_partitions_to_compact(3).await.unwrap();
assert_eq!(candidates.len(), 2);
assert_eq!(candidates[0].partition_id, partition4.id);
assert_eq!(candidates[1].partition_id, partition2.id);
// --------------------------------------
// Case 6: has partition candidates for 2 sequencers
// Case 6: has partition candidates for 2 shards
//
// The another_sequencer now has non-deleted level-0 file ingested 38 hours ago
// The another_shard now has non-deleted level-0 file ingested 38 hours ago
let mut txn = catalog.catalog.start_transaction().await.unwrap();
let p6 = ParquetFileParams {
object_store_id: Uuid::new_v4(),
sequencer_id: another_sequencer.id,
shard_id: another_shard.id,
table_id: another_table.id,
partition_id: another_partition.id,
created_at: time_38_hour_ago,
@ -1045,25 +1045,25 @@ mod tests {
let _pf6 = txn.parquet_files().create(p6).await.unwrap();
txn.commit().await.unwrap();
// Will have 2 candidates, one for each sequencer
// Will have 2 candidates, one for each shard
let mut candidates = compactor.cold_partitions_to_compact(1).await.unwrap();
candidates.sort();
assert_eq!(candidates.len(), 2);
assert_eq!(candidates[0].partition_id, partition4.id);
assert_eq!(candidates[0].sequencer_id, sequencer.id);
assert_eq!(candidates[0].shard_id, shard.id);
assert_eq!(candidates[1].partition_id, another_partition.id);
assert_eq!(candidates[1].sequencer_id, another_sequencer.id);
assert_eq!(candidates[1].shard_id, another_shard.id);
// Ask for 2 candidates per sequencer; get back 3: 2 from sequencer and 1 from
// another_sequencer
// Ask for 2 candidates per shard; get back 3: 2 from shard and 1 from
// another_shard
let mut candidates = compactor.cold_partitions_to_compact(2).await.unwrap();
candidates.sort();
assert_eq!(candidates.len(), 3);
assert_eq!(candidates[0].partition_id, partition2.id);
assert_eq!(candidates[0].sequencer_id, sequencer.id);
assert_eq!(candidates[0].shard_id, shard.id);
assert_eq!(candidates[1].partition_id, partition4.id);
assert_eq!(candidates[1].sequencer_id, sequencer.id);
assert_eq!(candidates[1].shard_id, shard.id);
assert_eq!(candidates[2].partition_id, another_partition.id);
assert_eq!(candidates[2].sequencer_id, another_sequencer.id);
assert_eq!(candidates[2].shard_id, another_shard.id);
}
}

View File

@ -68,7 +68,7 @@ impl GarbageCollector {
let path = ParquetFilePath::new(
catalog_record.namespace_id,
catalog_record.table_id,
catalog_record.sequencer_id,
catalog_record.shard_id,
catalog_record.partition_id,
catalog_record.object_store_id,
);
@ -114,7 +114,7 @@ mod tests {
let path = ParquetFilePath::new(
catalog_record.namespace_id,
catalog_record.table_id,
catalog_record.sequencer_id,
catalog_record.shard_id,
catalog_record.partition_id,
catalog_record.object_store_id,
);
@ -159,14 +159,14 @@ mod tests {
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = txn
.sequencers()
let shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
let partition = txn
.partitions()
.create_or_get("one".into(), sequencer.id, table.id)
.create_or_get("one".into(), shard.id, table.id)
.await
.unwrap();
@ -174,7 +174,7 @@ mod tests {
let max_time = Timestamp::new(10);
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: partition.table_id,
partition_id: partition.id,
@ -240,14 +240,14 @@ mod tests {
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = txn
.sequencers()
let shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
let partition = txn
.partitions()
.create_or_get("one".into(), sequencer.id, table.id)
.create_or_get("one".into(), shard.id, table.id)
.await
.unwrap();
@ -255,7 +255,7 @@ mod tests {
let max_time = Timestamp::new(10);
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: partition.table_id,
partition_id: partition.id,
@ -325,14 +325,14 @@ mod tests {
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = txn
.sequencers()
let shard = txn
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
let partition = txn
.partitions()
.create_or_get("one".into(), sequencer.id, table.id)
.create_or_get("one".into(), shard.id, table.id)
.await
.unwrap();
@ -340,7 +340,7 @@ mod tests {
let max_time = Timestamp::new(10);
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: partition.table_id,
partition_id: partition.id,

View File

@ -51,7 +51,7 @@ pub(crate) async fn compact_hot_partition(
partition: PartitionCompactionCandidateWithInfo,
) -> Result<(), Error> {
let start_time = compactor.time_provider.now();
let sequencer_id = partition.sequencer_id();
let shard_id = partition.shard_id();
let parquet_files_for_compaction =
parquet_file_lookup::ParquetFilesForCompaction::for_partition(
@ -85,7 +85,7 @@ pub(crate) async fn compact_hot_partition(
.context(CombiningSnafu);
let attributes = Attributes::from([
("sequencer_id", format!("{}", sequencer_id).into()),
("shard_id", format!("{}", shard_id).into()),
("partition_type", "hot".into()),
]);
if let Some(delta) = compactor
@ -106,7 +106,7 @@ pub(crate) async fn compact_cold_partition(
partition: PartitionCompactionCandidateWithInfo,
) -> Result<(), Error> {
let start_time = compactor.time_provider.now();
let sequencer_id = partition.sequencer_id();
let shard_id = partition.shard_id();
let parquet_files_for_compaction =
parquet_file_lookup::ParquetFilesForCompaction::for_partition(
@ -153,7 +153,7 @@ pub(crate) async fn compact_cold_partition(
};
let attributes = Attributes::from([
("sequencer_id", format!("{}", sequencer_id).into()),
("shard_id", format!("{}", shard_id).into()),
("partition_type", "cold".into()),
]);
if let Some(delta) = compactor
@ -250,7 +250,7 @@ mod tests {
let config = make_compactor_config();
let metrics = Arc::new(metric::Registry::new());
let compactor = Compactor::new(
vec![sequencer.sequencer.id],
vec![sequencer.shard.id],
Arc::clone(&catalog.catalog),
ParquetStorage::new(Arc::clone(&catalog.object_store)),
Arc::new(Executor::new(1)),
@ -325,7 +325,7 @@ mod tests {
partition.create_parquet_file(builder).await;
// should have 4 level-0 files before compacting
let count = catalog.count_level_0_files(sequencer.sequencer.id).await;
let count = catalog.count_level_0_files(sequencer.shard.id).await;
assert_eq!(count, 4);
// ------------------------------------------------
@ -472,7 +472,7 @@ mod tests {
let config = make_compactor_config();
let metrics = Arc::new(metric::Registry::new());
let compactor = Compactor::new(
vec![sequencer.sequencer.id],
vec![sequencer.shard.id],
Arc::clone(&catalog.catalog),
ParquetStorage::new(Arc::clone(&catalog.object_store)),
Arc::new(Executor::new(1)),
@ -547,7 +547,7 @@ mod tests {
partition.create_parquet_file(builder).await;
// should have 4 level-0 files before compacting
let count = catalog.count_level_0_files(sequencer.sequencer.id).await;
let count = catalog.count_level_0_files(sequencer.shard.id).await;
assert_eq!(count, 4);
// ------------------------------------------------
@ -659,7 +659,7 @@ mod tests {
let config = make_compactor_config();
let metrics = Arc::new(metric::Registry::new());
let compactor = Compactor::new(
vec![sequencer.sequencer.id],
vec![sequencer.shard.id],
Arc::clone(&catalog.catalog),
ParquetStorage::new(Arc::clone(&catalog.object_store)),
Arc::new(Executor::new(1)),
@ -693,7 +693,7 @@ mod tests {
partition.create_parquet_file(builder).await;
// should have 1 level-0 file before compacting
let count = catalog.count_level_0_files(sequencer.sequencer.id).await;
let count = catalog.count_level_0_files(sequencer.shard.id).await;
assert_eq!(count, 1);
// ------------------------------------------------

View File

@ -260,7 +260,7 @@ pub(crate) async fn compact_parquet_files(
let meta = IoxMetadata {
object_store_id: Uuid::new_v4(),
creation_timestamp: time_provider.now(),
sequencer_id: partition.sequencer_id(),
shard_id: partition.shard_id(),
namespace_id: partition.namespace_id(),
namespace_name: partition.namespace.name.clone().into(),
table_id: partition.table.id,
@ -320,8 +320,8 @@ pub(crate) async fn compact_parquet_files(
info!(?partition_id, "compaction complete");
let attributes = Attributes::from([(
"sequencer_id",
format!("{}", partition.sequencer_id()).into(),
"shard_id",
format!("{}", partition.shard_id()).into(),
)]);
let compaction_input_file_bytes = compaction_input_file_bytes.recorder(attributes);
for size in file_sizes {
@ -360,7 +360,7 @@ fn to_queryable_parquet_chunk(
trace!(
parquet_file_id=?file.id,
parquet_file_sequencer_id=?file.sequencer_id,
parquet_file_shard_id=?file.shard_id,
parquet_file_namespace_id=?file.namespace_id,
parquet_file_table_id=?file.table_id,
parquet_file_partition_id=?file.partition_id,
@ -466,7 +466,7 @@ mod tests {
use super::*;
use arrow::record_batch::RecordBatch;
use arrow_util::assert_batches_sorted_eq;
use data_types::{ColumnType, PartitionParam, SequencerId};
use data_types::{ColumnType, PartitionParam, ShardId};
use iox_tests::util::{TestCatalog, TestParquetFileBuilder, TestTable};
use metric::U64HistogramOptions;
use parquet_file::ParquetFilePath;
@ -527,7 +527,7 @@ mod tests {
namespace: Arc::new(ns.namespace.clone()),
candidate: PartitionParam {
partition_id: partition.partition.id,
sequencer_id: partition.partition.sequencer_id,
shard_id: partition.partition.shard_id,
namespace_id: ns.namespace.id,
table_id: partition.partition.table_id,
},
@ -649,7 +649,7 @@ mod tests {
..
} = test_setup().await;
let compaction_input_file_bytes = metrics();
let sequencer_id = candidate_partition.sequencer_id();
let shard_id = candidate_partition.shard_id();
let files = vec![];
let result = compact_parquet_files(
@ -669,7 +669,7 @@ mod tests {
// No metrics recorded because the compaction didn't succeed
assert_eq!(
extract_byte_metrics(&compaction_input_file_bytes, sequencer_id),
extract_byte_metrics(&compaction_input_file_bytes, shard_id),
ExtractedByteMetrics {
sample_count: 0,
buckets_with_counts: vec![],
@ -689,7 +689,7 @@ mod tests {
} = test_setup().await;
let table_id = candidate_partition.table_id();
let compaction_input_file_bytes = metrics();
let sequencer_id = candidate_partition.sequencer_id();
let shard_id = candidate_partition.shard_id();
let parquet_file = parquet_files.remove(0);
compact_parquet_files(
@ -732,7 +732,7 @@ mod tests {
// Verify the metrics
assert_eq!(
extract_byte_metrics(&compaction_input_file_bytes, sequencer_id),
extract_byte_metrics(&compaction_input_file_bytes, shard_id),
ExtractedByteMetrics {
sample_count: 1,
buckets_with_counts: vec![(BUCKET_500_KB, 1)],
@ -751,7 +751,7 @@ mod tests {
parquet_files,
} = test_setup().await;
let compaction_input_file_bytes = metrics();
let sequencer_id = candidate_partition.sequencer_id();
let shard_id = candidate_partition.shard_id();
compact_parquet_files(
parquet_files.into_iter().take(4).collect(),
@ -792,7 +792,7 @@ mod tests {
// Verify the metrics
assert_eq!(
extract_byte_metrics(&compaction_input_file_bytes, sequencer_id),
extract_byte_metrics(&compaction_input_file_bytes, shard_id),
ExtractedByteMetrics {
sample_count: 4,
buckets_with_counts: vec![(BUCKET_500_KB, 4)],
@ -835,7 +835,7 @@ mod tests {
parquet_files,
} = test_setup().await;
let compaction_input_file_bytes = metrics();
let sequencer_id = candidate_partition.sequencer_id();
let shard_id = candidate_partition.shard_id();
compact_parquet_files(
parquet_files.into_iter().take(5).collect(),
@ -875,7 +875,7 @@ mod tests {
// Verify the metrics
assert_eq!(
extract_byte_metrics(&compaction_input_file_bytes, sequencer_id),
extract_byte_metrics(&compaction_input_file_bytes, shard_id),
ExtractedByteMetrics {
sample_count: 5,
buckets_with_counts: vec![(BUCKET_500_KB, 4), (u64::MAX, 1)],
@ -931,7 +931,7 @@ mod tests {
parquet_files,
} = test_setup().await;
let compaction_input_file_bytes = metrics();
let sequencer_id = candidate_partition.sequencer_id();
let shard_id = candidate_partition.shard_id();
let files_to_compact: Vec<_> = parquet_files.into_iter().take(5).collect();
@ -977,7 +977,7 @@ mod tests {
// Verify the metrics
assert_eq!(
extract_byte_metrics(&compaction_input_file_bytes, sequencer_id),
extract_byte_metrics(&compaction_input_file_bytes, shard_id),
ExtractedByteMetrics {
sample_count: 5,
buckets_with_counts: vec![(BUCKET_500_KB, 4), (u64::MAX, 1)],
@ -1020,7 +1020,7 @@ mod tests {
parquet_files,
} = test_setup().await;
let compaction_input_file_bytes = metrics();
let sequencer_id = candidate_partition.sequencer_id();
let shard_id = candidate_partition.shard_id();
compact_parquet_files(
parquet_files,
@ -1058,7 +1058,7 @@ mod tests {
// Verify the metrics
assert_eq!(
extract_byte_metrics(&compaction_input_file_bytes, sequencer_id),
extract_byte_metrics(&compaction_input_file_bytes, shard_id),
ExtractedByteMetrics {
sample_count: 6,
buckets_with_counts: vec![(BUCKET_500_KB, 4), (u64::MAX, 2)],
@ -1147,9 +1147,9 @@ mod tests {
fn extract_byte_metrics(
metric: &Metric<U64Histogram>,
sequencer_id: SequencerId,
shard_id: ShardId,
) -> ExtractedByteMetrics {
let attributes = Attributes::from([("sequencer_id", format!("{}", sequencer_id).into())]);
let attributes = Attributes::from([("shard_id", format!("{}", shard_id).into())]);
let (sample_count, buckets_with_counts) =
if let Some(observer) = metric.get_observer(&attributes) {

View File

@ -327,7 +327,7 @@ mod tests {
use super::*;
use data_types::{
ColumnSet, CompactionLevel, NamespaceId, ParquetFileId, PartitionId, SequenceNumber,
SequencerId, TableId, Timestamp,
ShardId, TableId, Timestamp,
};
use metric::{ObservationBucket, U64HistogramOptions};
use std::sync::Arc;
@ -1324,7 +1324,7 @@ mod tests {
ParquetFile {
id: ParquetFileId::new(id),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
namespace_id: NamespaceId::new(3),
table_id: TableId::new(4),
partition_id: PartitionId::new(5),

View File

@ -133,7 +133,7 @@ impl ParquetFileWithTombstone {
trace!(
parquet_file_id=?self.id,
parquet_file_sequencer_id=?self.sequencer_id,
parquet_file_shard_id=?self.shard_id,
parquet_file_namespace_id=?self.namespace_id,
parquet_file_table_id=?self.table_id,
parquet_file_partition_id=?self.partition_id,

View File

@ -153,17 +153,13 @@ impl PgHasArrayType for ColumnId {
}
}
/// Unique ID for a `Sequencer`. Note this is NOT the same as the
/// "sequencer_id" in the `write_buffer` which currently means
/// "kafka partition".
///
/// <https://github.com/influxdata/influxdb_iox/issues/4237>
/// Unique ID for a `Shard`.
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)]
#[sqlx(transparent)]
pub struct SequencerId(i64);
pub struct ShardId(i64);
#[allow(missing_docs)]
impl SequencerId {
impl ShardId {
pub fn new(v: i64) -> Self {
Self(v)
}
@ -172,7 +168,7 @@ impl SequencerId {
}
}
impl std::fmt::Display for SequencerId {
impl std::fmt::Display for ShardId {
fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
write!(f, "{}", self.0)
}
@ -199,16 +195,16 @@ impl std::fmt::Display for KafkaPartition {
}
}
/// Potential configurations of ingester connections for the querier to associate with a sequencer.
/// Potential configurations of ingester connections for the querier to associate with a shard.
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum IngesterMapping {
/// Deliberately not mapping this sequencer to an ingester. If the querier gets a query for
/// this sequencer, it should return an error.
/// Deliberately not mapping this shard to an ingester. If the querier gets a query for
/// this shard, it should return an error.
NotMapped,
/// Deliberately not contacting ingesters for this sequencer. If the querier gets a query for
/// this sequencer, it should only return persisted data.
/// Deliberately not contacting ingesters for this shard. If the querier gets a query for
/// this shard, it should only return persisted data.
Ignore,
/// The address of the ingester to contact for this sequencer.
/// The address of the ingester to contact for this shard.
Addr(Arc<str>),
}
@ -233,12 +229,12 @@ impl std::fmt::Display for PartitionId {
}
}
/// Combination of Sequencer ID, Table ID, and Partition ID useful for identifying groups of
/// Combination of Shard ID, Table ID, and Partition ID useful for identifying groups of
/// Parquet files to be compacted together.
#[derive(Debug, Clone, Copy, PartialEq, PartialOrd, Eq, Ord)]
pub struct TablePartition {
/// The sequencer ID
pub sequencer_id: SequencerId,
/// The shard ID
pub shard_id: ShardId,
/// The table ID
pub table_id: TableId,
/// The partition ID
@ -247,9 +243,9 @@ pub struct TablePartition {
impl TablePartition {
/// Combine the relevant parts
pub fn new(sequencer_id: SequencerId, table_id: TableId, partition_id: PartitionId) -> Self {
pub fn new(shard_id: ShardId, table_id: TableId, partition_id: PartitionId) -> Self {
Self {
sequencer_id,
shard_id,
table_id,
partition_id,
}
@ -277,7 +273,7 @@ impl std::fmt::Display for TombstoneId {
}
}
/// A sequence number from a `Sequencer` (kafka partition)
/// A sequence number from a `router::Sequencer` (kafka partition)
#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash, sqlx::Type)]
#[sqlx(transparent)]
pub struct SequenceNumber(i64);
@ -710,15 +706,15 @@ pub fn column_type_from_field(field_value: &FieldValue) -> ColumnType {
}
}
/// Data object for a sequencer. Only one sequencer record can exist for a given
/// Data object for a shard. Only one shard record can exist for a given
/// kafka topic and partition (enforced via uniqueness constraint).
#[derive(Debug, Copy, Clone, PartialEq, Eq, sqlx::FromRow)]
pub struct Sequencer {
/// the id of the sequencer
pub id: SequencerId,
/// the topic the sequencer is reading from
pub struct Shard {
/// the id of the shard
pub id: ShardId,
/// the topic the shard is reading from
pub kafka_topic_id: KafkaTopicId,
/// the kafka partition the sequencer is reading from
/// the kafka partition the shard is reading from
pub kafka_partition: KafkaPartition,
/// The minimum unpersisted sequence number. Because different tables
/// can be persisted at different times, it is possible some data has been persisted
@ -784,14 +780,15 @@ impl sqlx::Decode<'_, sqlx::Postgres> for PartitionKey {
}
}
/// Data object for a partition. The combination of sequencer, table and key are unique (i.e. only
/// Data object for a partition. The combination of shard, table and key are unique (i.e. only
/// one record can exist for each combo)
#[derive(Debug, Clone, PartialEq, Eq, sqlx::FromRow)]
pub struct Partition {
/// the id of the partition
pub id: PartitionId,
/// the sequencer the data in the partition arrived from
pub sequencer_id: SequencerId,
/// the shard the data in the partition arrived from
#[sqlx(rename = "sequencer_id")]
pub shard_id: ShardId,
/// the table the partition is under
pub table_id: TableId,
/// the string key of the partition
@ -849,8 +846,9 @@ pub struct PartitionInfo {
pub struct PartitionParam {
/// the partition
pub partition_id: PartitionId,
/// the partition's sequencer
pub sequencer_id: SequencerId,
/// the partition's shard
#[sqlx(rename = "sequencer_id")]
pub shard_id: ShardId,
/// the partition's namespace
pub namespace_id: NamespaceId,
/// the partition's table
@ -864,9 +862,10 @@ pub struct Tombstone {
pub id: TombstoneId,
/// the table the tombstone is associated with
pub table_id: TableId,
/// the sequencer the tombstone was sent through
pub sequencer_id: SequencerId,
/// the sequence nubmer assigned to the tombstone from the sequencer
/// the shard the tombstone was sent through
#[sqlx(rename = "sequencer_id")]
pub shard_id: ShardId,
/// the sequence number assigned to the tombstone from the `router::Sequencer`
pub sequence_number: SequenceNumber,
/// the min time (inclusive) that the delete applies to
pub min_time: Timestamp,
@ -937,8 +936,9 @@ impl Deref for ColumnSet {
pub struct ParquetFile {
/// the id of the file in the catalog
pub id: ParquetFileId,
/// the sequencer that sequenced writes that went into this file
pub sequencer_id: SequencerId,
/// the shard that sequenced writes that went into this file
#[sqlx(rename = "sequencer_id")]
pub shard_id: ShardId,
/// the namespace
pub namespace_id: NamespaceId,
/// the table
@ -1003,8 +1003,8 @@ impl ParquetFile {
/// Data for a parquet file to be inserted into the catalog.
#[derive(Debug, Clone, PartialEq, Eq)]
pub struct ParquetFileParams {
/// the sequencer that sequenced writes that went into this file
pub sequencer_id: SequencerId,
/// the shard that sequenced writes that went into this file
pub shard_id: ShardId,
/// the namespace
pub namespace_id: NamespaceId,
/// the table

View File

@ -116,7 +116,7 @@ mod tests {
use chrono::TimeZone;
use data_types::{
ColumnId, ColumnSet, CompactionLevel, KafkaPartition, NamespaceId, ParquetFile,
ParquetFileParams, PartitionId, SequenceNumber, SequencerId, TableId, Timestamp,
ParquetFileParams, PartitionId, SequenceNumber, ShardId, TableId, Timestamp,
};
use iox_catalog::{interface::Catalog, mem::MemCatalog};
use object_store::path::Path;
@ -145,19 +145,19 @@ mod tests {
.create_or_get("test_table", namespace.id)
.await
.unwrap();
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
let partition = repos
.partitions()
.create_or_get("one".into(), sequencer.id, table.id)
.create_or_get("one".into(), shard.id, table.id)
.await
.unwrap();
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: partition.table_id,
partition_id: partition.id,
@ -190,7 +190,7 @@ mod tests {
let location = ParquetFilePath::new(
file_in_catalog.namespace_id,
file_in_catalog.table_id,
file_in_catalog.sequencer_id,
file_in_catalog.shard_id,
file_in_catalog.partition_id,
file_in_catalog.object_store_id,
)
@ -218,7 +218,7 @@ mod tests {
let location = ParquetFilePath::new(
NamespaceId::new(1),
TableId::new(2),
SequencerId::new(3),
ShardId::new(3),
PartitionId::new(4),
Uuid::new_v4(),
)
@ -264,7 +264,7 @@ mod tests {
let location = ParquetFilePath::new(
file_in_catalog.namespace_id,
file_in_catalog.table_id,
file_in_catalog.sequencer_id,
file_in_catalog.shard_id,
file_in_catalog.partition_id,
file_in_catalog.object_store_id,
)
@ -292,7 +292,7 @@ mod tests {
let location = ParquetFilePath::new(
NamespaceId::new(1),
TableId::new(2),
SequencerId::new(3),
ShardId::new(3),
PartitionId::new(4),
Uuid::new_v4(),
)

View File

@ -6,7 +6,7 @@ use clap_blocks::{catalog_dsn::CatalogDsnConfig, object_store::ObjectStoreConfig
use data_types::{
ColumnId, ColumnSet, ColumnType, KafkaPartition, NamespaceId,
NamespaceSchema as CatalogNamespaceSchema, ParquetFile as CatalogParquetFile,
ParquetFileParams, PartitionId, SequenceNumber, SequencerId, TableId, Timestamp,
ParquetFileParams, PartitionId, SequenceNumber, ShardId, TableId, Timestamp,
};
use futures::future::join_all;
use influxdb_iox_client::{
@ -170,7 +170,7 @@ pub async fn command(connection: Connection, config: Config) -> Result<(), Error
let path = ParquetFilePath::new(
parquet_file.namespace_id,
parquet_file.table_id,
parquet_file.sequencer_id,
parquet_file.shard_id,
parquet_file.partition_id,
parquet_file.object_store_id,
);
@ -241,9 +241,9 @@ async fn load_schema(
let mut repos = catalog.repositories().await;
let kafka_topic = repos.kafka_topics().create_or_get(KAFKA_NAME).await?;
let query_pool = repos.query_pools().create_or_get(QUERY_POOL).await?;
// ensure there's a sequencer for this partition so it can be used later
let _sequencer = repos
.sequencers()
// ensure there's a shard for this partition so it can be used later
let _shard = repos
.shards()
.create_or_get(&kafka_topic, KafkaPartition::new(KAFKA_PARTITION))
.await?;
@ -309,22 +309,22 @@ async fn load_partition(
.get_by_name(KAFKA_NAME)
.await?
.expect("topic should have been inserted earlier");
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.get_by_topic_id_and_partition(topic.id, KafkaPartition::new(KAFKA_PARTITION))
.await?
.expect("sequencer should have been inserted earlier");
.expect("shard should have been inserted earlier");
let table = schema
.tables
.get(table_name)
.expect("table should have been loaded");
let partition = repos
.partitions()
.create_or_get(remote_partition.key.clone().into(), sequencer.id, table.id)
.create_or_get(remote_partition.key.clone().into(), shard.id, table.id)
.await?;
Ok(PartitionMapping {
sequencer_id: sequencer.id,
shard_id: shard.id,
table_id: table.id,
partition_id: partition.id,
remote_partition_id: remote_partition.id,
@ -350,7 +350,7 @@ async fn load_parquet_files(
None => {
println!("creating file {} in catalog", uuid);
let params = ParquetFileParams {
sequencer_id: partition_mapping.sequencer_id,
shard_id: partition_mapping.shard_id,
namespace_id,
table_id: partition_mapping.table_id,
partition_id: partition_mapping.partition_id,
@ -378,9 +378,9 @@ async fn load_parquet_files(
Ok(files)
}
// keeps a mapping of the locally created partition and sequence to the remote partition id
// keeps a mapping of the locally created partition and shard to the remote partition id
struct PartitionMapping {
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
partition_id: PartitionId,
remote_partition_id: i64,
@ -514,7 +514,7 @@ mod tests {
async fn load_parquet_files() {
let metrics = Arc::new(metric::Registry::new());
let catalog: Arc<dyn Catalog> = Arc::new(MemCatalog::new(Arc::clone(&metrics)));
let sequencer;
let shard;
let namespace;
let table;
let partition;
@ -527,8 +527,8 @@ mod tests {
.await
.unwrap();
let query_pool = repos.query_pools().create_or_get(QUERY_POOL).await.unwrap();
sequencer = repos
.sequencers()
shard = repos
.shards()
.create_or_get(&kafka_topic, KafkaPartition::new(KAFKA_PARTITION))
.await
.unwrap();
@ -544,13 +544,13 @@ mod tests {
.unwrap();
partition = repos
.partitions()
.create_or_get("1970-01-01".into(), sequencer.id, table.id)
.create_or_get("1970-01-01".into(), shard.id, table.id)
.await
.unwrap();
}
let partition_mapping = PartitionMapping {
sequencer_id: sequencer.id,
shard_id: shard.id,
table_id: table.id,
partition_id: partition.id,
remote_partition_id: 4,
@ -589,12 +589,12 @@ mod tests {
.await
.unwrap();
// the inserted parquet file should have sequencer, namespace, table, and partition ids
// the inserted parquet file should have shard, namespace, table, and partition ids
// that match with the ones in the catalog, not the remote. The other values should
// match those of the remote.
let expected = vec![CatalogParquetFile {
id: ParquetFileId::new(1),
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: table.id,
partition_id: partition.id,

View File

@ -121,7 +121,7 @@ pub async fn compact_persisting_batch(
let iox_metadata = IoxMetadata {
object_store_id: batch.object_store_id,
creation_timestamp: time_provider.now(),
sequencer_id: batch.sequencer_id,
shard_id: batch.shard_id,
namespace_id: NamespaceId::new(namespace_id),
namespace_name: Arc::from(namespace_name.as_str()),
table_id: batch.table_id,
@ -181,7 +181,7 @@ mod tests {
make_persisting_batch, make_queryable_batch, make_queryable_batch_with_deletes,
};
use arrow_util::assert_batches_eq;
use data_types::{Partition, PartitionId, SequencerId, TableId};
use data_types::{Partition, PartitionId, ShardId, TableId};
use iox_time::SystemProvider;
use mutable_batch_lp::lines_to_batches;
use schema::selection::Selection;
@ -205,12 +205,12 @@ mod tests {
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let table_id = 1;
let partition_id = 1;
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -234,7 +234,7 @@ mod tests {
table_name: table_name.into(),
partition: Partition {
id: PartitionId::new(partition_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_key: partition_key.into(),
sort_key: vec![],
@ -273,14 +273,14 @@ mod tests {
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let seq_num_end: i64 = seq_num_start; // one batch
let namespace_id = 1;
let table_id = 1;
let partition_id = 1;
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -304,7 +304,7 @@ mod tests {
table_name: table_name.into(),
partition: Partition {
id: PartitionId::new(partition_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_key: partition_key.into(),
sort_key: vec![],
@ -340,7 +340,7 @@ mod tests {
let expected_meta = make_meta(
uuid,
iox_metadata.creation_timestamp,
seq_id,
shard_id,
namespace_id,
namespace_name,
table_id,
@ -369,14 +369,14 @@ mod tests {
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let seq_num_end: i64 = seq_num_start; // one batch
let namespace_id = 1;
let table_id = 1;
let partition_id = 1;
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -400,7 +400,7 @@ mod tests {
table_name: table_name.into(),
partition: Partition {
id: PartitionId::new(partition_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_key: partition_key.into(),
// NO SORT KEY from the catalog here, first persisting batch
@ -438,7 +438,7 @@ mod tests {
let expected_meta = make_meta(
uuid,
iox_metadata.creation_timestamp,
seq_id,
shard_id,
namespace_id,
namespace_name,
table_id,
@ -468,14 +468,14 @@ mod tests {
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let seq_num_end: i64 = seq_num_start; // one batch
let namespace_id = 1;
let table_id = 1;
let partition_id = 1;
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -499,7 +499,7 @@ mod tests {
table_name: table_name.into(),
partition: Partition {
id: PartitionId::new(partition_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_key: partition_key.into(),
// SPECIFY A SORT KEY HERE to simulate a sort key being stored in the catalog
@ -539,7 +539,7 @@ mod tests {
let expected_meta = make_meta(
uuid,
iox_metadata.creation_timestamp,
seq_id,
shard_id,
namespace_id,
namespace_name,
table_id,
@ -568,14 +568,14 @@ mod tests {
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let seq_num_end: i64 = seq_num_start; // one batch
let namespace_id = 1;
let table_id = 1;
let partition_id = 1;
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -599,7 +599,7 @@ mod tests {
table_name: table_name.into(),
partition: Partition {
id: PartitionId::new(partition_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_key: partition_key.into(),
// SPECIFY A SORT KEY HERE to simulate a sort key being stored in the catalog
@ -640,7 +640,7 @@ mod tests {
let expected_meta = make_meta(
uuid,
iox_metadata.creation_timestamp,
seq_id,
shard_id,
namespace_id,
namespace_name,
table_id,
@ -672,14 +672,14 @@ mod tests {
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let seq_num_end: i64 = seq_num_start; // one batch
let namespace_id = 1;
let table_id = 1;
let partition_id = 1;
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -703,7 +703,7 @@ mod tests {
table_name: table_name.into(),
partition: Partition {
id: PartitionId::new(partition_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_key: partition_key.into(),
// SPECIFY A SORT KEY HERE to simulate a sort key being stored in the catalog
@ -749,7 +749,7 @@ mod tests {
let expected_meta = make_meta(
uuid,
iox_metadata.creation_timestamp,
seq_id,
shard_id,
namespace_id,
namespace_name,
table_id,

View File

@ -11,7 +11,7 @@ use async_trait::async_trait;
use backoff::{Backoff, BackoffConfig};
use data_types::{
DeletePredicate, KafkaPartition, NamespaceId, PartitionId, PartitionInfo, PartitionKey,
SequenceNumber, SequencerId, TableId, Timestamp, Tombstone,
SequenceNumber, ShardId, TableId, Timestamp, Tombstone,
};
use datafusion::physical_plan::SendableRecordBatchStream;
use dml::DmlOperation;
@ -34,7 +34,7 @@ use std::{
sync::Arc,
};
use uuid::Uuid;
use write_summary::SequencerProgress;
use write_summary::ShardProgress;
mod triggers;
use self::triggers::TestTriggers;
@ -42,8 +42,8 @@ use self::triggers::TestTriggers;
#[derive(Debug, Snafu)]
#[allow(missing_copy_implementations, missing_docs)]
pub enum Error {
#[snafu(display("Sequencer {} not found in data map", sequencer_id))]
SequencerNotFound { sequencer_id: SequencerId },
#[snafu(display("Shard {} not found in data map", shard_id))]
ShardNotFound { shard_id: ShardId },
#[snafu(display("Namespace {} not found in catalog", namespace))]
NamespaceNotFound { namespace: String },
@ -85,9 +85,9 @@ pub struct IngesterData {
catalog: Arc<dyn Catalog>,
/// This map gets set up on initialization of the ingester so it won't ever be modified.
/// The content of each SequenceData will get changed when more namespaces and tables
/// The content of each ShardData will get changed when more namespaces and tables
/// get ingested.
sequencers: BTreeMap<SequencerId, SequencerData>,
shards: BTreeMap<ShardId, ShardData>,
/// Executor for running queries and compacting and persisting
exec: Arc<Executor>,
@ -104,7 +104,7 @@ impl IngesterData {
pub fn new(
object_store: Arc<DynObjectStore>,
catalog: Arc<dyn Catalog>,
sequencers: BTreeMap<SequencerId, SequencerData>,
shards: BTreeMap<ShardId, ShardData>,
exec: Arc<Executor>,
backoff_config: BackoffConfig,
metrics: Arc<metric::Registry>,
@ -127,7 +127,7 @@ impl IngesterData {
Self {
store: ParquetStorage::new(object_store),
catalog,
sequencers,
shards,
exec,
backoff_config,
persisted_file_size_bytes,
@ -139,15 +139,15 @@ impl IngesterData {
&self.exec
}
/// Get sequencer data for specific sequencer.
/// Get shard data for specific shard.
#[allow(dead_code)] // Used in tests
pub(crate) fn sequencer(&self, sequencer_id: SequencerId) -> Option<&SequencerData> {
self.sequencers.get(&sequencer_id)
pub(crate) fn shard(&self, shard_id: ShardId) -> Option<&ShardData> {
self.shards.get(&shard_id)
}
/// Get iterator over sequencers (ID and data).
pub(crate) fn sequencers(&self) -> impl Iterator<Item = (&SequencerId, &SequencerData)> {
self.sequencers.iter()
/// Get iterator over shards (ID and data).
pub(crate) fn shards(&self) -> impl Iterator<Item = (&ShardId, &ShardData)> {
self.shards.iter()
}
/// Store the write or delete in the in memory buffer. Deletes will
@ -158,18 +158,18 @@ impl IngesterData {
/// be paused, this function will return true.
pub async fn buffer_operation(
&self,
sequencer_id: SequencerId,
shard_id: ShardId,
dml_operation: DmlOperation,
lifecycle_handle: &dyn LifecycleHandle,
) -> Result<bool> {
let sequencer_data = self
.sequencers
.get(&sequencer_id)
.context(SequencerNotFoundSnafu { sequencer_id })?;
sequencer_data
let shard_data = self
.shards
.get(&shard_id)
.context(ShardNotFoundSnafu { shard_id })?;
shard_data
.buffer_operation(
dml_operation,
sequencer_id,
shard_id,
self.catalog.as_ref(),
lifecycle_handle,
&self.exec,
@ -178,23 +178,23 @@ impl IngesterData {
}
/// Return the ingestion progress for the specified kafka
/// partitions. Returns an empty `SequencerProgress` for any kafka
/// partitions. Returns an empty `ShardProgress` for any kafka
/// partitions that this ingester doesn't know about.
pub(crate) async fn progresses(
&self,
partitions: Vec<KafkaPartition>,
) -> BTreeMap<KafkaPartition, SequencerProgress> {
) -> BTreeMap<KafkaPartition, ShardProgress> {
let mut progresses = BTreeMap::new();
for kafka_partition in partitions {
let sequencer_data = self
.sequencers
let shard_data = self
.shards
.iter()
.map(|(_, sequencer_data)| sequencer_data)
.find(|sequencer_data| sequencer_data.kafka_partition == kafka_partition);
.map(|(_, shard_data)| shard_data)
.find(|shard_data| shard_data.kafka_partition == kafka_partition);
let progress = match sequencer_data {
Some(sequencer_data) => sequencer_data.progress().await,
None => SequencerProgress::new(), // don't know about this sequencer
let progress = match shard_data {
Some(shard_data) => shard_data.progress().await,
None => ShardProgress::new(), // don't know about this shard
};
progresses.insert(kafka_partition, progress);
@ -204,19 +204,19 @@ impl IngesterData {
}
/// The Persister has a function to persist a given partition ID and to update the
/// assocated sequencer's `min_unpersisted_sequence_number`.
/// assocated shard's `min_unpersisted_sequence_number`.
#[async_trait]
pub trait Persister: Send + Sync + 'static {
/// Persits the partition ID. Will retry forever until it succeeds.
async fn persist(&self, partition_id: PartitionId);
/// Updates the sequencer's `min_unpersisted_sequence_number` in the catalog.
/// Updates the shard's `min_unpersisted_sequence_number` in the catalog.
/// This number represents the minimum that might be unpersisted, which is the
/// farthest back the ingester would need to read in the write buffer to ensure
/// that all data would be correctly replayed on startup.
async fn update_min_unpersisted_sequence_number(
&self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
);
}
@ -237,21 +237,21 @@ impl Persister for IngesterData {
// so someone can take a look.
let partition_info = partition_info
.unwrap_or_else(|| panic!("partition {} not found in catalog", partition_id));
let sequencer_data = self
.sequencers
.get(&partition_info.partition.sequencer_id)
let shard_data = self
.shards
.get(&partition_info.partition.shard_id)
.unwrap_or_else(|| {
panic!(
"sequencer state for {} not in ingester data",
partition_info.partition.sequencer_id
"shard state for {} not in ingester data",
partition_info.partition.shard_id
)
}); //{
let namespace = sequencer_data
let namespace = shard_data
.namespace(&partition_info.namespace_name)
.unwrap_or_else(|| {
panic!(
"namespace {} not in sequencer {} state",
partition_info.namespace_name, partition_info.partition.sequencer_id
"namespace {} not in shard {} state",
partition_info.namespace_name, partition_info.partition.shard_id
)
});
debug!(?partition_id, ?partition_info, "Persisting");
@ -366,8 +366,8 @@ impl Persister for IngesterData {
// Record metrics
let attributes = Attributes::from([(
"sequencer_id",
format!("{}", partition_info.partition.sequencer_id).into(),
"shard_id",
format!("{}", partition_info.partition.shard_id).into(),
)]);
self.persisted_file_size_bytes
.recorder(attributes)
@ -393,7 +393,7 @@ impl Persister for IngesterData {
async fn update_min_unpersisted_sequence_number(
&self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) {
Backoff::new(&self.backoff_config)
@ -401,8 +401,8 @@ impl Persister for IngesterData {
self.catalog
.repositories()
.await
.sequencers()
.update_min_unpersisted_sequence_number(sequencer_id, sequence_number)
.shards()
.update_min_unpersisted_sequence_number(shard_id, sequence_number)
.await
})
.await
@ -412,8 +412,8 @@ impl Persister for IngesterData {
/// Data of a Shard
#[derive(Debug)]
pub struct SequencerData {
/// The kafka partition for this sequencer
pub struct ShardData {
/// The kafka partition for this shard
kafka_partition: KafkaPartition,
// New namespaces can come in at any time so we need to be able to add new ones
@ -423,8 +423,8 @@ pub struct SequencerData {
namespace_count: U64Counter,
}
impl SequencerData {
/// Initialise a new [`SequencerData`] that emits metrics to `metrics`.
impl ShardData {
/// Initialise a new [`ShardData`] that emits metrics to `metrics`.
pub fn new(kafka_partition: KafkaPartition, metrics: Arc<metric::Registry>) -> Self {
let namespace_count = metrics
.register_metric::<U64Counter>(
@ -441,7 +441,7 @@ impl SequencerData {
}
}
/// Initialize new SequncerData with namespace for testing purpose only
/// Initialize new ShardData with namespace for testing purpose only
#[cfg(test)]
pub fn new_for_test(
kafka_partition: KafkaPartition,
@ -455,14 +455,14 @@ impl SequencerData {
}
}
/// Store the write or delete in the sequencer. Deletes will
/// Store the write or delete in the shard. Deletes will
/// be written into the catalog before getting stored in the buffer.
/// Any writes that create new IOx partitions will have those records
/// created in the catalog before putting into the buffer.
pub async fn buffer_operation(
&self,
dml_operation: DmlOperation,
sequencer_id: SequencerId,
shard_id: ShardId,
catalog: &dyn Catalog,
lifecycle_handle: &dyn LifecycleHandle,
executor: &Executor,
@ -478,7 +478,7 @@ impl SequencerData {
namespace_data
.buffer_operation(
dml_operation,
sequencer_id,
shard_id,
catalog,
lifecycle_handle,
executor,
@ -521,11 +521,11 @@ impl SequencerData {
Ok(data)
}
/// Return the progress of this sequencer
async fn progress(&self) -> SequencerProgress {
/// Return the progress of this shard
async fn progress(&self) -> ShardProgress {
let namespaces: Vec<_> = self.namespaces.read().values().map(Arc::clone).collect();
let mut progress = SequencerProgress::new();
let mut progress = ShardProgress::new();
for namespace_data in namespaces {
progress = progress.combine(namespace_data.progress().await);
@ -627,7 +627,7 @@ impl NamespaceData {
pub async fn buffer_operation(
&self,
dml_operation: DmlOperation,
sequencer_id: SequencerId,
shard_id: ShardId,
catalog: &dyn Catalog,
lifecycle_handle: &dyn LifecycleHandle,
executor: &Executor,
@ -658,7 +658,7 @@ impl NamespaceData {
for (t, b) in write.into_tables() {
let table_data = match self.table_data(&t) {
Some(t) => t,
None => self.insert_table(sequencer_id, &t, catalog).await?,
None => self.insert_table(shard_id, &t, catalog).await?,
};
{
@ -669,7 +669,7 @@ impl NamespaceData {
sequence_number,
b,
partition_key.clone(),
sequencer_id,
shard_id,
catalog,
lifecycle_handle,
)
@ -685,7 +685,7 @@ impl NamespaceData {
let table_name = delete.table_name().context(TableNotPresentSnafu)?;
let table_data = match self.table_data(table_name) {
Some(t) => t,
None => self.insert_table(sequencer_id, table_name, catalog).await?,
None => self.insert_table(shard_id, table_name, catalog).await?,
};
let mut table_data = table_data.write().await;
@ -694,7 +694,7 @@ impl NamespaceData {
.buffer_delete(
table_name,
delete.predicate(),
sequencer_id,
shard_id,
sequence_number,
catalog,
executor,
@ -743,7 +743,7 @@ impl NamespaceData {
.get_mut(&partition_info.partition.partition_key)
.and_then(|partition_data| {
partition_data.snapshot_to_persisting_batch(
partition_info.partition.sequencer_id,
partition_info.partition.shard_id,
partition_info.partition.table_id,
partition_info.partition.id,
&partition_info.table_name,
@ -766,14 +766,14 @@ impl NamespaceData {
/// Inserts the table or returns it if it happens to be inserted by some other thread
async fn insert_table(
&self,
sequencer_id: SequencerId,
shard_id: ShardId,
table_name: &str,
catalog: &dyn Catalog,
) -> Result<Arc<tokio::sync::RwLock<TableData>>> {
let mut repos = catalog.repositories().await;
let info = repos
.tables()
.get_table_persist_info(sequencer_id, self.namespace_id, table_name)
.get_table_persist_info(shard_id, self.namespace_id, table_name)
.await
.context(CatalogSnafu)?
.context(TableNotFoundSnafu { table_name })?;
@ -818,11 +818,11 @@ impl NamespaceData {
}
/// Return progress from this Namespace
async fn progress(&self) -> SequencerProgress {
async fn progress(&self) -> ShardProgress {
let tables: Vec<_> = self.tables.read().values().map(Arc::clone).collect();
// Consolidate progtress across partitions.
let mut progress = SequencerProgress::new()
let mut progress = ShardProgress::new()
// Properly account for any sequence number that is
// actively buffering and thus not yet completely
// readable.
@ -924,14 +924,14 @@ impl TableData {
sequence_number: SequenceNumber,
batch: MutableBatch,
partition_key: PartitionKey,
sequencer_id: SequencerId,
shard_id: ShardId,
catalog: &dyn Catalog,
lifecycle_handle: &dyn LifecycleHandle,
) -> Result<bool> {
let partition_data = match self.partition_data.get_mut(&partition_key) {
Some(p) => p,
None => {
self.insert_partition(partition_key.clone(), sequencer_id, catalog)
self.insert_partition(partition_key.clone(), shard_id, catalog)
.await?;
self.partition_data.get_mut(&partition_key).unwrap()
}
@ -946,7 +946,7 @@ impl TableData {
let should_pause = lifecycle_handle.log_write(
partition_data.id,
sequencer_id,
shard_id,
sequence_number,
batch.size(),
);
@ -959,7 +959,7 @@ impl TableData {
&mut self,
table_name: &str,
predicate: &DeletePredicate,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
catalog: &dyn Catalog,
executor: &Executor,
@ -972,7 +972,7 @@ impl TableData {
.tombstones()
.create_or_get(
self.table_id,
sequencer_id,
shard_id,
sequence_number,
min_time,
max_time,
@ -1013,13 +1013,13 @@ impl TableData {
async fn insert_partition(
&mut self,
partition_key: PartitionKey,
sequencer_id: SequencerId,
shard_id: ShardId,
catalog: &dyn Catalog,
) -> Result<()> {
let mut repos = catalog.repositories().await;
let partition = repos
.partitions()
.create_or_get(partition_key, sequencer_id, self.table_id)
.create_or_get(partition_key, shard_id, self.table_id)
.await
.context(CatalogSnafu)?;
@ -1042,8 +1042,8 @@ impl TableData {
}
/// Return progress from this Table
fn progress(&self) -> SequencerProgress {
let progress = SequencerProgress::new();
fn progress(&self) -> ShardProgress {
let progress = ShardProgress::new();
let progress = match self.parquet_max_sequence_number() {
Some(n) => progress.with_persisted(n),
None => progress,
@ -1085,13 +1085,13 @@ impl PartitionData {
/// Snapshot anything in the buffer and move all snapshot data into a persisting batch
pub fn snapshot_to_persisting_batch(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
partition_id: PartitionId,
table_name: &str,
) -> Option<Arc<PersistingBatch>> {
self.data
.snapshot_to_persisting(sequencer_id, table_id, partition_id, table_name)
.snapshot_to_persisting(shard_id, table_id, partition_id, table_name)
}
/// Snapshot whatever is in the buffer and return a new vec of the
@ -1217,7 +1217,7 @@ impl PartitionData {
}
/// Return the progress from this Partition
fn progress(&self) -> SequencerProgress {
fn progress(&self) -> ShardProgress {
self.data.progress()
}
}
@ -1366,7 +1366,7 @@ impl DataBuffer {
/// Panics if there is already a persisting batch.
pub fn snapshot_to_persisting(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
partition_id: PartitionId,
table_name: &str,
@ -1377,7 +1377,7 @@ impl DataBuffer {
if let Some(queryable_batch) = self.snapshot_to_queryable_batch(table_name, None) {
let persisting_batch = Arc::new(PersistingBatch {
sequencer_id,
shard_id,
table_id,
partition_id,
object_store_id: Uuid::new_v4(),
@ -1409,8 +1409,8 @@ impl DataBuffer {
}
/// Return the progress in this DataBuffer
fn progress(&self) -> SequencerProgress {
let progress = SequencerProgress::new();
fn progress(&self) -> ShardProgress {
let progress = ShardProgress::new();
let progress = if let Some(buffer) = &self.buffer {
progress.combine(buffer.progress())
@ -1450,8 +1450,8 @@ pub struct BufferBatch {
impl BufferBatch {
/// Return the progress in this DataBuffer
fn progress(&self) -> SequencerProgress {
SequencerProgress::new()
fn progress(&self) -> ShardProgress {
ShardProgress::new()
.with_buffered(self.min_sequence_number)
.with_buffered(self.max_sequence_number)
}
@ -1497,8 +1497,8 @@ impl SnapshotBatch {
}
/// Return progress in this data
fn progress(&self) -> SequencerProgress {
SequencerProgress::new()
fn progress(&self) -> ShardProgress {
ShardProgress::new()
.with_buffered(self.min_sequencer_number)
.with_buffered(self.max_sequencer_number)
}
@ -1508,8 +1508,8 @@ impl SnapshotBatch {
/// a parquet file for given set of SnapshotBatches
#[derive(Debug, PartialEq, Clone)]
pub struct PersistingBatch {
/// Sequencer id of the data
pub(crate) sequencer_id: SequencerId,
/// Shard id of the data
pub(crate) shard_id: ShardId,
/// Table id of the data
pub(crate) table_id: TableId,
@ -1540,7 +1540,7 @@ pub struct QueryableBatch {
/// Status of a partition that has unpersisted data.
///
/// Note that this structure is specific to a partition (which itself is bound to a table and
/// sequencer)!
/// shard)!
#[derive(Debug, Clone, PartialEq, Eq)]
#[allow(missing_copy_implementations)]
pub struct PartitionStatus {
@ -1806,17 +1806,17 @@ mod tests {
.create("foo", "inf", kafka_topic.id, query_pool.id)
.await
.unwrap();
let sequencer1 = repos
.sequencers()
let shard1 = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
let mut sequencers = BTreeMap::new();
let mut shards = BTreeMap::new();
let kafka_partition = KafkaPartition::new(0);
sequencers.insert(
sequencer1.id,
SequencerData::new(kafka_partition, Arc::clone(&metrics)),
shards.insert(
shard1.id,
ShardData::new(kafka_partition, Arc::clone(&metrics)),
);
let object_store: Arc<DynObjectStore> = Arc::new(InMemory::new());
@ -1824,7 +1824,7 @@ mod tests {
let data = Arc::new(IngesterData::new(
Arc::clone(&object_store),
Arc::clone(&catalog),
sequencers,
shards,
Arc::new(Executor::new(1)),
BackoffConfig::default(),
Arc::clone(&metrics),
@ -1866,7 +1866,7 @@ mod tests {
);
let should_pause = data
.buffer_operation(
sequencer1.id,
shard1.id,
DmlOperation::Write(w1.clone()),
&manager.handle(),
)
@ -1874,7 +1874,7 @@ mod tests {
.unwrap();
assert!(!should_pause);
let should_pause = data
.buffer_operation(sequencer1.id, DmlOperation::Write(w1), &manager.handle())
.buffer_operation(shard1.id, DmlOperation::Write(w1), &manager.handle())
.await
.unwrap();
assert!(should_pause);
@ -1893,24 +1893,24 @@ mod tests {
.create("foo", "inf", kafka_topic.id, query_pool.id)
.await
.unwrap();
let sequencer1 = repos
.sequencers()
let shard1 = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
let sequencer2 = repos
.sequencers()
let shard2 = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
let mut sequencers = BTreeMap::new();
sequencers.insert(
sequencer1.id,
SequencerData::new(sequencer1.kafka_partition, Arc::clone(&metrics)),
let mut shards = BTreeMap::new();
shards.insert(
shard1.id,
ShardData::new(shard1.kafka_partition, Arc::clone(&metrics)),
);
sequencers.insert(
sequencer2.id,
SequencerData::new(sequencer2.kafka_partition, Arc::clone(&metrics)),
shards.insert(
shard2.id,
ShardData::new(shard2.kafka_partition, Arc::clone(&metrics)),
);
let object_store: Arc<DynObjectStore> = Arc::new(InMemory::new());
@ -1918,7 +1918,7 @@ mod tests {
let data = Arc::new(IngesterData::new(
Arc::clone(&object_store),
Arc::clone(&catalog),
sequencers,
shards,
Arc::new(Executor::new(1)),
BackoffConfig::default(),
Arc::clone(&metrics),
@ -1980,22 +1980,22 @@ mod tests {
Arc::new(SystemProvider::new()),
);
data.buffer_operation(sequencer1.id, DmlOperation::Write(w1), &manager.handle())
data.buffer_operation(shard1.id, DmlOperation::Write(w1), &manager.handle())
.await
.unwrap();
data.buffer_operation(sequencer2.id, DmlOperation::Write(w2), &manager.handle())
data.buffer_operation(shard2.id, DmlOperation::Write(w2), &manager.handle())
.await
.unwrap();
data.buffer_operation(sequencer1.id, DmlOperation::Write(w3), &manager.handle())
data.buffer_operation(shard1.id, DmlOperation::Write(w3), &manager.handle())
.await
.unwrap();
let expected_progress = SequencerProgress::new()
let expected_progress = ShardProgress::new()
.with_buffered(SequenceNumber::new(1))
.with_buffered(SequenceNumber::new(2));
assert_progress(&data, kafka_partition, expected_progress).await;
let sd = data.sequencers.get(&sequencer1.id).unwrap();
let sd = data.shards.get(&shard1.id).unwrap();
let n = sd.namespace("foo").unwrap();
let partition_id;
let table_id;
@ -2036,7 +2036,7 @@ mod tests {
// verify it put the record in the catalog
let parquet_files = repos
.parquet_files()
.list_by_sequencer_greater_than(sequencer1.id, SequenceNumber::new(0))
.list_by_shard_greater_than(shard1.id, SequenceNumber::new(0))
.await
.unwrap();
assert_eq!(parquet_files.len(), 1);
@ -2046,7 +2046,7 @@ mod tests {
assert_eq!(pf.min_time, Timestamp::new(10));
assert_eq!(pf.max_time, Timestamp::new(30));
assert_eq!(pf.max_sequence_number, SequenceNumber::new(2));
assert_eq!(pf.sequencer_id, sequencer1.id);
assert_eq!(pf.shard_id, shard1.id);
assert!(pf.to_delete.is_none());
// This value should be recorded in the metrics asserted next;
@ -2075,8 +2075,8 @@ mod tests {
let observation = persisted_file_size_bytes
.get_observer(&Attributes::from([(
"sequencer_id",
format!("{}", sequencer1.id).into(),
"shard_id",
format!("{}", shard1.id).into(),
)]))
.unwrap()
.fetch();
@ -2108,7 +2108,7 @@ mod tests {
);
// check progresses after persist
let expected_progress = SequencerProgress::new()
let expected_progress = ShardProgress::new()
.with_buffered(SequenceNumber::new(1))
.with_persisted(SequenceNumber::new(2));
assert_progress(&data, kafka_partition, expected_progress).await;
@ -2128,24 +2128,24 @@ mod tests {
.create("foo", "inf", kafka_topic.id, query_pool.id)
.await
.unwrap();
let sequencer1 = repos
.sequencers()
let shard1 = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
let sequencer2 = repos
.sequencers()
let shard2 = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
let mut sequencers = BTreeMap::new();
sequencers.insert(
sequencer1.id,
SequencerData::new(sequencer1.kafka_partition, Arc::clone(&metrics)),
let mut shards = BTreeMap::new();
shards.insert(
shard1.id,
ShardData::new(shard1.kafka_partition, Arc::clone(&metrics)),
);
sequencers.insert(
sequencer2.id,
SequencerData::new(sequencer2.kafka_partition, Arc::clone(&metrics)),
shards.insert(
shard2.id,
ShardData::new(shard2.kafka_partition, Arc::clone(&metrics)),
);
let object_store: Arc<DynObjectStore> = Arc::new(InMemory::new());
@ -2153,7 +2153,7 @@ mod tests {
let data = Arc::new(IngesterData::new(
Arc::clone(&object_store),
Arc::clone(&catalog),
sequencers,
shards,
Arc::new(Executor::new(1)),
BackoffConfig::default(),
Arc::clone(&metrics),
@ -2206,15 +2206,15 @@ mod tests {
);
// buffer operation 1, expect progress buffered sequence number should be 1
data.buffer_operation(sequencer1.id, DmlOperation::Write(w1), &manager.handle())
data.buffer_operation(shard1.id, DmlOperation::Write(w1), &manager.handle())
.await
.unwrap();
// Get the namespace
let sd = data.sequencers.get(&sequencer1.id).unwrap();
let sd = data.shards.get(&shard1.id).unwrap();
let n = sd.namespace("foo").unwrap();
let expected_progress = SequencerProgress::new().with_buffered(SequenceNumber::new(1));
let expected_progress = ShardProgress::new().with_buffered(SequenceNumber::new(1));
assert_progress(&data, kafka_partition, expected_progress).await;
// configure the the namespace to wait after each insert.
@ -2224,7 +2224,7 @@ mod tests {
let captured_data = Arc::clone(&data);
let task = tokio::task::spawn(async move {
captured_data
.buffer_operation(sequencer1.id, DmlOperation::Write(w2), &manager.handle())
.buffer_operation(shard1.id, DmlOperation::Write(w2), &manager.handle())
.await
.unwrap();
});
@ -2233,7 +2233,7 @@ mod tests {
// Check that while the write is only partially complete, the
// buffered sequence number hasn't increased
let expected_progress = SequencerProgress::new()
let expected_progress = ShardProgress::new()
// sequence 2 hasn't been buffered yet
.with_buffered(SequenceNumber::new(1));
assert_progress(&data, kafka_partition, expected_progress).await;
@ -2243,7 +2243,7 @@ mod tests {
task.await.expect("task completed unsuccessfully");
// check progresses after the write completes
let expected_progress = SequencerProgress::new()
let expected_progress = ShardProgress::new()
.with_buffered(SequenceNumber::new(1))
.with_buffered(SequenceNumber::new(2));
assert_progress(&data, kafka_partition, expected_progress).await;
@ -2290,7 +2290,7 @@ mod tests {
let ts = create_tombstone(
1, // tombstone id
t_id, // table id
s_id, // sequencer id
s_id, // shard id
3, // delete's seq_number
0, // min time of data to get deleted
20, // max time of data to get deleted
@ -2352,7 +2352,7 @@ mod tests {
let ts = create_tombstone(
2, // tombstone id
t_id, // table id
s_id, // sequencer id
s_id, // shard id
6, // delete's seq_number
10, // min time of data to get deleted
50, // max time of data to get deleted
@ -2385,7 +2385,7 @@ mod tests {
// Persisting
let p_batch = p
.snapshot_to_persisting_batch(
SequencerId::new(s_id),
ShardId::new(s_id),
TableId::new(t_id),
PartitionId::new(p_id),
table_name,
@ -2404,7 +2404,7 @@ mod tests {
let ts = create_tombstone(
3, // tombstone id
t_id, // table id
s_id, // sequencer id
s_id, // shard id
7, // delete's seq_number
10, // min time of data to get deleted
50, // max time of data to get deleted
@ -2473,7 +2473,7 @@ mod tests {
let ts = create_tombstone(
4, // tombstone id
t_id, // table id
s_id, // sequencer id
s_id, // shard id
9, // delete's seq_number
10, // min time of data to get deleted
50, // max time of data to get deleted
@ -2517,8 +2517,8 @@ mod tests {
.create("foo", "inf", kafka_topic.id, query_pool.id)
.await
.unwrap();
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
@ -2563,17 +2563,17 @@ mod tests {
.unwrap();
let partition = repos
.partitions()
.create_or_get("1970-01-01".into(), sequencer.id, table.id)
.create_or_get("1970-01-01".into(), shard.id, table.id)
.await
.unwrap();
let partition2 = repos
.partitions()
.create_or_get("1970-01-02".into(), sequencer.id, table.id)
.create_or_get("1970-01-02".into(), shard.id, table.id)
.await
.unwrap();
let parquet_file_params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: table.id,
partition_id: partition.id,
@ -2622,7 +2622,7 @@ mod tests {
let should_pause = data
.buffer_operation(
DmlOperation::Write(w1),
sequencer.id,
shard.id,
catalog.as_ref(),
&manager.handle(),
&exec,
@ -2644,7 +2644,7 @@ mod tests {
// w2 should be in the buffer
data.buffer_operation(
DmlOperation::Write(w2),
sequencer.id,
shard.id,
catalog.as_ref(),
&manager.handle(),
&exec,
@ -2678,17 +2678,17 @@ mod tests {
.create("foo", "inf", kafka_topic.id, query_pool.id)
.await
.unwrap();
let sequencer1 = repos
.sequencers()
let shard1 = repos
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
let mut sequencers = BTreeMap::new();
let mut shards = BTreeMap::new();
let kafka_partition = KafkaPartition::new(0);
sequencers.insert(
sequencer1.id,
SequencerData::new(kafka_partition, Arc::clone(&metrics)),
shards.insert(
shard1.id,
ShardData::new(kafka_partition, Arc::clone(&metrics)),
);
let object_store: Arc<DynObjectStore> = Arc::new(InMemory::new());
@ -2696,7 +2696,7 @@ mod tests {
let data = Arc::new(IngesterData::new(
Arc::clone(&object_store),
Arc::clone(&catalog),
sequencers,
shards,
Arc::new(Executor::new(1)),
BackoffConfig::default(),
Arc::clone(&metrics),
@ -2737,7 +2737,7 @@ mod tests {
Arc::new(SystemProvider::new()),
);
data.buffer_operation(
sequencer1.id,
shard1.id,
DmlOperation::Write(w1.clone()),
&manager.handle(),
)
@ -2745,7 +2745,7 @@ mod tests {
.unwrap();
assert_eq!(
data.sequencer(sequencer1.id)
data.shard(shard1.id)
.unwrap()
.namespace(&namespace.name)
.unwrap()
@ -2772,12 +2772,12 @@ mod tests {
1337,
),
);
data.buffer_operation(sequencer1.id, DmlOperation::Delete(d1), &manager.handle())
data.buffer_operation(shard1.id, DmlOperation::Delete(d1), &manager.handle())
.await
.unwrap();
assert_eq!(
data.sequencer(sequencer1.id)
data.shard(shard1.id)
.unwrap()
.namespace(&namespace.name)
.unwrap()
@ -2794,7 +2794,7 @@ mod tests {
async fn assert_progress(
data: &IngesterData,
kafka_partition: KafkaPartition,
expected_progress: SequencerProgress,
expected_progress: ShardProgress,
) {
let progresses = data.progresses(vec![kafka_partition]).await;
let expected_progresses = [(kafka_partition, expected_progress)]

View File

@ -1,7 +1,7 @@
//! Ingest handler
use crate::{
data::{IngesterData, IngesterQueryResponse, SequencerData},
data::{IngesterData, IngesterQueryResponse, ShardData},
lifecycle::{run_lifecycle_manager, LifecycleConfig, LifecycleManager},
poison::PoisonCabinet,
querier_handler::prepare_data_to_querier,
@ -12,7 +12,7 @@ use crate::{
};
use async_trait::async_trait;
use backoff::BackoffConfig;
use data_types::{KafkaPartition, KafkaTopic, Sequencer};
use data_types::{KafkaPartition, KafkaTopic, Shard};
use futures::{
future::{BoxFuture, Shared},
stream::FuturesUnordered,
@ -33,7 +33,7 @@ use tokio::{
};
use tokio_util::sync::CancellationToken;
use write_buffer::core::WriteBufferReading;
use write_summary::SequencerProgress;
use write_summary::ShardProgress;
#[derive(Debug, Snafu)]
#[allow(missing_copy_implementations, missing_docs)]
@ -70,7 +70,7 @@ pub trait IngestHandler: Send + Sync {
async fn progresses(
&self,
sequencers: Vec<KafkaPartition>,
) -> BTreeMap<KafkaPartition, SequencerProgress>;
) -> BTreeMap<KafkaPartition, ShardProgress>;
/// Wait until the handler finished to shutdown.
///
@ -134,7 +134,7 @@ impl IngestHandlerImpl {
pub async fn new(
lifecycle_config: LifecycleConfig,
topic: KafkaTopic,
sequencer_states: BTreeMap<KafkaPartition, Sequencer>,
sequencer_states: BTreeMap<KafkaPartition, Shard>, // XXXJPG
catalog: Arc<dyn Catalog>,
object_store: Arc<DynObjectStore>,
write_buffer: Arc<dyn WriteBufferReading>,
@ -144,17 +144,17 @@ impl IngestHandlerImpl {
max_requests: usize,
) -> Result<Self> {
// build the initial ingester data state
let mut sequencers = BTreeMap::new();
let mut shards = BTreeMap::new();
for s in sequencer_states.values() {
sequencers.insert(
shards.insert(
s.id,
SequencerData::new(s.kafka_partition, Arc::clone(&metric_registry)),
ShardData::new(s.kafka_partition, Arc::clone(&metric_registry)),
);
}
let data = Arc::new(IngesterData::new(
object_store,
catalog,
sequencers,
shards,
exec,
BackoffConfig::default(),
Arc::clone(&metric_registry),
@ -186,7 +186,7 @@ impl IngestHandlerImpl {
let mut join_handles = Vec::with_capacity(sequencer_states.len() + 1);
join_handles.push(("lifecycle manager".to_owned(), shared_handle(handle)));
for (kafka_partition, sequencer) in sequencer_states {
for (kafka_partition, shard) in sequencer_states {
let metric_registry = Arc::clone(&metric_registry);
// Acquire a write buffer stream and seek it to the last
@ -197,18 +197,18 @@ impl IngestHandlerImpl {
.context(WriteBufferSnafu)?;
info!(
kafka_partition = kafka_partition.get(),
min_unpersisted_sequence_number = sequencer.min_unpersisted_sequence_number.get(),
min_unpersisted_sequence_number = shard.min_unpersisted_sequence_number.get(),
"Seek stream",
);
op_stream
.seek(sequencer.min_unpersisted_sequence_number)
.seek(shard.min_unpersisted_sequence_number)
.await
.context(WriteBufferSnafu)?;
// Initialise the DmlSink stack.
let watermark_fetcher = PeriodicWatermarkFetcher::new(
Arc::clone(&write_buffer),
sequencer.kafka_partition,
shard.kafka_partition,
Duration::from_secs(10),
&*metric_registry,
);
@ -216,14 +216,14 @@ impl IngestHandlerImpl {
let sink = IngestSinkAdaptor::new(
Arc::clone(&ingester_data),
lifecycle_handle.clone(),
sequencer.id,
shard.id,
);
// Emit metrics when ops flow through the sink
let sink = SinkInstrumentation::new(
sink,
watermark_fetcher,
kafka_topic_name.clone(),
sequencer.kafka_partition,
shard.kafka_partition,
&*metric_registry,
);
@ -236,11 +236,11 @@ impl IngestHandlerImpl {
async move {
let handler = SequencedStreamHandler::new(
op_stream,
sequencer.min_unpersisted_sequence_number,
shard.min_unpersisted_sequence_number,
sink,
lifecycle_handle,
kafka_topic_name,
sequencer.kafka_partition,
shard.kafka_partition,
&*metric_registry,
skip_to_oldest_available,
);
@ -365,7 +365,7 @@ impl IngestHandler for IngestHandlerImpl {
async fn progresses(
&self,
partitions: Vec<KafkaPartition>,
) -> BTreeMap<KafkaPartition, SequencerProgress> {
) -> BTreeMap<KafkaPartition, ShardProgress> {
self.data.progresses(partitions).await
}
}
@ -473,7 +473,7 @@ mod tests {
loop {
let mut has_measurement = false;
if let Some(data) = ingester.ingester.data.sequencer(ingester.sequencer.id) {
if let Some(data) = ingester.ingester.data.shard(ingester.shard.id) {
if let Some(data) = data.namespace(&ingester.namespace.name) {
// verify there's data in the buffer
if let Some((b, _)) = data.snapshot("a", &"1970-01-01".into()).await {
@ -486,17 +486,17 @@ mod tests {
}
}
// and ensure that the sequencer state was actually updated
let seq = ingester
// and ensure that the shard state was actually updated
let shard = ingester
.catalog
.repositories()
.await
.sequencers()
.shards()
.create_or_get(&ingester.kafka_topic, ingester.kafka_partition)
.await
.unwrap();
if has_measurement && seq.min_unpersisted_sequence_number == SequenceNumber::new(9)
if has_measurement && shard.min_unpersisted_sequence_number == SequenceNumber::new(9)
{
break;
}
@ -625,7 +625,7 @@ mod tests {
write_operations: Vec<DmlWrite>,
min_unpersisted_sequence_number: i64,
skip_to_oldest_available: bool,
) -> (IngestHandlerImpl, Sequencer, Namespace) {
) -> (IngestHandlerImpl, Shard, Namespace) {
let metrics: Arc<metric::Registry> = Default::default();
let catalog: Arc<dyn Catalog> = Arc::new(MemCatalog::new(Arc::clone(&metrics)));
@ -638,25 +638,25 @@ mod tests {
.create("foo", "inf", kafka_topic.id, query_pool.id)
.await
.unwrap();
let mut sequencer = txn
.sequencers()
let mut shard = txn
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
// update the min unpersisted
sequencer.min_unpersisted_sequence_number =
shard.min_unpersisted_sequence_number =
SequenceNumber::new(min_unpersisted_sequence_number);
// this probably isn't necessary, but just in case something changes later
txn.sequencers()
txn.shards()
.update_min_unpersisted_sequence_number(
sequencer.id,
shard.id,
SequenceNumber::new(min_unpersisted_sequence_number),
)
.await
.unwrap();
let mut sequencer_states = BTreeMap::new();
sequencer_states.insert(kafka_partition, sequencer);
sequencer_states.insert(kafka_partition, shard);
let write_buffer_state =
MockBufferSharedState::empty_with_n_sequencers(NonZeroU32::try_from(1).unwrap());
@ -697,12 +697,12 @@ mod tests {
.await
.unwrap();
(ingester, sequencer, namespace)
(ingester, shard, namespace)
}
async fn verify_ingester_buffer_has_data(
ingester: IngestHandlerImpl,
sequencer: Sequencer,
shard: Shard,
namespace: Namespace,
custom_batch_verification: impl Fn(&SnapshotBatch) + Send,
) {
@ -712,7 +712,7 @@ mod tests {
loop {
let mut has_measurement = false;
if let Some(data) = ingester.data.sequencer(sequencer.id) {
if let Some(data) = ingester.data.shard(shard.id) {
if let Some(data) = data.namespace(&namespace.name) {
// verify there's data in the buffer
if let Some((b, _)) = data.snapshot("cpu", &"1970-01-01".into()).await {
@ -911,7 +911,7 @@ mod tests {
struct TestIngester {
catalog: Arc<dyn Catalog>,
sequencer: Sequencer,
shard: Shard,
namespace: Namespace,
kafka_topic: KafkaTopic,
kafka_partition: KafkaPartition,
@ -936,7 +936,7 @@ mod tests {
.await
.unwrap();
let sequencer = txn
.sequencers()
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();
@ -975,7 +975,7 @@ mod tests {
Self {
catalog,
sequencer,
shard: sequencer,
namespace,
kafka_topic,
kafka_partition,

View File

@ -1,4 +1,4 @@
//! Manages the persistence and eviction lifecycle of data in the buffer across all sequencers.
//! Manages the persistence and eviction lifecycle of data in the buffer across all shards.
//! Note that the byte counts logged by the lifecycle manager and when exactly persistence gets
//! triggered aren't required to be absolutely accurate. The byte count is just an estimate
//! anyway, this just needs to keep things moving along to keep memory use roughly under
@ -10,7 +10,7 @@ use crate::{
job::{Job, JobRegistry},
poison::{PoisonCabinet, PoisonPill},
};
use data_types::{PartitionId, SequenceNumber, SequencerId};
use data_types::{PartitionId, SequenceNumber, ShardId};
use iox_time::{Time, TimeProvider};
use metric::{Metric, U64Counter};
use observability_deps::tracing::{error, info};
@ -27,7 +27,7 @@ pub trait LifecycleHandle: Send + Sync + 'static {
fn log_write(
&self,
partition_id: PartitionId,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
bytes_written: usize,
) -> bool;
@ -37,7 +37,7 @@ pub trait LifecycleHandle: Send + Sync + 'static {
fn can_resume_ingest(&self) -> bool;
}
/// A handle for sequencer consumers to interact with the global
/// A handle for shard consumers to interact with the global
/// [`LifecycleManager`] instance.
///
/// This handle presents an API suitable for ingester tasks to query and update
@ -56,7 +56,7 @@ impl LifecycleHandle for LifecycleHandleImpl {
fn log_write(
&self,
partition_id: PartitionId,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
bytes_written: usize,
) -> bool {
@ -67,7 +67,7 @@ impl LifecycleHandle for LifecycleHandleImpl {
s.partition_stats
.entry(partition_id)
.or_insert_with(|| PartitionLifecycleStats {
sequencer_id,
shard_id,
partition_id,
first_write: now,
last_write: now,
@ -89,7 +89,7 @@ impl LifecycleHandle for LifecycleHandleImpl {
}
/// The lifecycle manager keeps track of the size and age of partitions across
/// all sequencers. It triggers persistence based on keeping total memory usage
/// all shards. It triggers persistence based on keeping total memory usage
/// around a set amount while ensuring that partitions don't get too old or
/// large before being persisted.
///
@ -193,7 +193,7 @@ impl LifecycleState {
/// A snapshot of the stats for the lifecycle manager
#[derive(Debug)]
struct LifecycleStats {
/// total number of bytes the lifecycle manager is aware of across all sequencers and
/// total number of bytes the lifecycle manager is aware of across all shards and
/// partitions. Based on the mutable batch sizes received into all partitions.
pub total_bytes: usize,
/// the stats for every partition the lifecycle manager is tracking.
@ -203,8 +203,8 @@ struct LifecycleStats {
/// The stats for a partition
#[derive(Debug, Clone, Copy)]
struct PartitionLifecycleStats {
/// The sequencer this partition is under
sequencer_id: SequencerId,
/// The shard this partition is under
shard_id: ShardId,
/// The partition identifier
partition_id: PartitionId,
/// Time that the partition received its first write. This is reset anytime
@ -299,7 +299,7 @@ impl LifecycleManager {
let sized_out = s.bytes_written > self.config.partition_size_threshold;
if sized_out {
self.persist_size_counter.inc(1);
info!(sequencer_id=%s.sequencer_id,
info!(shard_id=%s.shard_id,
partition_id=%s.partition_id,
bytes_written=s.bytes_written,
partition_size_threshold=self.config.partition_size_threshold,
@ -338,12 +338,12 @@ impl LifecycleManager {
rest = remaining;
}
// for the sequencers that are getting data persisted, keep track of what
// for the shards that are getting data persisted, keep track of what
// the highest seqeunce number was for each.
let mut sequencer_maxes = BTreeMap::new();
for s in &to_persist {
sequencer_maxes
.entry(s.sequencer_id)
.entry(s.shard_id)
.and_modify(|sn| {
if *sn < s.first_sequence_number {
*sn = s.first_sequence_number;
@ -386,29 +386,29 @@ impl LifecycleManager {
res.expect("not aborted").expect("task finished");
}
// for the sequencers that had data persisted, update their min_unpersisted_sequence_number to
// for the shards that had data persisted, update their min_unpersisted_sequence_number to
// either the minimum remaining in everything that didn't get persisted, or the highest
// number that was persisted. Marking the highest number as the state is ok because it
// just needs to represent the farthest we'd have to seek back in the write buffer. Any
// data replayed during recovery that has already been persisted will just be ignored.
//
// The calculation of the min unpersisted sequence number is:
// - If there is any unpersisted data (i.e. `rest` contains entries for that sequencer) then we take the
// - If there is any unpersisted data (i.e. `rest` contains entries for that shard) then we take the
// minimum sequence number from there. Note that there might be a gap between the data that we have just
// persisted and the unpersisted section.
// - If there is NO unpersisted data, we take the max sequence number of the part that we have just
// persisted. Note that can cannot use "max + 1" because the lifecycle handler receives writes on a
// partition level and therefore might run while data for a single sequence number is added. So the max
// sequence number that we have just persisted might have more data.
for (sequencer_id, sequence_number) in sequencer_maxes {
for (shard_id, sequence_number) in sequencer_maxes {
let min = rest
.iter()
.filter(|s| s.sequencer_id == sequencer_id)
.filter(|s| s.shard_id == shard_id)
.map(|s| s.first_sequence_number)
.min()
.unwrap_or(sequence_number);
persister
.update_min_unpersisted_sequence_number(sequencer_id, min)
.update_min_unpersisted_sequence_number(shard_id, min)
.await;
}
}
@ -478,7 +478,7 @@ mod tests {
#[derive(Default)]
struct TestPersister {
persist_called: Mutex<BTreeSet<PartitionId>>,
update_min_calls: Mutex<Vec<(SequencerId, SequenceNumber)>>,
update_min_calls: Mutex<Vec<(ShardId, SequenceNumber)>>,
}
#[async_trait]
@ -490,11 +490,11 @@ mod tests {
async fn update_min_unpersisted_sequence_number(
&self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) {
let mut u = self.update_min_calls.lock();
u.push((sequencer_id, sequence_number));
u.push((shard_id, sequence_number));
}
}
@ -504,7 +504,7 @@ mod tests {
p.contains(&partition_id)
}
fn update_min_calls(&self) -> Vec<(SequencerId, SequenceNumber)> {
fn update_min_calls(&self) -> Vec<(ShardId, SequenceNumber)> {
let u = self.update_min_calls.lock();
u.clone()
}
@ -553,11 +553,11 @@ mod tests {
async fn update_min_unpersisted_sequence_number(
&self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) {
self.inner
.update_min_unpersisted_sequence_number(sequencer_id, sequence_number)
.update_min_unpersisted_sequence_number(shard_id, sequence_number)
.await
}
}
@ -607,18 +607,18 @@ mod tests {
let TestLifecycleManger {
m, time_provider, ..
} = TestLifecycleManger::new(config);
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let h = m.handle();
// log first two writes at different times
assert!(!h.log_write(PartitionId::new(1), sequencer_id, SequenceNumber::new(1), 1));
assert!(!h.log_write(PartitionId::new(1), shard_id, SequenceNumber::new(1), 1));
time_provider.inc(Duration::from_nanos(10));
assert!(!h.log_write(PartitionId::new(1), sequencer_id, SequenceNumber::new(2), 1));
assert!(!h.log_write(PartitionId::new(1), shard_id, SequenceNumber::new(2), 1));
// log another write for different partition using a different handle
assert!(!m.handle().log_write(
PartitionId::new(2),
sequencer_id,
shard_id,
SequenceNumber::new(3),
3
));
@ -648,14 +648,14 @@ mod tests {
};
let partition_id = PartitionId::new(1);
let TestLifecycleManger { mut m, .. } = TestLifecycleManger::new(config);
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let h = m.handle();
// write more than the limit (10)
assert!(!h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 15));
assert!(!h.log_write(partition_id, shard_id, SequenceNumber::new(1), 15));
// all subsequent writes should also indicate a pause
assert!(h.log_write(partition_id, sequencer_id, SequenceNumber::new(2), 10));
assert!(h.log_write(partition_id, shard_id, SequenceNumber::new(2), 10));
assert!(!h.can_resume_ingest());
// persist the partition
@ -664,7 +664,7 @@ mod tests {
// ingest can resume
assert!(h.can_resume_ingest());
assert!(!h.log_write(partition_id, sequencer_id, SequenceNumber::new(3), 3));
assert!(!h.log_write(partition_id, shard_id, SequenceNumber::new(3), 3));
}
#[tokio::test]
@ -678,11 +678,11 @@ mod tests {
};
let partition_id = PartitionId::new(1);
let TestLifecycleManger { mut m, .. } = TestLifecycleManger::new(config);
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let h = m.handle();
// write more than the limit (20)
h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 25);
h.log_write(partition_id, shard_id, SequenceNumber::new(1), 25);
// can not resume ingest as we are overall the pause ingest limit
assert!(!h.can_resume_ingest());
@ -710,7 +710,7 @@ mod tests {
// ingest can resume
assert!(h.can_resume_ingest());
assert!(!h.log_write(partition_id, sequencer_id, SequenceNumber::new(2), 3));
assert!(!h.log_write(partition_id, shard_id, SequenceNumber::new(2), 3));
}
#[tokio::test]
@ -729,10 +729,10 @@ mod tests {
} = TestLifecycleManger::new(config);
let partition_id = PartitionId::new(1);
let persister = Arc::new(TestPersister::default());
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let h = m.handle();
h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 10);
h.log_write(partition_id, shard_id, SequenceNumber::new(1), 10);
m.maybe_persist(&persister).await;
let stats = m.stats();
@ -746,7 +746,7 @@ mod tests {
assert!(!persister.persist_called_for(partition_id));
// write in data for a new partition so we can be sure it isn't persisted, but the older one is
h.log_write(PartitionId::new(2), sequencer_id, SequenceNumber::new(2), 6);
h.log_write(PartitionId::new(2), shard_id, SequenceNumber::new(2), 6);
m.maybe_persist(&persister).await;
@ -754,7 +754,7 @@ mod tests {
assert!(!persister.persist_called_for(PartitionId::new(2)));
assert_eq!(
persister.update_min_calls(),
vec![(sequencer_id, SequenceNumber::new(2))]
vec![(shard_id, SequenceNumber::new(2))]
);
let stats = m.stats();
@ -782,10 +782,10 @@ mod tests {
} = TestLifecycleManger::new(config);
let partition_id = PartitionId::new(1);
let persister = Arc::new(TestPersister::default());
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let h = m.handle();
h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 10);
h.log_write(partition_id, shard_id, SequenceNumber::new(1), 10);
m.maybe_persist(&persister).await;
let stats = m.stats();
@ -799,8 +799,8 @@ mod tests {
assert!(!persister.persist_called_for(partition_id));
// write in data for a new partition so we can be sure it isn't persisted, but the older one is
h.log_write(PartitionId::new(2), sequencer_id, SequenceNumber::new(2), 6);
h.log_write(PartitionId::new(3), sequencer_id, SequenceNumber::new(3), 7);
h.log_write(PartitionId::new(2), shard_id, SequenceNumber::new(2), 6);
h.log_write(PartitionId::new(3), shard_id, SequenceNumber::new(3), 7);
m.maybe_persist(&persister).await;
@ -808,7 +808,7 @@ mod tests {
assert!(!persister.persist_called_for(PartitionId::new(2)));
assert_eq!(
persister.update_min_calls(),
vec![(sequencer_id, SequenceNumber::new(2))]
vec![(shard_id, SequenceNumber::new(2))]
);
let stats = m.stats();
@ -835,12 +835,12 @@ mod tests {
metric_registry,
..
} = TestLifecycleManger::new(config);
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let h = m.handle();
let partition_id = PartitionId::new(1);
let persister = Arc::new(TestPersister::default());
h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 4);
h.log_write(partition_id, shard_id, SequenceNumber::new(1), 4);
m.maybe_persist(&persister).await;
@ -850,8 +850,8 @@ mod tests {
assert!(!persister.persist_called_for(partition_id));
// introduce a new partition under the limit to verify it doesn't get taken with the other
h.log_write(PartitionId::new(2), sequencer_id, SequenceNumber::new(2), 3);
h.log_write(partition_id, sequencer_id, SequenceNumber::new(3), 5);
h.log_write(PartitionId::new(2), shard_id, SequenceNumber::new(2), 3);
h.log_write(partition_id, shard_id, SequenceNumber::new(3), 5);
m.maybe_persist(&persister).await;
@ -859,7 +859,7 @@ mod tests {
assert!(!persister.persist_called_for(PartitionId::new(2)));
assert_eq!(
persister.update_min_calls(),
vec![(sequencer_id, SequenceNumber::new(2))]
vec![(shard_id, SequenceNumber::new(2))]
);
let stats = m.stats();
@ -880,7 +880,7 @@ mod tests {
partition_age_threshold: Duration::from_millis(1000),
partition_cold_threshold: Duration::from_secs(500),
};
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let TestLifecycleManger {
mut m,
metric_registry,
@ -889,10 +889,10 @@ mod tests {
let h = m.handle();
let partition_id = PartitionId::new(1);
let persister = Arc::new(TestPersister::default());
h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 8);
h.log_write(partition_id, shard_id, SequenceNumber::new(1), 8);
h.log_write(
PartitionId::new(2),
sequencer_id,
shard_id,
SequenceNumber::new(2),
13,
);
@ -907,14 +907,14 @@ mod tests {
assert!(persister.persist_called_for(PartitionId::new(2)));
assert_eq!(
persister.update_min_calls(),
vec![(sequencer_id, SequenceNumber::new(1))]
vec![(shard_id, SequenceNumber::new(1))]
);
// add that partition back in over size
h.log_write(partition_id, sequencer_id, SequenceNumber::new(3), 20);
h.log_write(partition_id, shard_id, SequenceNumber::new(3), 20);
h.log_write(
PartitionId::new(2),
sequencer_id,
shard_id,
SequenceNumber::new(4),
21,
);
@ -930,8 +930,8 @@ mod tests {
assert_eq!(
persister.update_min_calls(),
vec![
(sequencer_id, SequenceNumber::new(1)),
(sequencer_id, SequenceNumber::new(4))
(shard_id, SequenceNumber::new(1)),
(shard_id, SequenceNumber::new(4))
]
);
@ -952,7 +952,7 @@ mod tests {
partition_age_threshold: Duration::from_millis(1000),
partition_cold_threshold: Duration::from_secs(500),
};
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
let TestLifecycleManger {
mut m,
time_provider,
@ -960,11 +960,11 @@ mod tests {
} = TestLifecycleManger::new(config);
let h = m.handle();
let persister = Arc::new(TestPersister::default());
h.log_write(PartitionId::new(1), sequencer_id, SequenceNumber::new(1), 4);
h.log_write(PartitionId::new(1), shard_id, SequenceNumber::new(1), 4);
time_provider.inc(Duration::from_nanos(1));
h.log_write(PartitionId::new(2), sequencer_id, SequenceNumber::new(2), 6);
h.log_write(PartitionId::new(2), shard_id, SequenceNumber::new(2), 6);
time_provider.inc(Duration::from_nanos(1));
h.log_write(PartitionId::new(3), sequencer_id, SequenceNumber::new(3), 3);
h.log_write(PartitionId::new(3), shard_id, SequenceNumber::new(3), 3);
m.maybe_persist(&persister).await;
@ -977,7 +977,7 @@ mod tests {
assert!(persister.persist_called_for(PartitionId::new(1)));
assert_eq!(
persister.update_min_calls(),
vec![(sequencer_id, SequenceNumber::new(3))]
vec![(shard_id, SequenceNumber::new(3))]
);
let memory_counter = get_counter(&metric_registry, "memory");
@ -1001,9 +1001,9 @@ mod tests {
let h = m.handle();
let partition_id = PartitionId::new(1);
let persister = Arc::new(TestPersister::default());
let sequencer_id = SequencerId::new(1);
let shard_id = ShardId::new(1);
h.log_write(partition_id, sequencer_id, SequenceNumber::new(1), 10);
h.log_write(partition_id, shard_id, SequenceNumber::new(1), 10);
m.maybe_persist(&persister).await;
let stats = m.stats();
@ -1017,7 +1017,7 @@ mod tests {
assert!(!persister.persist_called_for(partition_id));
// write in data for a new partition so we can be sure it isn't persisted, but the older one is
h.log_write(PartitionId::new(2), sequencer_id, SequenceNumber::new(2), 6);
h.log_write(PartitionId::new(2), shard_id, SequenceNumber::new(2), 6);
m.maybe_persist(&persister).await;
@ -1025,7 +1025,7 @@ mod tests {
assert!(!persister.persist_called_for(PartitionId::new(2)));
assert_eq!(
persister.update_min_calls(),
vec![(sequencer_id, SequenceNumber::new(2))]
vec![(shard_id, SequenceNumber::new(2))]
);
let stats = m.stats();

View File

@ -87,9 +87,9 @@ pub async fn prepare_data_to_querier(
debug!(?request, "prepare_data_to_querier");
let mut unpersisted_partitions = vec![];
let mut found_namespace = false;
for (sequencer_id, sequencer_data) in ingest_data.sequencers() {
debug!(sequencer_id=%sequencer_id.get());
let namespace_data = match sequencer_data.namespace(&request.namespace) {
for (shard_id, shard_data) in ingest_data.shards() {
debug!(shard_id=%shard_id.get());
let namespace_data = match shard_data.namespace(&request.namespace) {
Some(namespace_data) => {
debug!(namespace=%request.namespace, "found namespace");
found_namespace = true;

View File

@ -3,7 +3,7 @@
use super::DmlSink;
use crate::{data::IngesterData, lifecycle::LifecycleHandleImpl};
use async_trait::async_trait;
use data_types::SequencerId;
use data_types::ShardId;
use dml::DmlOperation;
use std::sync::Arc;
@ -12,7 +12,7 @@ use std::sync::Arc;
pub struct IngestSinkAdaptor {
ingest_data: Arc<IngesterData>,
lifecycle_handle: LifecycleHandleImpl,
sequencer_id: SequencerId,
shard_id: ShardId,
}
impl IngestSinkAdaptor {
@ -21,12 +21,12 @@ impl IngestSinkAdaptor {
pub fn new(
ingest_data: Arc<IngesterData>,
lifecycle_handle: LifecycleHandleImpl,
sequencer_id: SequencerId,
shard_id: ShardId,
) -> Self {
Self {
ingest_data,
lifecycle_handle,
sequencer_id,
shard_id,
}
}
}
@ -35,7 +35,7 @@ impl IngestSinkAdaptor {
impl DmlSink for IngestSinkAdaptor {
async fn apply(&self, op: DmlOperation) -> Result<bool, crate::data::Error> {
self.ingest_data
.buffer_operation(self.sequencer_id, op, &self.lifecycle_handle)
.buffer_operation(self.shard_id, op, &self.lifecycle_handle)
.await
}
}

View File

@ -3,7 +3,7 @@
#![allow(missing_docs)]
use crate::data::{
IngesterData, NamespaceData, PartitionData, PersistingBatch, QueryableBatch, SequencerData,
IngesterData, NamespaceData, PartitionData, PersistingBatch, QueryableBatch, ShardData,
SnapshotBatch, TableData,
};
use arrow::record_batch::RecordBatch;
@ -11,7 +11,7 @@ use arrow_util::assert_batches_eq;
use bitflags::bitflags;
use data_types::{
CompactionLevel, KafkaPartition, NamespaceId, PartitionId, PartitionKey, SequenceNumber,
SequencerId, TableId, Timestamp, Tombstone, TombstoneId,
ShardId, TableId, Timestamp, Tombstone, TombstoneId,
};
use iox_catalog::{interface::Catalog, mem::MemCatalog};
use iox_query::test::{raw_data, TestChunk};
@ -52,7 +52,7 @@ pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tom
let namespace_name = "test_namespace";
let partition_key = "test_partition_key";
let table_name = "test_table";
let seq_id = 1;
let shard_id = 1;
let seq_num_start: i64 = 1;
let seq_num_end: i64 = seq_num_start + 1; // 2 batches
let namespace_id = 1;
@ -61,7 +61,7 @@ pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tom
// make the persisting batch
let persisting_batch = make_persisting_batch(
seq_id,
shard_id,
seq_num_start,
table_id,
table_name,
@ -76,7 +76,7 @@ pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tom
let meta = make_meta(
uuid,
time_provider.now(),
seq_id,
shard_id,
namespace_id,
namespace_name,
table_id,
@ -95,7 +95,7 @@ pub async fn make_persisting_batch_with_meta() -> (Arc<PersistingBatch>, Vec<Tom
pub fn create_tombstone(
id: i64,
table_id: i64,
seq_id: i64,
shard_id: i64,
seq_num: i64,
min_time: i64,
max_time: i64,
@ -104,7 +104,7 @@ pub fn create_tombstone(
Tombstone {
id: TombstoneId::new(id),
table_id: TableId::new(table_id),
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
sequence_number: SequenceNumber::new(seq_num),
min_time: Timestamp::new(min_time),
max_time: Timestamp::new(max_time),
@ -116,7 +116,7 @@ pub fn create_tombstone(
pub fn make_meta(
object_store_id: Uuid,
creation_timestamp: Time,
sequencer_id: i64,
shard_id: i64,
namespace_id: i64,
namespace_name: &str,
table_id: i64,
@ -130,7 +130,7 @@ pub fn make_meta(
IoxMetadata {
object_store_id,
creation_timestamp,
sequencer_id: SequencerId::new(sequencer_id),
shard_id: ShardId::new(shard_id),
namespace_id: NamespaceId::new(namespace_id),
namespace_name: Arc::from(namespace_name),
table_id: TableId::new(table_id),
@ -145,7 +145,7 @@ pub fn make_meta(
#[allow(clippy::too_many_arguments)]
pub fn make_persisting_batch(
seq_id: i64,
shard_id: i64,
seq_num_start: i64,
table_id: i64,
table_name: &str,
@ -158,7 +158,7 @@ pub fn make_persisting_batch(
make_queryable_batch_with_deletes(table_name, seq_num_start, batches, tombstones);
Arc::new(PersistingBatch {
sequencer_id: SequencerId::new(seq_id),
shard_id: ShardId::new(shard_id),
table_id: TableId::new(table_id),
partition_id: PartitionId::new(partition_id),
object_store_id,
@ -659,13 +659,13 @@ pub fn make_ingester_data(two_partitions: bool, loc: DataLocation) -> IngesterDa
let object_store = Arc::new(InMemory::new());
let exec = Arc::new(iox_query::exec::Executor::new(1));
// Make data for one sequencer/shard and two tables
let seq_id = SequencerId::new(1);
// Make data for one shard and two tables
let shard_id = ShardId::new(1);
let empty_table_id = TableId::new(1);
let data_table_id = TableId::new(2);
// Make partitions per requested
let partitions = make_partitions(two_partitions, loc, seq_id, data_table_id, TEST_TABLE);
let partitions = make_partitions(two_partitions, loc, shard_id, data_table_id, TEST_TABLE);
// Two tables: one empty and one with data of one or two partitions
let mut tables = BTreeMap::new();
@ -688,17 +688,17 @@ pub fn make_ingester_data(two_partitions: bool, loc: DataLocation) -> IngesterDa
namespaces.insert(TEST_NAMESPACE_EMPTY.to_string(), empty_ns);
namespaces.insert(TEST_NAMESPACE.to_string(), data_ns);
// One sequencer/shard that contains 2 namespaces
// One shard that contains 2 namespaces
let kafka_partition = KafkaPartition::new(0);
let seq_data = SequencerData::new_for_test(kafka_partition, namespaces);
let mut sequencers = BTreeMap::new();
sequencers.insert(seq_id, seq_data);
let shard_data = ShardData::new_for_test(kafka_partition, namespaces);
let mut shards = BTreeMap::new();
shards.insert(shard_id, shard_data);
// Ingester data that includes one sequencer/shard
// Ingester data that includes one shard
IngesterData::new(
object_store,
catalog,
sequencers,
shards,
exec,
backoff::BackoffConfig::default(),
metrics,
@ -712,13 +712,13 @@ pub async fn make_ingester_data_with_tombstones(loc: DataLocation) -> IngesterDa
let object_store = Arc::new(InMemory::new());
let exec = Arc::new(iox_query::exec::Executor::new(1));
// Make data for one sequencer/shard and two tables
let seq_id = SequencerId::new(1);
// Make data for one shard and two tables
let shard_id = ShardId::new(1);
let data_table_id = TableId::new(2);
// Make partitions per requested
let partitions =
make_one_partition_with_tombstones(&exec, loc, seq_id, data_table_id, TEST_TABLE).await;
make_one_partition_with_tombstones(&exec, loc, shard_id, data_table_id, TEST_TABLE).await;
// Two tables: one empty and one with data of one or two partitions
let mut tables = BTreeMap::new();
@ -733,17 +733,17 @@ pub async fn make_ingester_data_with_tombstones(loc: DataLocation) -> IngesterDa
let data_ns = Arc::new(NamespaceData::new_for_test(NamespaceId::new(2), tables));
namespaces.insert(TEST_NAMESPACE.to_string(), data_ns);
// One sequencer/shard that contains 1 namespace
// One shard that contains 1 namespace
let kafka_partition = KafkaPartition::new(0);
let seq_data = SequencerData::new_for_test(kafka_partition, namespaces);
let mut sequencers = BTreeMap::new();
sequencers.insert(seq_id, seq_data);
let shard_data = ShardData::new_for_test(kafka_partition, namespaces);
let mut shards = BTreeMap::new();
shards.insert(shard_id, shard_data);
// Ingester data that includes one sequencer/shard
// Ingester data that includes one shard
IngesterData::new(
object_store,
catalog,
sequencers,
shards,
exec,
backoff::BackoffConfig::default(),
metrics,
@ -754,7 +754,7 @@ pub async fn make_ingester_data_with_tombstones(loc: DataLocation) -> IngesterDa
pub(crate) fn make_partitions(
two_partitions: bool,
loc: DataLocation,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
table_name: &str,
) -> BTreeMap<PartitionKey, PartitionData> {
@ -781,7 +781,7 @@ pub(crate) fn make_partitions(
// Build the first partition
let partition_id = PartitionId::new(1);
let (mut p1, seq_num) =
make_first_partition_data(partition_id, loc, sequencer_id, table_id, table_name);
make_first_partition_data(partition_id, loc, shard_id, table_id, table_name);
// ------------------------------------------
// Build the second partition if asked
@ -827,7 +827,7 @@ pub(crate) fn make_partitions(
pub(crate) async fn make_one_partition_with_tombstones(
exec: &iox_query::exec::Executor,
loc: DataLocation,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
table_name: &str,
) -> BTreeMap<PartitionKey, PartitionData> {
@ -850,7 +850,7 @@ pub(crate) async fn make_one_partition_with_tombstones(
let partition_id = PartitionId::new(1);
let (mut p1, seq_num) =
make_first_partition_data(partition_id, loc, sequencer_id, table_id, table_name);
make_first_partition_data(partition_id, loc, shard_id, table_id, table_name);
// Add tombtones
// Depending on where the existing data is, they (buffer & snapshot) will be either moved to a new sanpshot after
@ -862,7 +862,7 @@ pub(crate) async fn make_one_partition_with_tombstones(
let ts = create_tombstone(
2, // tombstone id
table_id.get(), // table id
sequencer_id.get(), // sequencer id
shard_id.get(), // sequencer id
seq_num, // delete's seq_number
10, // min time of data to get deleted
50, // max time of data to get deleted
@ -892,7 +892,7 @@ pub(crate) async fn make_one_partition_with_tombstones(
fn make_first_partition_data(
partition_id: PartitionId,
loc: DataLocation,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
table_name: &str,
) -> (PartitionData, SequenceNumber) {
@ -932,7 +932,7 @@ fn make_first_partition_data(
if loc.contains(DataLocation::PERSISTING) {
// Move group 1 data to persisting
p1.snapshot_to_persisting_batch(sequencer_id, table_id, partition_id, table_name);
p1.snapshot_to_persisting_batch(shard_id, table_id, partition_id, table_name);
} else if loc.contains(DataLocation::SNAPSHOT) {
// move group 1 data to snapshot
p1.snapshot().unwrap();

File diff suppressed because it is too large Load Diff

View File

@ -13,7 +13,7 @@
use crate::interface::{ColumnUpsertRequest, Error, RepoCollection, Result, Transaction};
use data_types::{
ColumnType, KafkaPartition, KafkaTopic, NamespaceSchema, QueryPool, Sequencer, SequencerId,
ColumnType, KafkaPartition, KafkaTopic, NamespaceSchema, QueryPool, Shard, ShardId,
TableSchema,
};
use mutable_batch::MutableBatch;
@ -195,28 +195,28 @@ where
Ok(())
}
/// Creates or gets records in the catalog for the shared kafka topic, query pool, and sequencers
/// Creates or gets records in the catalog for the shared kafka topic, query pool, and shards
/// for each of the partitions.
///
/// Used in tests and when creating an in-memory catalog.
pub async fn create_or_get_default_records(
kafka_partition_count: i32,
txn: &mut dyn Transaction,
) -> Result<(KafkaTopic, QueryPool, BTreeMap<SequencerId, Sequencer>)> {
) -> Result<(KafkaTopic, QueryPool, BTreeMap<ShardId, Shard>)> {
let kafka_topic = txn.kafka_topics().create_or_get(SHARED_KAFKA_TOPIC).await?;
let query_pool = txn.query_pools().create_or_get(SHARED_QUERY_POOL).await?;
let mut sequencers = BTreeMap::new();
let mut shards = BTreeMap::new();
// Start at 0 to match the one write buffer partition ID used in all-in-one mode
for partition in 0..kafka_partition_count {
let sequencer = txn
.sequencers()
let shard = txn
.shards()
.create_or_get(&kafka_topic, KafkaPartition::new(partition))
.await?;
sequencers.insert(sequencer.id, sequencer);
shards.insert(shard.id, shard);
}
Ok((kafka_topic, query_pool, sequencers))
Ok((kafka_topic, query_pool, shards))
}
#[cfg(test)]

View File

@ -5,7 +5,7 @@ use crate::{
interface::{
sealed::TransactionFinalize, Catalog, ColumnRepo, ColumnUpsertRequest, Error,
KafkaTopicRepo, NamespaceRepo, ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo,
QueryPoolRepo, RepoCollection, Result, SequencerRepo, TablePersistInfo, TableRepo,
QueryPoolRepo, RepoCollection, Result, ShardRepo, TablePersistInfo, TableRepo,
TombstoneRepo, Transaction,
},
metrics::MetricDecorator,
@ -15,7 +15,7 @@ use data_types::{
Column, ColumnId, ColumnType, CompactionLevel, KafkaPartition, KafkaTopic, KafkaTopicId,
Namespace, NamespaceId, ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId,
PartitionInfo, PartitionKey, PartitionParam, ProcessedTombstone, QueryPool, QueryPoolId,
SequenceNumber, Sequencer, SequencerId, Table, TableId, TablePartition, Timestamp, Tombstone,
SequenceNumber, Shard, ShardId, Table, TableId, TablePartition, Timestamp, Tombstone,
TombstoneId,
};
use iox_time::{SystemProvider, TimeProvider};
@ -62,7 +62,7 @@ struct MemCollections {
namespaces: Vec<Namespace>,
tables: Vec<Table>,
columns: Vec<Column>,
sequencers: Vec<Sequencer>,
shards: Vec<Shard>,
partitions: Vec<Partition>,
tombstones: Vec<Tombstone>,
parquet_files: Vec<ParquetFile>,
@ -208,7 +208,7 @@ impl RepoCollection for MemTxn {
self
}
fn sequencers(&mut self) -> &mut dyn SequencerRepo {
fn shards(&mut self) -> &mut dyn ShardRepo {
self
}
@ -445,7 +445,7 @@ impl TableRepo for MemTxn {
async fn get_table_persist_info(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
namespace_id: NamespaceId,
table_name: &str,
) -> Result<Option<TablePersistInfo>> {
@ -459,12 +459,12 @@ impl TableRepo for MemTxn {
let tombstone_max_sequence_number = stage
.tombstones
.iter()
.filter(|t| t.sequencer_id == sequencer_id && t.table_id == table.id)
.filter(|t| t.shard_id == shard_id && t.table_id == table.id)
.max_by_key(|t| t.sequence_number)
.map(|t| t.sequence_number);
return Ok(Some(TablePersistInfo {
sequencer_id,
shard_id,
table_id: table.id,
tombstone_max_sequence_number,
}));
@ -604,76 +604,76 @@ impl ColumnRepo for MemTxn {
}
#[async_trait]
impl SequencerRepo for MemTxn {
impl ShardRepo for MemTxn {
async fn create_or_get(
&mut self,
topic: &KafkaTopic,
partition: KafkaPartition,
) -> Result<Sequencer> {
) -> Result<Shard> {
let stage = self.stage();
let sequencer = match stage
.sequencers
let shard = match stage
.shards
.iter()
.find(|s| s.kafka_topic_id == topic.id && s.kafka_partition == partition)
{
Some(t) => t,
None => {
let sequencer = Sequencer {
id: SequencerId::new(stage.sequencers.len() as i64 + 1),
let shard = Shard {
id: ShardId::new(stage.shards.len() as i64 + 1),
kafka_topic_id: topic.id,
kafka_partition: partition,
min_unpersisted_sequence_number: SequenceNumber::new(0),
};
stage.sequencers.push(sequencer);
stage.sequencers.last().unwrap()
stage.shards.push(shard);
stage.shards.last().unwrap()
}
};
Ok(*sequencer)
Ok(*shard)
}
async fn get_by_topic_id_and_partition(
&mut self,
topic_id: KafkaTopicId,
partition: KafkaPartition,
) -> Result<Option<Sequencer>> {
) -> Result<Option<Shard>> {
let stage = self.stage();
let sequencer = stage
.sequencers
let shard = stage
.shards
.iter()
.find(|s| s.kafka_topic_id == topic_id && s.kafka_partition == partition)
.cloned();
Ok(sequencer)
Ok(shard)
}
async fn list(&mut self) -> Result<Vec<Sequencer>> {
async fn list(&mut self) -> Result<Vec<Shard>> {
let stage = self.stage();
Ok(stage.sequencers.clone())
Ok(stage.shards.clone())
}
async fn list_by_kafka_topic(&mut self, topic: &KafkaTopic) -> Result<Vec<Sequencer>> {
async fn list_by_kafka_topic(&mut self, topic: &KafkaTopic) -> Result<Vec<Shard>> {
let stage = self.stage();
let sequencers: Vec<_> = stage
.sequencers
let shards: Vec<_> = stage
.shards
.iter()
.filter(|s| s.kafka_topic_id == topic.id)
.cloned()
.collect();
Ok(sequencers)
Ok(shards)
}
async fn update_min_unpersisted_sequence_number(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) -> Result<()> {
let stage = self.stage();
if let Some(s) = stage.sequencers.iter_mut().find(|s| s.id == sequencer_id) {
if let Some(s) = stage.shards.iter_mut().find(|s| s.id == shard_id) {
s.min_unpersisted_sequence_number = sequence_number
};
@ -686,19 +686,19 @@ impl PartitionRepo for MemTxn {
async fn create_or_get(
&mut self,
key: PartitionKey,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
) -> Result<Partition> {
let stage = self.stage();
let partition = match stage.partitions.iter().find(|p| {
p.partition_key == key && p.sequencer_id == sequencer_id && p.table_id == table_id
p.partition_key == key && p.shard_id == shard_id && p.table_id == table_id
}) {
Some(p) => p,
None => {
let p = Partition {
id: PartitionId::new(stage.partitions.len() as i64 + 1),
sequencer_id,
shard_id,
table_id,
partition_key: key,
sort_key: vec![],
@ -721,13 +721,13 @@ impl PartitionRepo for MemTxn {
.cloned())
}
async fn list_by_sequencer(&mut self, sequencer_id: SequencerId) -> Result<Vec<Partition>> {
async fn list_by_shard(&mut self, shard_id: ShardId) -> Result<Vec<Partition>> {
let stage = self.stage();
let partitions: Vec<_> = stage
.partitions
.iter()
.filter(|p| p.sequencer_id == sequencer_id)
.filter(|p| p.shard_id == shard_id)
.cloned()
.collect();
Ok(partitions)
@ -820,7 +820,7 @@ impl TombstoneRepo for MemTxn {
async fn create_or_get(
&mut self,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
min_time: Timestamp,
max_time: Timestamp,
@ -830,7 +830,7 @@ impl TombstoneRepo for MemTxn {
let tombstone = match stage.tombstones.iter().find(|t| {
t.table_id == table_id
&& t.sequencer_id == sequencer_id
&& t.shard_id == shard_id
&& t.sequence_number == sequence_number
}) {
Some(t) => t,
@ -838,7 +838,7 @@ impl TombstoneRepo for MemTxn {
let t = Tombstone {
id: TombstoneId::new(stage.tombstones.len() as i64 + 1),
table_id,
sequencer_id,
shard_id,
sequence_number,
min_time,
max_time,
@ -887,9 +887,9 @@ impl TombstoneRepo for MemTxn {
Ok(stage.tombstones.iter().find(|t| t.id == id).cloned())
}
async fn list_tombstones_by_sequencer_greater_than(
async fn list_tombstones_by_shard_greater_than(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) -> Result<Vec<Tombstone>> {
let stage = self.stage();
@ -897,7 +897,7 @@ impl TombstoneRepo for MemTxn {
let tombstones: Vec<_> = stage
.tombstones
.iter()
.filter(|t| t.sequencer_id == sequencer_id && t.sequence_number > sequence_number)
.filter(|t| t.shard_id == shard_id && t.sequence_number > sequence_number)
.cloned()
.collect();
Ok(tombstones)
@ -921,7 +921,7 @@ impl TombstoneRepo for MemTxn {
async fn list_tombstones_for_time_range(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
sequence_number: SequenceNumber,
min_time: Timestamp,
@ -933,7 +933,7 @@ impl TombstoneRepo for MemTxn {
.tombstones
.iter()
.filter(|t| {
t.sequencer_id == sequencer_id
t.shard_id == shard_id
&& t.table_id == table_id
&& t.sequence_number > sequence_number
&& ((t.min_time <= min_time && t.max_time >= min_time)
@ -951,7 +951,7 @@ impl ParquetFileRepo for MemTxn {
let stage = self.stage();
let ParquetFileParams {
sequencer_id,
shard_id,
namespace_id,
table_id,
partition_id,
@ -976,7 +976,7 @@ impl ParquetFileRepo for MemTxn {
let parquet_file = ParquetFile {
id: ParquetFileId::new(stage.parquet_files.len() as i64 + 1),
sequencer_id,
shard_id,
namespace_id,
table_id,
partition_id,
@ -1008,9 +1008,9 @@ impl ParquetFileRepo for MemTxn {
Ok(())
}
async fn list_by_sequencer_greater_than(
async fn list_by_shard_greater_than(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) -> Result<Vec<ParquetFile>> {
let stage = self.stage();
@ -1018,7 +1018,7 @@ impl ParquetFileRepo for MemTxn {
let files: Vec<_> = stage
.parquet_files
.iter()
.filter(|f| f.sequencer_id == sequencer_id && f.max_sequence_number > sequence_number)
.filter(|f| f.shard_id == shard_id && f.max_sequence_number > sequence_number)
.cloned()
.collect();
Ok(files)
@ -1068,14 +1068,14 @@ impl ParquetFileRepo for MemTxn {
Ok(delete)
}
async fn level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>> {
async fn level_0(&mut self, shard_id: ShardId) -> Result<Vec<ParquetFile>> {
let stage = self.stage();
Ok(stage
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == sequencer_id
f.shard_id == shard_id
&& f.compaction_level == CompactionLevel::Initial
&& f.to_delete.is_none()
})
@ -1095,7 +1095,7 @@ impl ParquetFileRepo for MemTxn {
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == table_partition.sequencer_id
f.shard_id == table_partition.shard_id
&& f.table_id == table_partition.table_id
&& f.partition_id == table_partition.partition_id
&& f.compaction_level == CompactionLevel::FileNonOverlapped
@ -1109,7 +1109,7 @@ impl ParquetFileRepo for MemTxn {
async fn recent_highest_throughput_partitions(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
num_hours: u32,
min_num_files: usize,
num_partitions: usize,
@ -1126,14 +1126,14 @@ impl ParquetFileRepo for MemTxn {
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == sequencer_id
f.shard_id == shard_id
&& f.created_at > recent_time
&& f.compaction_level == CompactionLevel::Initial
&& f.to_delete.is_none()
})
.map(|pf| PartitionParam {
partition_id: pf.partition_id,
sequencer_id: pf.sequencer_id,
shard_id: pf.shard_id,
namespace_id: pf.namespace_id,
table_id: pf.table_id,
})
@ -1168,7 +1168,7 @@ impl ParquetFileRepo for MemTxn {
async fn most_level_0_files_partitions(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
older_than_num_hours: u32,
num_partitions: usize,
) -> Result<Vec<PartitionParam>> {
@ -1182,7 +1182,7 @@ impl ParquetFileRepo for MemTxn {
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == sequencer_id
f.shard_id == shard_id
&& f.compaction_level == CompactionLevel::Initial
&& f.to_delete.is_none()
})
@ -1195,7 +1195,7 @@ impl ParquetFileRepo for MemTxn {
for pf in relevant_parquet_files {
let key = PartitionParam {
partition_id: pf.partition_id,
sequencer_id: pf.sequencer_id,
shard_id: pf.shard_id,
namespace_id: pf.namespace_id,
table_id: pf.table_id,
};
@ -1220,7 +1220,7 @@ impl ParquetFileRepo for MemTxn {
.map(|(k, _)| *k)
.map(|pf| PartitionParam {
partition_id: pf.partition_id,
sequencer_id: pf.sequencer_id,
shard_id: pf.shard_id,
namespace_id: pf.namespace_id,
table_id: pf.table_id,
})
@ -1284,7 +1284,7 @@ impl ParquetFileRepo for MemTxn {
async fn count_by_overlaps_with_level_0(
&mut self,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
min_time: Timestamp,
max_time: Timestamp,
sequence_number: SequenceNumber,
@ -1295,7 +1295,7 @@ impl ParquetFileRepo for MemTxn {
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == sequencer_id
f.shard_id == shard_id
&& f.table_id == table_id
&& f.max_sequence_number < sequence_number
&& f.to_delete.is_none()
@ -1311,7 +1311,7 @@ impl ParquetFileRepo for MemTxn {
async fn count_by_overlaps_with_level_1(
&mut self,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
min_time: Timestamp,
max_time: Timestamp,
) -> Result<i64> {
@ -1321,7 +1321,7 @@ impl ParquetFileRepo for MemTxn {
.parquet_files
.iter()
.filter(|f| {
f.sequencer_id == sequencer_id
f.shard_id == shard_id
&& f.table_id == table_id
&& f.to_delete.is_none()
&& f.compaction_level == CompactionLevel::FileNonOverlapped

View File

@ -3,14 +3,14 @@
use crate::interface::{
sealed::TransactionFinalize, ColumnRepo, ColumnUpsertRequest, KafkaTopicRepo, NamespaceRepo,
ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo, QueryPoolRepo, RepoCollection, Result,
SequencerRepo, TablePersistInfo, TableRepo, TombstoneRepo,
ShardRepo, TablePersistInfo, TableRepo, TombstoneRepo,
};
use async_trait::async_trait;
use data_types::{
Column, ColumnType, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace, NamespaceId,
ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId, PartitionInfo,
PartitionKey, PartitionParam, ProcessedTombstone, QueryPool, QueryPoolId, SequenceNumber,
Sequencer, SequencerId, Table, TableId, TablePartition, Timestamp, Tombstone, TombstoneId,
Shard, ShardId, Table, TableId, TablePartition, Timestamp, Tombstone, TombstoneId,
};
use iox_time::{SystemProvider, TimeProvider};
use metric::{DurationHistogram, Metric};
@ -48,7 +48,7 @@ where
+ NamespaceRepo
+ TableRepo
+ ColumnRepo
+ SequencerRepo
+ ShardRepo
+ PartitionRepo
+ TombstoneRepo
+ ProcessedTombstoneRepo
@ -76,7 +76,7 @@ where
self
}
fn sequencers(&mut self) -> &mut dyn SequencerRepo {
fn shards(&mut self) -> &mut dyn ShardRepo {
self
}
@ -208,7 +208,7 @@ decorate!(
"table_get_by_id" = get_by_id(&mut self, table_id: TableId) -> Result<Option<Table>>;
"table_get_by_namespace_and_name" = get_by_namespace_and_name(&mut self, namespace_id: NamespaceId, name: &str) -> Result<Option<Table>>;
"table_list_by_namespace_id" = list_by_namespace_id(&mut self, namespace_id: NamespaceId) -> Result<Vec<Table>>;
"get_table_persist_info" = get_table_persist_info(&mut self, sequencer_id: SequencerId, namespace_id: NamespaceId, table_name: &str) -> Result<Option<TablePersistInfo>>;
"get_table_persist_info" = get_table_persist_info(&mut self, shard_id: ShardId, namespace_id: NamespaceId, table_name: &str) -> Result<Option<TablePersistInfo>>;
"table_list" = list(&mut self) -> Result<Vec<Table>>;
]
);
@ -225,22 +225,22 @@ decorate!(
);
decorate!(
impl_trait = SequencerRepo,
impl_trait = ShardRepo,
methods = [
"sequencer_create_or_get" = create_or_get(&mut self, topic: &KafkaTopic, partition: KafkaPartition) -> Result<Sequencer>;
"sequencer_get_by_topic_id_and_partition" = get_by_topic_id_and_partition(&mut self, topic_id: KafkaTopicId, partition: KafkaPartition) -> Result<Option<Sequencer>>;
"sequencer_list" = list(&mut self) -> Result<Vec<Sequencer>>;
"sequencer_list_by_kafka_topic" = list_by_kafka_topic(&mut self, topic: &KafkaTopic) -> Result<Vec<Sequencer>>;
"sequencer_update_min_unpersisted_sequence_number" = update_min_unpersisted_sequence_number(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result<()>;
"shard_create_or_get" = create_or_get(&mut self, topic: &KafkaTopic, partition: KafkaPartition) -> Result<Shard>;
"shard_get_by_topic_id_and_partition" = get_by_topic_id_and_partition(&mut self, topic_id: KafkaTopicId, partition: KafkaPartition) -> Result<Option<Shard>>;
"shard_list" = list(&mut self) -> Result<Vec<Shard>>;
"shard_list_by_kafka_topic" = list_by_kafka_topic(&mut self, topic: &KafkaTopic) -> Result<Vec<Shard>>;
"shard_update_min_unpersisted_sequence_number" = update_min_unpersisted_sequence_number(&mut self, shard_id: ShardId, sequence_number: SequenceNumber) -> Result<()>;
]
);
decorate!(
impl_trait = PartitionRepo,
methods = [
"partition_create_or_get" = create_or_get(&mut self, key: PartitionKey, sequencer_id: SequencerId, table_id: TableId) -> Result<Partition>;
"partition_create_or_get" = create_or_get(&mut self, key: PartitionKey, shard_id: ShardId, table_id: TableId) -> Result<Partition>;
"partition_get_by_id" = get_by_id(&mut self, partition_id: PartitionId) -> Result<Option<Partition>>;
"partition_list_by_sequencer" = list_by_sequencer(&mut self, sequencer_id: SequencerId) -> Result<Vec<Partition>>;
"partition_list_by_shard" = list_by_shard(&mut self, shard_id: ShardId) -> Result<Vec<Partition>>;
"partition_list_by_namespace" = list_by_namespace(&mut self, namespace_id: NamespaceId) -> Result<Vec<Partition>>;
"partition_list_by_table_id" = list_by_table_id(&mut self, table_id: TableId) -> Result<Vec<Partition>>;
"partition_partition_info_by_id" = partition_info_by_id(&mut self, partition_id: PartitionId) -> Result<Option<PartitionInfo>>;
@ -251,13 +251,13 @@ decorate!(
decorate!(
impl_trait = TombstoneRepo,
methods = [
"tombstone_create_or_get" = create_or_get( &mut self, table_id: TableId, sequencer_id: SequencerId, sequence_number: SequenceNumber, min_time: Timestamp, max_time: Timestamp, predicate: &str) -> Result<Tombstone>;
"tombstone_create_or_get" = create_or_get( &mut self, table_id: TableId, shard_id: ShardId, sequence_number: SequenceNumber, min_time: Timestamp, max_time: Timestamp, predicate: &str) -> Result<Tombstone>;
"tombstone_list_by_namespace" = list_by_namespace(&mut self, namespace_id: NamespaceId) -> Result<Vec<Tombstone>>;
"tombstone_list_by_table" = list_by_table(&mut self, table_id: TableId) -> Result<Vec<Tombstone>>;
"tombstone_get_by_id" = get_by_id(&mut self, id: TombstoneId) -> Result<Option<Tombstone>>;
"tombstone_list_tombstones_by_sequencer_greater_than" = list_tombstones_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result<Vec<Tombstone>>;
"tombstone_list_tombstones_by_shard_greater_than" = list_tombstones_by_shard_greater_than(&mut self, shard_id: ShardId, sequence_number: SequenceNumber) -> Result<Vec<Tombstone>>;
"tombstone_remove" = remove(&mut self, tombstone_ids: &[TombstoneId]) -> Result<()>;
"tombstone_list_tombstones_for_time_range" = list_tombstones_for_time_range(&mut self, sequencer_id: SequencerId, table_id: TableId, sequence_number: SequenceNumber, min_time: Timestamp, max_time: Timestamp) -> Result<Vec<Tombstone>>;
"tombstone_list_tombstones_for_time_range" = list_tombstones_for_time_range(&mut self, shard_id: ShardId, table_id: TableId, sequence_number: SequenceNumber, min_time: Timestamp, max_time: Timestamp) -> Result<Vec<Tombstone>>;
]
);
@ -266,21 +266,21 @@ decorate!(
methods = [
"parquet_create" = create( &mut self, parquet_file_params: ParquetFileParams) -> Result<ParquetFile>;
"parquet_flag_for_delete" = flag_for_delete(&mut self, id: ParquetFileId) -> Result<()>;
"parquet_list_by_sequencer_greater_than" = list_by_sequencer_greater_than(&mut self, sequencer_id: SequencerId, sequence_number: SequenceNumber) -> Result<Vec<ParquetFile>>;
"parquet_list_by_shard_greater_than" = list_by_shard_greater_than(&mut self, shard_id: ShardId, sequence_number: SequenceNumber) -> Result<Vec<ParquetFile>>;
"parquet_list_by_namespace_not_to_delete" = list_by_namespace_not_to_delete(&mut self, namespace_id: NamespaceId) -> Result<Vec<ParquetFile>>;
"parquet_list_by_table_not_to_delete" = list_by_table_not_to_delete(&mut self, table_id: TableId) -> Result<Vec<ParquetFile>>;
"parquet_delete_old" = delete_old(&mut self, older_than: Timestamp) -> Result<Vec<ParquetFile>>;
"parquet_list_by_partition_not_to_delete" = list_by_partition_not_to_delete(&mut self, partition_id: PartitionId) -> Result<Vec<ParquetFile>>;
"parquet_level_0" = level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>>;
"parquet_level_0" = level_0(&mut self, shard_id: ShardId) -> Result<Vec<ParquetFile>>;
"parquet_level_1" = level_1(&mut self, table_partition: TablePartition, min_time: Timestamp, max_time: Timestamp) -> Result<Vec<ParquetFile>>;
"parquet_update_to_level_1" = update_to_level_1(&mut self, parquet_file_ids: &[ParquetFileId]) -> Result<Vec<ParquetFileId>>;
"parquet_exist" = exist(&mut self, id: ParquetFileId) -> Result<bool>;
"parquet_count" = count(&mut self) -> Result<i64>;
"parquet_count_by_overlaps_with_level_0" = count_by_overlaps_with_level_0(&mut self, table_id: TableId, sequencer_id: SequencerId, min_time: Timestamp, max_time: Timestamp, sequence_number: SequenceNumber) -> Result<i64>;
"parquet_count_by_overlaps_with_level_1" = count_by_overlaps_with_level_1(&mut self, table_id: TableId, sequencer_id: SequencerId, min_time: Timestamp, max_time: Timestamp) -> Result<i64>;
"parquet_count_by_overlaps_with_level_0" = count_by_overlaps_with_level_0(&mut self, table_id: TableId, shard_id: ShardId, min_time: Timestamp, max_time: Timestamp, sequence_number: SequenceNumber) -> Result<i64>;
"parquet_count_by_overlaps_with_level_1" = count_by_overlaps_with_level_1(&mut self, table_id: TableId, shard_id: ShardId, min_time: Timestamp, max_time: Timestamp) -> Result<i64>;
"parquet_get_by_object_store_id" = get_by_object_store_id(&mut self, object_store_id: Uuid) -> Result<Option<ParquetFile>>;
"recent_highest_throughput_partitions" = recent_highest_throughput_partitions(&mut self, sequencer_id: SequencerId, num_hours: u32, min_num_files: usize, num_partitions: usize) -> Result<Vec<PartitionParam>>;
"most_level_0_files_partitions" = most_level_0_files_partitions(&mut self, sequencer_id: SequencerId, older_than_num_hours: u32, num_partitions: usize) -> Result<Vec<PartitionParam>>;
"recent_highest_throughput_partitions" = recent_highest_throughput_partitions(&mut self, shard_id: ShardId, num_hours: u32, min_num_files: usize, num_partitions: usize) -> Result<Vec<PartitionParam>>;
"most_level_0_files_partitions" = most_level_0_files_partitions(&mut self, shard_id: ShardId, older_than_num_hours: u32, num_partitions: usize) -> Result<Vec<PartitionParam>>;
]
);

View File

@ -4,7 +4,7 @@ use crate::{
interface::{
sealed::TransactionFinalize, Catalog, ColumnRepo, ColumnUpsertRequest, Error,
KafkaTopicRepo, NamespaceRepo, ParquetFileRepo, PartitionRepo, ProcessedTombstoneRepo,
QueryPoolRepo, RepoCollection, Result, SequencerRepo, TablePersistInfo, TableRepo,
QueryPoolRepo, RepoCollection, Result, ShardRepo, TablePersistInfo, TableRepo,
TombstoneRepo, Transaction,
},
metrics::MetricDecorator,
@ -14,7 +14,7 @@ use data_types::{
Column, ColumnType, CompactionLevel, KafkaPartition, KafkaTopic, KafkaTopicId, Namespace,
NamespaceId, ParquetFile, ParquetFileId, ParquetFileParams, Partition, PartitionId,
PartitionInfo, PartitionKey, PartitionParam, ProcessedTombstone, QueryPool, QueryPoolId,
SequenceNumber, Sequencer, SequencerId, Table, TableId, TablePartition, Timestamp, Tombstone,
SequenceNumber, Shard, ShardId, Table, TableId, TablePartition, Timestamp, Tombstone,
TombstoneId,
};
use iox_time::{SystemProvider, TimeProvider};
@ -481,7 +481,7 @@ impl RepoCollection for PostgresTxn {
self
}
fn sequencers(&mut self) -> &mut dyn SequencerRepo {
fn shards(&mut self) -> &mut dyn ShardRepo {
self
}
@ -828,7 +828,7 @@ WHERE namespace_id = $1;
async fn get_table_persist_info(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
namespace_id: NamespaceId,
table_name: &str,
) -> Result<Option<TablePersistInfo>> {
@ -847,7 +847,7 @@ LEFT JOIN (
) tombstone ON tombstone.table_id = tid.id
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.bind(&table_name) // $2
.bind(&namespace_id) // $3
.fetch_one(&mut self.inner)
@ -1012,13 +1012,13 @@ RETURNING *;
}
#[async_trait]
impl SequencerRepo for PostgresTxn {
impl ShardRepo for PostgresTxn {
async fn create_or_get(
&mut self,
topic: &KafkaTopic,
partition: KafkaPartition,
) -> Result<Sequencer> {
sqlx::query_as::<_, Sequencer>(
) -> Result<Shard> {
sqlx::query_as::<_, Shard>(
r#"
INSERT INTO sequencer
( kafka_topic_id, kafka_partition, min_unpersisted_sequence_number )
@ -1046,8 +1046,8 @@ RETURNING *;;
&mut self,
topic_id: KafkaTopicId,
partition: KafkaPartition,
) -> Result<Option<Sequencer>> {
let rec = sqlx::query_as::<_, Sequencer>(
) -> Result<Option<Shard>> {
let rec = sqlx::query_as::<_, Shard>(
r#"
SELECT *
FROM sequencer
@ -1064,20 +1064,20 @@ WHERE kafka_topic_id = $1
return Ok(None);
}
let sequencer = rec.map_err(|e| Error::SqlxError { source: e })?;
let shard = rec.map_err(|e| Error::SqlxError { source: e })?;
Ok(Some(sequencer))
Ok(Some(shard))
}
async fn list(&mut self) -> Result<Vec<Sequencer>> {
sqlx::query_as::<_, Sequencer>(r#"SELECT * FROM sequencer;"#)
async fn list(&mut self) -> Result<Vec<Shard>> {
sqlx::query_as::<_, Shard>(r#"SELECT * FROM sequencer;"#)
.fetch_all(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })
}
async fn list_by_kafka_topic(&mut self, topic: &KafkaTopic) -> Result<Vec<Sequencer>> {
sqlx::query_as::<_, Sequencer>(r#"SELECT * FROM sequencer WHERE kafka_topic_id = $1;"#)
async fn list_by_kafka_topic(&mut self, topic: &KafkaTopic) -> Result<Vec<Shard>> {
sqlx::query_as::<_, Shard>(r#"SELECT * FROM sequencer WHERE kafka_topic_id = $1;"#)
.bind(&topic.id) // $1
.fetch_all(&mut self.inner)
.await
@ -1086,14 +1086,14 @@ WHERE kafka_topic_id = $1
async fn update_min_unpersisted_sequence_number(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) -> Result<()> {
let _ = sqlx::query(
r#"UPDATE sequencer SET min_unpersisted_sequence_number = $1 WHERE id = $2;"#,
)
.bind(&sequence_number.get()) // $1
.bind(&sequencer_id) // $2
.bind(&shard_id) // $2
.execute(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })?;
@ -1107,7 +1107,7 @@ impl PartitionRepo for PostgresTxn {
async fn create_or_get(
&mut self,
key: PartitionKey,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
) -> Result<Partition> {
// Note: since sort_key is now an array, we must explicitly insert '{}' which is an empty array
@ -1125,7 +1125,7 @@ RETURNING *;
"#,
)
.bind(key) // $1
.bind(&sequencer_id) // $2
.bind(&shard_id) // $2
.bind(&table_id) // $3
.fetch_one(&mut self.inner)
.await
@ -1139,11 +1139,11 @@ RETURNING *;
// If the partition_key_unique constraint was hit because there was an
// existing record for (table_id, partition_key) ensure the partition
// key in the DB is mapped to the same sequencer_id the caller
// key in the DB is mapped to the same shard_id the caller
// requested.
assert_eq!(
v.sequencer_id, sequencer_id,
"attempted to overwrite partition with different sequencer ID"
v.shard_id, shard_id,
"attempted to overwrite partition with different shard ID"
);
Ok(v)
@ -1164,9 +1164,9 @@ RETURNING *;
Ok(Some(partition))
}
async fn list_by_sequencer(&mut self, sequencer_id: SequencerId) -> Result<Vec<Partition>> {
async fn list_by_shard(&mut self, shard_id: ShardId) -> Result<Vec<Partition>> {
sqlx::query_as::<_, Partition>(r#"SELECT * FROM partition WHERE sequencer_id = $1;"#)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.fetch_all(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })
@ -1223,7 +1223,7 @@ WHERE partition.id = $1;
let table_name = info.get("table_name");
let partition = Partition {
id: info.get("id"),
sequencer_id: info.get("sequencer_id"),
shard_id: info.get("sequencer_id"),
table_id: info.get("table_id"),
partition_key: info.get("partition_key"),
sort_key: info.get("sort_key"),
@ -1270,7 +1270,7 @@ impl TombstoneRepo for PostgresTxn {
async fn create_or_get(
&mut self,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
min_time: Timestamp,
max_time: Timestamp,
@ -1288,7 +1288,7 @@ RETURNING *;
"#,
)
.bind(&table_id) // $1
.bind(&sequencer_id) // $2
.bind(&shard_id) // $2
.bind(&sequence_number) // $3
.bind(&min_time) // $4
.bind(&max_time) // $5
@ -1303,7 +1303,7 @@ RETURNING *;
}
})?;
// If tombstone_unique is hit, a record with (table_id, sequencer_id,
// If tombstone_unique is hit, a record with (table_id, shard_id,
// sequence_number) already exists.
//
// Ensure the caller does not falsely believe they have created the
@ -1383,9 +1383,9 @@ WHERE id = $1;
Ok(Some(tombstone))
}
async fn list_tombstones_by_sequencer_greater_than(
async fn list_tombstones_by_shard_greater_than(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) -> Result<Vec<Tombstone>> {
sqlx::query_as::<_, Tombstone>(
@ -1397,7 +1397,7 @@ WHERE sequencer_id = $1
ORDER BY id;
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.bind(&sequence_number) // $2
.fetch_all(&mut self.inner)
.await
@ -1438,7 +1438,7 @@ WHERE id = ANY($1);
async fn list_tombstones_for_time_range(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
table_id: TableId,
sequence_number: SequenceNumber,
min_time: Timestamp,
@ -1456,7 +1456,7 @@ WHERE sequencer_id = $1
ORDER BY id;
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.bind(&table_id) // $2
.bind(&sequence_number) // $3
.bind(&min_time) // $4
@ -1471,7 +1471,7 @@ ORDER BY id;
impl ParquetFileRepo for PostgresTxn {
async fn create(&mut self, parquet_file_params: ParquetFileParams) -> Result<ParquetFile> {
let ParquetFileParams {
sequencer_id,
shard_id,
namespace_id,
table_id,
partition_id,
@ -1496,7 +1496,7 @@ VALUES ( $1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11, $12, $13 )
RETURNING *;
"#,
)
.bind(sequencer_id) // $1
.bind(shard_id) // $1
.bind(table_id) // $2
.bind(partition_id) // $3
.bind(object_store_id) // $4
@ -1537,9 +1537,9 @@ RETURNING *;
Ok(())
}
async fn list_by_sequencer_greater_than(
async fn list_by_shard_greater_than(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
) -> Result<Vec<ParquetFile>> {
// Deliberately doesn't use `SELECT *` to avoid the performance hit of fetching the large
@ -1555,7 +1555,7 @@ WHERE sequencer_id = $1
ORDER BY id;
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.bind(&sequence_number) // $2
.fetch_all(&mut self.inner)
.await
@ -1619,7 +1619,7 @@ RETURNING *;
.map_err(|e| Error::SqlxError { source: e })
}
async fn level_0(&mut self, sequencer_id: SequencerId) -> Result<Vec<ParquetFile>> {
async fn level_0(&mut self, shard_id: ShardId) -> Result<Vec<ParquetFile>> {
// this intentionally limits the returned files to 10,000 as it is used to make
// a decision on the highest priority partitions. If compaction has never been
// run this could end up returning millions of results and taking too long to run.
@ -1637,7 +1637,7 @@ WHERE parquet_file.sequencer_id = $1
LIMIT 1000;
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.fetch_all(&mut self.inner)
.await
.map_err(|e| Error::SqlxError { source: e })
@ -1666,7 +1666,7 @@ WHERE parquet_file.sequencer_id = $1
OR (parquet_file.min_time > $5 AND parquet_file.min_time <= $6));
"#,
)
.bind(&table_partition.sequencer_id) // $1
.bind(&table_partition.shard_id) // $1
.bind(&table_partition.table_id) // $2
.bind(&table_partition.partition_id) // $3
.bind(CompactionLevel::FileNonOverlapped) // $4
@ -1679,7 +1679,7 @@ WHERE parquet_file.sequencer_id = $1
async fn recent_highest_throughput_partitions(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
num_hours: u32,
min_num_files: usize,
num_partitions: usize,
@ -1689,9 +1689,9 @@ WHERE parquet_file.sequencer_id = $1
let num_partitions = num_partitions as i32;
// The preliminary performance test on 6 days of data, this query runs around 55ms
// We have index on (sequencer_id, comapction_level, to_delete)
// We have index on (shard_id, comapction_level, to_delete)
// If this query happens to be a lot slower (>500ms), we might think to add
// and index on (sequencer_id, comapction_level, to_delete, created_at)
// and index on (shard_id, comapction_level, to_delete, created_at)
sqlx::query_as::<_, PartitionParam>(
r#"
SELECT partition_id, sequencer_id, namespace_id, table_id, count(id)
@ -1705,7 +1705,7 @@ order by 5 DESC
limit $4;
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.bind(num_hours) //$2
.bind(&min_num_files) // $3
.bind(&num_partitions) // $4
@ -1716,7 +1716,7 @@ limit $4;
async fn most_level_0_files_partitions(
&mut self,
sequencer_id: SequencerId,
shard_id: ShardId,
older_than_num_hours: u32,
num_partitions: usize,
) -> Result<Vec<PartitionParam>> {
@ -1724,7 +1724,7 @@ limit $4;
let num_partitions = num_partitions as i32;
// The preliminary performance test says this query runs around 50ms
// We have index on (sequencer_id, comapction_level, to_delete)
// We have index on (shard_id, comapction_level, to_delete)
sqlx::query_as::<_, PartitionParam>(
r#"
SELECT partition_id, sequencer_id, namespace_id, table_id, count(id), max(created_at)
@ -1738,7 +1738,7 @@ ORDER BY 5 DESC
LIMIT $3;
"#,
)
.bind(&sequencer_id) // $1
.bind(&shard_id) // $1
.bind(&older_than_num_hours) // $2
.bind(&num_partitions) // $3
.fetch_all(&mut self.inner)
@ -1818,7 +1818,7 @@ RETURNING id;
async fn count_by_overlaps_with_level_0(
&mut self,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
min_time: Timestamp,
max_time: Timestamp,
sequence_number: SequenceNumber,
@ -1837,7 +1837,7 @@ WHERE table_id = $1
"#,
)
.bind(&table_id) // $1
.bind(&sequencer_id) // $2
.bind(&shard_id) // $2
.bind(sequence_number) // $3
.bind(min_time) // $4
.bind(max_time) // $5
@ -1851,7 +1851,7 @@ WHERE table_id = $1
async fn count_by_overlaps_with_level_1(
&mut self,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
min_time: Timestamp,
max_time: Timestamp,
) -> Result<i64> {
@ -1868,7 +1868,7 @@ WHERE table_id = $1
"#,
)
.bind(&table_id) // $1
.bind(&sequencer_id) // $2
.bind(&shard_id) // $2
.bind(min_time) // $3
.bind(max_time) // $4
.fetch_one(&mut self.inner)
@ -2172,7 +2172,7 @@ mod tests {
let postgres: Arc<dyn Catalog> = Arc::new(postgres);
let mut txn = postgres.start_transaction().await.expect("txn start");
let (kafka, query, sequencers) = create_or_get_default_records(1, txn.deref_mut())
let (kafka, query, shards) = create_or_get_default_records(1, txn.deref_mut())
.await
.expect("db init failed");
txn.commit().await.expect("txn commit");
@ -2194,7 +2194,7 @@ mod tests {
.expect("create table failed")
.id;
let sequencer_id = *sequencers.keys().next().expect("no sequencer");
let shard_id = *shards.keys().next().expect("no shard");
let sequence_number = SequenceNumber::new(3);
let min_timestamp = Timestamp::new(10);
let max_timestamp = Timestamp::new(100);
@ -2206,7 +2206,7 @@ mod tests {
.tombstones()
.create_or_get(
table_id,
sequencer_id,
shard_id,
sequence_number,
min_timestamp,
max_timestamp,
@ -2215,7 +2215,7 @@ mod tests {
.await
.expect("should create OK");
// Call create_or_get for the same (table_id, sequencer_id,
// Call create_or_get for the same (table_id, shard_id,
// sequence_number) triplet, setting the same metadata to ensure the
// write is idempotent.
let b = postgres
@ -2224,7 +2224,7 @@ mod tests {
.tombstones()
.create_or_get(
table_id,
sequencer_id,
shard_id,
sequence_number,
min_timestamp,
max_timestamp,
@ -2248,7 +2248,7 @@ mod tests {
let postgres: Arc<dyn Catalog> = Arc::new(postgres);
let mut txn = postgres.start_transaction().await.expect("txn start");
let (kafka, query, sequencers) = create_or_get_default_records(1, txn.deref_mut())
let (kafka, query, shards) = create_or_get_default_records(1, txn.deref_mut())
.await
.expect("db init failed");
txn.commit().await.expect("txn commit");
@ -2270,7 +2270,7 @@ mod tests {
.expect("create table failed")
.id;
let sequencer_id = *sequencers.keys().next().expect("no sequencer");
let shard_id = *shards.keys().next().expect("no shard");
let sequence_number = SequenceNumber::new(3);
let min_timestamp = Timestamp::new(10);
let max_timestamp = Timestamp::new(100);
@ -2281,7 +2281,7 @@ mod tests {
.tombstones()
.create_or_get(
table_id,
sequencer_id,
shard_id,
sequence_number,
min_timestamp,
max_timestamp,
@ -2290,7 +2290,7 @@ mod tests {
.await
.expect("should create OK");
// Call create_or_get for the same (table_id, sequencer_id,
// Call create_or_get for the same (table_id, shard_id,
// sequence_number) triplet with different metadata.
//
// The caller should not falsely believe it has persisted the incorrect
@ -2301,7 +2301,7 @@ mod tests {
.tombstones()
.create_or_get(
table_id,
sequencer_id,
shard_id,
sequence_number,
min_timestamp,
max_timestamp,
@ -2331,7 +2331,7 @@ mod tests {
let postgres: Arc<dyn Catalog> = Arc::new(postgres);
let mut txn = postgres.start_transaction().await.expect("txn start");
let (kafka, query, sequencers) = create_or_get_default_records(1, txn.deref_mut())
let (kafka, query, shards) = create_or_get_default_records(1, txn.deref_mut())
.await
.expect("db init failed");
txn.commit().await.expect("txn commit");
@ -2354,24 +2354,24 @@ mod tests {
.id;
let key = "bananas";
let sequencer_id = *sequencers.keys().next().expect("no sequencer");
let shard_id = *shards.keys().next().expect("no shard");
let a = postgres
.repositories()
.await
.partitions()
.create_or_get(key.into(), sequencer_id, table_id)
.create_or_get(key.into(), shard_id, table_id)
.await
.expect("should create OK");
// Call create_or_get for the same (key, table_id, sequencer_id)
// triplet, setting the same sequencer ID to ensure the write is
// Call create_or_get for the same (key, table_id, shard_id)
// triplet, setting the same shard ID to ensure the write is
// idempotent.
let b = postgres
.repositories()
.await
.partitions()
.create_or_get(key.into(), sequencer_id, table_id)
.create_or_get(key.into(), shard_id, table_id)
.await
.expect("idempotent write should succeed");
@ -2414,34 +2414,34 @@ mod tests {
let key = "bananas";
let sequencers = postgres
let shards = postgres
.repositories()
.await
.sequencers()
.shards()
.list()
.await
.expect("failed to list sequencers");
.expect("failed to list shards");
assert!(
sequencers.len() > 1,
"expected more sequencers to be created, got {}",
sequencers.len()
shards.len() > 1,
"expected more shards to be created, got {}",
shards.len()
);
let a = postgres
.repositories()
.await
.partitions()
.create_or_get(key.into(), sequencers[0].id, table_id)
.create_or_get(key.into(), shards[0].id, table_id)
.await
.expect("should create OK");
// Call create_or_get for the same (key, table_id) tuple, setting a
// different sequencer ID
// different shard ID
let b = postgres
.repositories()
.await
.partitions()
.create_or_get(key.into(), sequencers[1].id, table_id)
.create_or_get(key.into(), shards[1].id, table_id)
.await
.expect("result should not be evaluated");
@ -2551,7 +2551,7 @@ mod tests {
let postgres: Arc<dyn Catalog> = Arc::new(postgres);
let mut txn = postgres.start_transaction().await.expect("txn start");
let (kafka, query, _sequencers) = create_or_get_default_records(1, txn.deref_mut())
let (kafka, query, _shards) = create_or_get_default_records(1, txn.deref_mut())
.await
.expect("db init failed");
txn.commit().await.expect("txn commit");

View File

@ -7,7 +7,7 @@ use arrow::{
use data_types::{
Column, ColumnSet, ColumnType, CompactionLevel, KafkaPartition, KafkaTopic, Namespace,
NamespaceSchema, ParquetFile, ParquetFileParams, Partition, PartitionId, QueryPool,
SequenceNumber, Sequencer, SequencerId, Table, TableId, TableSchema, Timestamp, Tombstone,
SequenceNumber, Shard, ShardId, Table, TableId, TableSchema, Timestamp, Tombstone,
TombstoneId,
};
use datafusion::physical_plan::metrics::Count;
@ -105,8 +105,8 @@ impl TestCatalog {
Arc::clone(&self.exec)
}
/// Create a sequencer in the catalog
pub async fn create_sequencer(self: &Arc<Self>, sequencer: i32) -> Arc<Sequencer> {
/// Create a shard in the catalog
pub async fn create_shard(self: &Arc<Self>, shard: i32) -> Arc<Shard> {
let mut repos = self.catalog.repositories().await;
let kafka_topic = repos
@ -114,10 +114,10 @@ impl TestCatalog {
.create_or_get("kafka_topic")
.await
.unwrap();
let kafka_partition = KafkaPartition::new(sequencer);
let kafka_partition = KafkaPartition::new(shard);
Arc::new(
repos
.sequencers()
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap(),
@ -186,25 +186,25 @@ impl TestCatalog {
/// List level 0 files
pub async fn list_level_0_files(
self: &Arc<Self>,
sequencer_id: SequencerId,
shard_id: ShardId,
) -> Vec<ParquetFile> {
self.catalog
.repositories()
.await
.parquet_files()
.level_0(sequencer_id)
.level_0(shard_id)
.await
.unwrap()
}
/// Count level 0 files
pub async fn count_level_0_files(self: &Arc<Self>, sequencer_id: SequencerId) -> usize {
pub async fn count_level_0_files(self: &Arc<Self>, shard_id: ShardId) -> usize {
let level_0 = self
.catalog
.repositories()
.await
.parquet_files()
.level_0(sequencer_id)
.level_0(shard_id)
.await
.unwrap();
level_0.len()
@ -257,8 +257,8 @@ impl TestNamespace {
pub async fn create_sequencer(self: &Arc<Self>, sequencer: i32) -> Arc<TestSequencer> {
let mut repos = self.catalog.catalog.repositories().await;
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.create_or_get(&self.kafka_topic, KafkaPartition::new(sequencer))
.await
.unwrap();
@ -266,7 +266,7 @@ impl TestNamespace {
Arc::new(TestSequencer {
catalog: Arc::clone(&self.catalog),
namespace: Arc::clone(self),
sequencer,
shard,
})
}
@ -285,7 +285,7 @@ impl TestNamespace {
pub struct TestSequencer {
pub catalog: Arc<TestCatalog>,
pub namespace: Arc<TestNamespace>,
pub sequencer: Sequencer,
pub shard: Shard,
}
/// A test table of a namespace in the catalog
@ -376,7 +376,7 @@ impl TestTableBoundSequencer {
let partition = repos
.partitions()
.create_or_get(key.into(), self.sequencer.sequencer.id, self.table.table.id)
.create_or_get(key.into(), self.sequencer.shard.id, self.table.table.id)
.await
.unwrap();
@ -399,7 +399,7 @@ impl TestTableBoundSequencer {
let partition = repos
.partitions()
.create_or_get(key.into(), self.sequencer.sequencer.id, self.table.table.id)
.create_or_get(key.into(), self.sequencer.shard.id, self.table.table.id)
.await
.unwrap();
@ -432,7 +432,7 @@ impl TestTableBoundSequencer {
.tombstones()
.create_or_get(
self.table.table.id,
self.sequencer.sequencer.id,
self.sequencer.shard.id,
SequenceNumber::new(sequence_number),
Timestamp::new(min_time),
Timestamp::new(max_time),
@ -524,7 +524,7 @@ impl TestPartition {
creation_timestamp: now(),
namespace_id: self.namespace.namespace.id,
namespace_name: self.namespace.namespace.name.clone().into(),
sequencer_id: self.sequencer.sequencer.id,
shard_id: self.sequencer.shard.id,
table_id: self.table.table.id,
table_name: self.table.table.name.clone().into(),
partition_id: self.partition.id,
@ -549,7 +549,7 @@ impl TestPartition {
.await;
let parquet_file_params = ParquetFileParams {
sequencer_id: self.sequencer.sequencer.id,
shard_id: self.sequencer.shard.id,
namespace_id: self.namespace.namespace.id,
table_id: self.table.table.id,
partition_id: self.partition.id,

View File

@ -177,10 +177,10 @@ pub async fn build_compactor_from_config(
.map(KafkaPartition::new)
.collect();
let mut sequencers = Vec::with_capacity(kafka_partitions.len());
let mut shards = Vec::with_capacity(kafka_partitions.len());
for k in kafka_partitions {
let s = txn.sequencers().create_or_get(&kafka_topic, k).await?;
sequencers.push(s.id);
let s = txn.shards().create_or_get(&kafka_topic, k).await?;
shards.push(s.id);
}
txn.commit().await?;
@ -202,7 +202,7 @@ pub async fn build_compactor_from_config(
);
Ok(compactor::compact::Compactor::new(
sequencers,
shards,
catalog,
parquet_store,
exec,

View File

@ -163,10 +163,10 @@ pub async fn create_ingester_server_type(
.map(KafkaPartition::new)
.collect();
let mut sequencers = BTreeMap::new();
let mut shards = BTreeMap::new();
for k in kafka_partitions {
let s = txn.sequencers().create_or_get(&kafka_topic, k).await?;
sequencers.insert(k, s);
let s = txn.shards().create_or_get(&kafka_topic, k).await?;
shards.insert(k, s);
}
txn.commit().await?;
@ -187,7 +187,7 @@ pub async fn create_ingester_server_type(
IngestHandlerImpl::new(
lifecycle_config,
kafka_topic,
sequencers,
shards,
catalog,
object_store,
write_buffer,

View File

@ -64,8 +64,8 @@ mod tests {
async fn test_get_namespaces_empty() {
let catalog = TestCatalog::new();
// QuerierDatabase::new returns an error if there are no sequencers in the catalog
catalog.create_sequencer(0).await;
// QuerierDatabase::new returns an error if there are no shards in the catalog
catalog.create_shard(0).await;
let catalog_cache = Arc::new(QuerierCatalogCache::new_testing(
catalog.catalog(),
@ -100,8 +100,8 @@ mod tests {
async fn test_get_namespaces() {
let catalog = TestCatalog::new();
// QuerierDatabase::new returns an error if there are no sequencers in the catalog
catalog.create_sequencer(0).await;
// QuerierDatabase::new returns an error if there are no shards in the catalog
catalog.create_shard(0).await;
let catalog_cache = Arc::new(QuerierCatalogCache::new_testing(
catalog.catalog(),

View File

@ -20,7 +20,7 @@ pub mod metadata;
pub mod serialize;
pub mod storage;
use data_types::{NamespaceId, ParquetFile, PartitionId, SequencerId, TableId};
use data_types::{NamespaceId, ParquetFile, PartitionId, ShardId, TableId};
use object_store::path::Path;
use uuid::Uuid;
@ -30,7 +30,7 @@ use uuid::Uuid;
pub struct ParquetFilePath {
namespace_id: NamespaceId,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
partition_id: PartitionId,
object_store_id: Uuid,
}
@ -40,14 +40,14 @@ impl ParquetFilePath {
pub fn new(
namespace_id: NamespaceId,
table_id: TableId,
sequencer_id: SequencerId,
shard_id: ShardId,
partition_id: PartitionId,
object_store_id: Uuid,
) -> Self {
Self {
namespace_id,
table_id,
sequencer_id,
shard_id,
partition_id,
object_store_id,
}
@ -58,7 +58,7 @@ impl ParquetFilePath {
let Self {
namespace_id,
table_id,
sequencer_id,
shard_id,
partition_id,
object_store_id,
} = self;
@ -66,7 +66,7 @@ impl ParquetFilePath {
Path::from_iter([
namespace_id.to_string().as_str(),
table_id.to_string().as_str(),
sequencer_id.to_string().as_str(),
shard_id.to_string().as_str(),
partition_id.to_string().as_str(),
&format!("{}.parquet", object_store_id),
])
@ -84,7 +84,7 @@ impl From<&crate::metadata::IoxMetadata> for ParquetFilePath {
Self {
namespace_id: m.namespace_id,
table_id: m.table_id,
sequencer_id: m.sequencer_id,
shard_id: m.shard_id,
partition_id: m.partition_id,
object_store_id: m.object_store_id,
}
@ -96,7 +96,7 @@ impl From<&ParquetFile> for ParquetFilePath {
Self {
namespace_id: f.namespace_id,
table_id: f.table_id,
sequencer_id: f.sequencer_id,
shard_id: f.shard_id,
partition_id: f.partition_id,
object_store_id: f.object_store_id,
}
@ -112,7 +112,7 @@ mod tests {
let pfp = ParquetFilePath::new(
NamespaceId::new(1),
TableId::new(2),
SequencerId::new(3),
ShardId::new(3),
PartitionId::new(4),
Uuid::nil(),
);

View File

@ -89,7 +89,7 @@
use bytes::Bytes;
use data_types::{
ColumnId, ColumnSet, ColumnSummary, CompactionLevel, InfluxDbType, NamespaceId,
ParquetFileParams, PartitionId, PartitionKey, SequenceNumber, SequencerId, StatValues,
ParquetFileParams, PartitionId, PartitionKey, SequenceNumber, ShardId, StatValues,
Statistics, TableId, Timestamp,
};
use generated_types::influxdata::iox::ingester::v1 as proto;
@ -259,8 +259,8 @@ pub struct IoxMetadata {
/// namespace name of the data
pub namespace_name: Arc<str>,
/// sequencer id of the data
pub sequencer_id: SequencerId,
/// shard id of the data
pub shard_id: ShardId,
/// table id of the data
pub table_id: TableId,
@ -315,7 +315,7 @@ impl IoxMetadata {
creation_timestamp: Some(self.creation_timestamp.date_time().into()),
namespace_id: self.namespace_id.get(),
namespace_name: self.namespace_name.to_string(),
sequencer_id: self.sequencer_id.get(),
sequencer_id: self.shard_id.get(), // XXXJPG
table_id: self.table_id.get(),
table_name: self.table_name.to_string(),
partition_id: self.partition_id.get(),
@ -365,7 +365,7 @@ impl IoxMetadata {
creation_timestamp,
namespace_id: NamespaceId::new(proto_msg.namespace_id),
namespace_name,
sequencer_id: SequencerId::new(proto_msg.sequencer_id),
shard_id: ShardId::new(proto_msg.sequencer_id), // XXXJPG
table_id: TableId::new(proto_msg.table_id),
table_name,
partition_id: PartitionId::new(proto_msg.partition_id),
@ -451,7 +451,7 @@ impl IoxMetadata {
};
ParquetFileParams {
sequencer_id: self.sequencer_id,
shard_id: self.shard_id,
namespace_id: self.namespace_id,
table_id: self.table_id,
partition_id: self.partition_id,
@ -976,7 +976,7 @@ mod tests {
creation_timestamp: Time::from_timestamp(3234, 0),
namespace_id: NamespaceId::new(2),
namespace_name: Arc::from("hi"),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
table_id: TableId::new(3),
table_name: Arc::from("weather"),
partition_id: PartitionId::new(4),
@ -1000,7 +1000,7 @@ mod tests {
creation_timestamp: Time::from_timestamp_nanos(42),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id: PartitionId::new(4),

View File

@ -167,7 +167,7 @@ mod tests {
use arrow::array::{ArrayRef, StringArray};
use bytes::Bytes;
use data_types::{
CompactionLevel, NamespaceId, PartitionId, SequenceNumber, SequencerId, TableId,
CompactionLevel, NamespaceId, PartitionId, SequenceNumber, ShardId, TableId,
};
use iox_time::Time;
use parquet::{
@ -183,7 +183,7 @@ mod tests {
creation_timestamp: Time::from_timestamp_nanos(42),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id: PartitionId::new(4),

View File

@ -406,7 +406,7 @@ mod tests {
use super::*;
use arrow::array::{ArrayRef, Int64Array, StringArray};
use data_types::{
CompactionLevel, NamespaceId, PartitionId, SequenceNumber, SequencerId, TableId,
CompactionLevel, NamespaceId, PartitionId, SequenceNumber, ShardId, TableId,
};
use datafusion::common::DataFusionError;
use iox_time::Time;
@ -760,7 +760,7 @@ mod tests {
creation_timestamp: Time::from_timestamp_nanos(42),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id: PartitionId::new(4),

View File

@ -5,7 +5,7 @@ use arrow::{
record_batch::RecordBatch,
};
use data_types::{
ColumnId, CompactionLevel, NamespaceId, PartitionId, SequenceNumber, SequencerId, TableId,
ColumnId, CompactionLevel, NamespaceId, PartitionId, SequenceNumber, ShardId, TableId,
Timestamp,
};
use iox_time::Time;
@ -41,7 +41,7 @@ async fn test_decoded_iox_metadata() {
creation_timestamp: Time::from_timestamp_nanos(42),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id: PartitionId::new(4),
@ -133,7 +133,7 @@ async fn test_empty_parquet_file_panic() {
creation_timestamp: Time::from_timestamp_nanos(42),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id: PartitionId::new(4),
@ -210,7 +210,7 @@ async fn test_decoded_many_columns_with_null_cols_iox_metadata() {
creation_timestamp: Time::from_timestamp_nanos(42),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id: PartitionId::new(4),
@ -286,7 +286,7 @@ async fn test_derive_parquet_file_params() {
creation_timestamp: Time::from_timestamp_nanos(1234),
namespace_id: NamespaceId::new(1),
namespace_name: "bananas".into(),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
table_id: TableId::new(3),
table_name: "platanos".into(),
partition_id,
@ -330,7 +330,7 @@ async fn test_derive_parquet_file_params() {
//
// NOTE: thrift-encoded metadata not checked
// TODO: check thrift-encoded metadata which may be the issue of bug 4695
assert_eq!(catalog_data.sequencer_id, meta.sequencer_id);
assert_eq!(catalog_data.shard_id, meta.shard_id);
assert_eq!(catalog_data.namespace_id, meta.namespace_id);
assert_eq!(catalog_data.table_id, meta.table_id);
assert_eq!(catalog_data.partition_id, meta.partition_id);

View File

@ -11,7 +11,7 @@ use cache_system::{
loader::{metrics::MetricsLoader, FunctionLoader},
resource_consumption::FunctionEstimator,
};
use data_types::{PartitionId, SequencerId};
use data_types::{PartitionId, ShardId};
use iox_catalog::interface::Catalog;
use iox_time::TimeProvider;
use schema::sort::SortKey;
@ -68,7 +68,7 @@ impl PartitionCache {
.expect("partition gone from catalog?!");
CachedPartition {
sequencer_id: partition.sequencer_id,
shard_id: partition.shard_id,
sort_key: Arc::new(partition.sort_key()),
}
}
@ -108,9 +108,9 @@ impl PartitionCache {
}
}
/// Get sequencer ID.
pub async fn sequencer_id(&self, partition_id: PartitionId, span: Option<Span>) -> SequencerId {
self.cache.get(partition_id, ((), span)).await.sequencer_id
/// Get shard ID.
pub async fn shard_id(&self, partition_id: PartitionId, span: Option<Span>) -> ShardId {
self.cache.get(partition_id, ((), span)).await.shard_id
}
/// Get sort key
@ -138,7 +138,7 @@ impl PartitionCache {
#[derive(Debug, Clone)]
struct CachedPartition {
sequencer_id: SequencerId,
shard_id: ShardId,
sort_key: Arc<Option<SortKey>>,
}
@ -163,7 +163,7 @@ mod tests {
use iox_tests::util::TestCatalog;
#[tokio::test]
async fn test_sequencer_id() {
async fn test_shard_id() {
let catalog = TestCatalog::new();
let ns = catalog.create_namespace("ns").await;
@ -192,16 +192,16 @@ mod tests {
true,
);
let id1 = cache.sequencer_id(p1.id, None).await;
assert_eq!(id1, s1.sequencer.id);
let id1 = cache.shard_id(p1.id, None).await;
assert_eq!(id1, s1.shard.id);
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 1);
let id2 = cache.sequencer_id(p2.id, None).await;
assert_eq!(id2, s2.sequencer.id);
let id2 = cache.shard_id(p2.id, None).await;
assert_eq!(id2, s2.shard.id);
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 2);
let id1 = cache.sequencer_id(p1.id, None).await;
assert_eq!(id1, s1.sequencer.id);
let id1 = cache.shard_id(p1.id, None).await;
assert_eq!(id1, s1.shard.id);
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 2);
}
@ -284,16 +284,16 @@ mod tests {
true,
);
cache.sequencer_id(p2.id, None).await;
cache.shard_id(p2.id, None).await;
cache.sort_key(p3.id, &Vec::new(), None).await;
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 2);
cache.sequencer_id(p1.id, None).await;
cache.shard_id(p1.id, None).await;
cache.sort_key(p2.id, &Vec::new(), None).await;
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 3);
cache.sort_key(p1.id, &Vec::new(), None).await;
cache.sequencer_id(p2.id, None).await;
cache.shard_id(p2.id, None).await;
assert_histogram_metric_count(&catalog.metric_registry, "partition_get_by_id", 3);
}

View File

@ -2,8 +2,8 @@
use crate::cache::CatalogCache;
use data_types::{
ChunkId, ChunkOrder, CompactionLevel, DeletePredicate, NamespaceSchema, ParquetFile,
ParquetFileId, PartitionId, SequenceNumber, SequencerId, TableSummary, TimestampMinMax,
ChunkId, ChunkOrder, ColumnId, CompactionLevel, DeletePredicate, ParquetFile, ParquetFileId,
PartitionId, SequenceNumber, ShardId, TableSummary, TimestampMinMax,
};
use iox_catalog::interface::Catalog;
use parking_lot::RwLock;
@ -38,8 +38,8 @@ pub struct ChunkMeta {
/// Sort key.
sort_key: Option<SortKey>,
/// Sequencer that created the data within this chunk.
sequencer_id: SequencerId,
/// Shard that created the data within this chunk.
shard_id: ShardId,
/// Partition ID.
partition_id: PartitionId,
@ -67,9 +67,9 @@ impl ChunkMeta {
self.sort_key.as_ref()
}
/// Sequencer that created the data within this chunk.
pub fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
/// Shard that created the data within this chunk.
pub fn shard_id(&self) -> ShardId {
self.shard_id
}
/// Partition ID.
@ -524,7 +524,7 @@ impl ChunkAdapter {
table_name,
order,
sort_key: Some(sort_key),
sequencer_id: parquet_file.sequencer_id,
shard_id: parquet_file.shard_id,
partition_id: parquet_file.partition_id,
max_sequence_number: parquet_file.max_sequence_number,
compaction_level: parquet_file.compaction_level,

View File

@ -213,22 +213,22 @@ pub async fn create_sharder(
catalog: &dyn Catalog,
backoff_config: BackoffConfig,
) -> Result<JumpHash<Arc<KafkaPartition>>, Error> {
let sequencers = Backoff::new(&backoff_config)
.retry_all_errors("get sequencers", || async {
catalog.repositories().await.sequencers().list().await
let shards = Backoff::new(&backoff_config)
.retry_all_errors("get shards", || async {
catalog.repositories().await.shards().list().await
})
.await
.expect("retry forever");
// Construct the (ordered) set of sequencers.
// Construct the (ordered) set of shards.
//
// The sort order must be deterministic in order for all nodes to shard to
// the same sequencers, therefore we type assert the returned set is of the
// The sort order must be deterministic in order for all nodes to shard to // XXXJPG
// the same shards, therefore we type assert the returned set is of the
// ordered variety.
let shards: BTreeSet<_> = sequencers
let shards: BTreeSet<_> = shards
// ^ don't change this to an unordered set
.into_iter()
.map(|sequencer| sequencer.kafka_partition)
.map(|shard| shard.kafka_partition)
.collect();
if shards.is_empty() {
@ -273,7 +273,7 @@ mod tests {
}
#[tokio::test]
async fn sequencers_in_catalog_are_required_for_startup() {
async fn shards_in_catalog_are_required_for_startup() {
let catalog = TestCatalog::new();
let catalog_cache = Arc::new(CatalogCache::new_testing(
@ -301,8 +301,8 @@ mod tests {
#[tokio::test]
async fn test_namespace() {
let catalog = TestCatalog::new();
// QuerierDatabase::new returns an error if there are no sequencers in the catalog
catalog.create_sequencer(0).await;
// QuerierDatabase::new returns an error if there are no shards in the catalog
catalog.create_shard(0).await;
let catalog_cache = Arc::new(CatalogCache::new_testing(
catalog.catalog(),
@ -331,8 +331,8 @@ mod tests {
#[tokio::test]
async fn test_namespaces() {
let catalog = TestCatalog::new();
// QuerierDatabase::new returns an error if there are no sequencers in the catalog
catalog.create_sequencer(0).await;
// QuerierDatabase::new returns an error if there are no shards in the catalog
catalog.create_shard(0).await;
let catalog_cache = Arc::new(CatalogCache::new_testing(
catalog.catalog(),

View File

@ -173,7 +173,7 @@ mod tests {
Arc::clone(&metric_registry),
&Handle::current(),
));
// QuerierDatabase::new returns an error if there are no sequencers in the catalog
// QuerierDatabase::new returns an error if there are no shards in the catalog
{
let mut repos = catalog.repositories().await;
@ -184,7 +184,7 @@ mod tests {
.unwrap();
let kafka_partition = KafkaPartition::new(0);
repos
.sequencers()
.shards()
.create_or_get(&kafka_topic, kafka_partition)
.await
.unwrap();

View File

@ -7,7 +7,7 @@ use arrow::{datatypes::DataType, error::ArrowError, record_batch::RecordBatch};
use async_trait::async_trait;
use client_util::connection;
use data_types::{
ChunkId, ChunkOrder, IngesterMapping, KafkaPartition, PartitionId, SequenceNumber, SequencerId,
ChunkId, ChunkOrder, IngesterMapping, KafkaPartition, PartitionId, SequenceNumber, ShardId,
TableSummary, TimestampMinMax,
};
use datafusion_util::MemoryStream;
@ -570,13 +570,13 @@ impl IngesterStreamDecoder {
ingester_address: self.ingester_address.as_ref()
},
);
let sequencer_id = self
let shard_id = self
.catalog_cache
.partition()
.sequencer_id(
.shard_id(
partition_id,
self.span_recorder
.child_span("cache GET partition sequencer ID"),
.child_span("cache GET partition shard ID"),
)
.await;
@ -588,7 +588,7 @@ impl IngesterStreamDecoder {
Arc::clone(&self.ingester_address),
Arc::clone(&self.table_name),
partition_id,
sequencer_id,
shard_id,
status.parquet_max_sequence_number.map(SequenceNumber::new),
status
.tombstone_max_sequence_number
@ -807,18 +807,18 @@ async fn execute_get_write_infos(
/// Given the catalog hierarchy:
///
/// ```text
/// (Catalog) Sequencer -> (Catalog) Table --> (Catalog) Partition
/// (Catalog) Shard -> (Catalog) Table --> (Catalog) Partition
/// ```
///
/// An IngesterPartition contains the unpersisted data for a catalog
/// partition from a sequencer. Thus, there can be more than one
/// partition from a shard. Thus, there can be more than one
/// IngesterPartition for each table the ingester knows about.
#[derive(Debug, Clone)]
pub struct IngesterPartition {
ingester: Arc<str>,
table_name: Arc<str>,
partition_id: PartitionId,
sequencer_id: SequencerId,
shard_id: ShardId,
/// Maximum sequence number of parquet files the ingester has
/// persisted for this partition
@ -841,7 +841,7 @@ impl IngesterPartition {
ingester: Arc<str>,
table_name: Arc<str>,
partition_id: PartitionId,
sequencer_id: SequencerId,
shard_id: ShardId,
parquet_max_sequence_number: Option<SequenceNumber>,
tombstone_max_sequence_number: Option<SequenceNumber>,
partition_sort_key: Arc<Option<SortKey>>,
@ -850,7 +850,7 @@ impl IngesterPartition {
ingester,
table_name,
partition_id,
sequencer_id,
shard_id,
parquet_max_sequence_number,
tombstone_max_sequence_number,
partition_sort_key,
@ -928,8 +928,8 @@ impl IngesterPartition {
self.partition_id
}
pub(crate) fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
pub(crate) fn shard_id(&self) -> ShardId {
self.shard_id
}
pub(crate) fn parquet_max_sequence_number(&self) -> Option<SequenceNumber> {
@ -1297,7 +1297,7 @@ mod tests {
let p = &partitions[0];
assert_eq!(p.partition_id.get(), 1);
assert_eq!(p.sequencer_id.get(), 1);
assert_eq!(p.shard_id.get(), 1);
assert_eq!(p.parquet_max_sequence_number, None);
assert_eq!(p.tombstone_max_sequence_number, None);
assert_eq!(p.chunks.len(), 0);
@ -1518,7 +1518,7 @@ mod tests {
let p1 = &partitions[0];
assert_eq!(p1.partition_id.get(), 1);
assert_eq!(p1.sequencer_id.get(), 1);
assert_eq!(p1.shard_id.get(), 1);
assert_eq!(
p1.parquet_max_sequence_number,
Some(SequenceNumber::new(11))
@ -1538,7 +1538,7 @@ mod tests {
let p2 = &partitions[1];
assert_eq!(p2.partition_id.get(), 2);
assert_eq!(p2.sequencer_id.get(), 1);
assert_eq!(p2.shard_id.get(), 1);
assert_eq!(
p2.parquet_max_sequence_number,
Some(SequenceNumber::new(21))
@ -1554,7 +1554,7 @@ mod tests {
let p3 = &partitions[2];
assert_eq!(p3.partition_id.get(), 3);
assert_eq!(p3.sequencer_id.get(), 2);
assert_eq!(p3.shard_id.get(), 2);
assert_eq!(
p3.parquet_max_sequence_number,
Some(SequenceNumber::new(31))
@ -1717,7 +1717,7 @@ mod tests {
let p1 = &partitions[0];
assert_eq!(p1.partition_id.get(), 1);
assert_eq!(p1.sequencer_id.get(), 1);
assert_eq!(p1.shard_id.get(), 1);
assert_eq!(
p1.parquet_max_sequence_number,
Some(SequenceNumber::new(11))
@ -1894,7 +1894,7 @@ mod tests {
"ingester".into(),
"table".into(),
PartitionId::new(1),
SequencerId::new(1),
ShardId::new(1),
parquet_max_sequence_number,
tombstone_max_sequence_number,
Arc::new(None),
@ -1927,7 +1927,7 @@ mod tests {
"ingester".into(),
"table".into(),
PartitionId::new(1),
SequencerId::new(1),
ShardId::new(1),
parquet_max_sequence_number,
tombstone_max_sequence_number,
Arc::new(None),

View File

@ -2,7 +2,7 @@
mod interface;
use data_types::{CompactionLevel, PartitionId, SequencerId, Tombstone, TombstoneId};
use data_types::{CompactionLevel, PartitionId, ShardId, Tombstone, TombstoneId};
use iox_query::QueryChunk;
use observability_deps::tracing::debug;
use schema::sort::SortKey;
@ -104,12 +104,12 @@ impl Reconciler {
tombstones.into_iter().map(QuerierTombstone::from).collect();
// match chunks and tombstones
let mut tombstones_by_sequencer: HashMap<SequencerId, Vec<QuerierTombstone>> =
let mut tombstones_by_shard: HashMap<ShardId, Vec<QuerierTombstone>> =
HashMap::new();
for tombstone in querier_tombstones {
tombstones_by_sequencer
.entry(tombstone.sequencer_id())
tombstones_by_shard
.entry(tombstone.shard_id())
.or_default()
.push(tombstone);
}
@ -130,7 +130,7 @@ impl Reconciler {
for chunk in parquet_files.into_iter() {
let chunk = if let Some(tombstones) =
tombstones_by_sequencer.get(&chunk.meta().sequencer_id())
tombstones_by_shard.get(&chunk.meta().shard_id())
{
let mut delete_predicates = Vec::with_capacity(tombstones.len());
for tombstone in tombstones {
@ -333,8 +333,8 @@ where
{
// Build partition-based lookup table.
//
// Note that we don't need to take the sequencer ID into account here because each partition is
// not only bound to a table but also to a sequencer.
// Note that we don't need to take the shard ID into account here because each partition is
// not only bound to a table but also to a shard.
let lookup_table: HashMap<PartitionId, &I> = ingester_partitions
.iter()
.map(|i| (i.partition_id(), i))
@ -392,8 +392,8 @@ where
/// Generates "exclude" filter for tombstones.
///
/// Since tombstones are sequencer-wide but data persistence is partition-based (which are
/// sub-units of sequencers), we cannot just remove tombstones entirely but need to decide on a
/// Since tombstones are shard-wide but data persistence is partition-based (which are
/// sub-units of shards), we cannot just remove tombstones entirely but need to decide on a
/// per-partition basis. This function generates a lookup table of partition-tombstone tuples that
/// later need to be EXCLUDED/IGNORED when pairing tombstones with chunks.
fn tombstone_exclude_list<I, T>(
@ -404,18 +404,18 @@ where
I: IngesterPartitionInfo,
T: TombstoneInfo,
{
// Build sequencer-based lookup table.
let mut lookup_table: HashMap<SequencerId, Vec<&I>> = HashMap::default();
// Build shard-based lookup table.
let mut lookup_table: HashMap<ShardId, Vec<&I>> = HashMap::default();
for partition in ingester_partitions {
lookup_table
.entry(partition.sequencer_id())
.entry(partition.shard_id())
.or_default()
.push(partition);
}
let mut exclude = HashSet::new();
for t in tombstones {
if let Some(partitions) = lookup_table.get(&t.sequencer_id()) {
if let Some(partitions) = lookup_table.get(&t.shard_id()) {
for p in partitions {
if let Some(persisted_max) = p.tombstone_max_sequence_number() {
if t.sequence_number() > persisted_max {
@ -454,7 +454,7 @@ mod tests {
fn test_filter_parquet_files_compactor_conflict() {
let ingester_partitions = &[MockIngesterPartitionInfo {
partition_id: PartitionId::new(1),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: Some(SequenceNumber::new(10)),
tombstone_max_sequence_number: None,
}];
@ -472,19 +472,19 @@ mod tests {
let ingester_partitions = &[
MockIngesterPartitionInfo {
partition_id: PartitionId::new(1),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: Some(SequenceNumber::new(10)),
tombstone_max_sequence_number: None,
},
MockIngesterPartitionInfo {
partition_id: PartitionId::new(2),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: None,
tombstone_max_sequence_number: None,
},
MockIngesterPartitionInfo {
partition_id: PartitionId::new(3),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: Some(SequenceNumber::new(3)),
tombstone_max_sequence_number: None,
},
@ -553,25 +553,25 @@ mod tests {
let ingester_partitions = &[
MockIngesterPartitionInfo {
partition_id: PartitionId::new(1),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: None,
tombstone_max_sequence_number: Some(SequenceNumber::new(10)),
},
MockIngesterPartitionInfo {
partition_id: PartitionId::new(2),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: None,
tombstone_max_sequence_number: None,
},
MockIngesterPartitionInfo {
partition_id: PartitionId::new(3),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
parquet_max_sequence_number: None,
tombstone_max_sequence_number: Some(SequenceNumber::new(3)),
},
MockIngesterPartitionInfo {
partition_id: PartitionId::new(4),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
parquet_max_sequence_number: None,
tombstone_max_sequence_number: Some(SequenceNumber::new(7)),
},
@ -579,52 +579,52 @@ mod tests {
let tombstones = &[
MockTombstoneInfo {
id: TombstoneId::new(1),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
sequence_number: SequenceNumber::new(2),
},
MockTombstoneInfo {
id: TombstoneId::new(2),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
sequence_number: SequenceNumber::new(3),
},
MockTombstoneInfo {
id: TombstoneId::new(3),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
sequence_number: SequenceNumber::new(4),
},
MockTombstoneInfo {
id: TombstoneId::new(4),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
sequence_number: SequenceNumber::new(9),
},
MockTombstoneInfo {
id: TombstoneId::new(5),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
sequence_number: SequenceNumber::new(10),
},
MockTombstoneInfo {
id: TombstoneId::new(6),
sequencer_id: SequencerId::new(1),
shard_id: ShardId::new(1),
sequence_number: SequenceNumber::new(11),
},
MockTombstoneInfo {
id: TombstoneId::new(7),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
sequence_number: SequenceNumber::new(6),
},
MockTombstoneInfo {
id: TombstoneId::new(8),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
sequence_number: SequenceNumber::new(7),
},
MockTombstoneInfo {
id: TombstoneId::new(9),
sequencer_id: SequencerId::new(2),
shard_id: ShardId::new(2),
sequence_number: SequenceNumber::new(8),
},
MockTombstoneInfo {
id: TombstoneId::new(10),
sequencer_id: SequencerId::new(3),
shard_id: ShardId::new(3),
sequence_number: SequenceNumber::new(10),
},
];
@ -650,7 +650,7 @@ mod tests {
#[derive(Debug)]
struct MockIngesterPartitionInfo {
partition_id: PartitionId,
sequencer_id: SequencerId,
shard_id: ShardId,
parquet_max_sequence_number: Option<SequenceNumber>,
tombstone_max_sequence_number: Option<SequenceNumber>,
}
@ -660,8 +660,8 @@ mod tests {
self.partition_id
}
fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
fn shard_id(&self) -> ShardId {
self.shard_id
}
fn parquet_max_sequence_number(&self) -> Option<SequenceNumber> {
@ -697,7 +697,7 @@ mod tests {
#[derive(Debug)]
struct MockTombstoneInfo {
id: TombstoneId,
sequencer_id: SequencerId,
shard_id: ShardId,
sequence_number: SequenceNumber,
}
@ -706,8 +706,8 @@ mod tests {
self.id
}
fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
fn shard_id(&self) -> ShardId {
self.shard_id
}
fn sequence_number(&self) -> SequenceNumber {

View File

@ -2,7 +2,7 @@
use crate::{chunk::QuerierChunk, ingester::IngesterPartition};
use data_types::{
CompactionLevel, ParquetFile, PartitionId, SequenceNumber, SequencerId, Tombstone, TombstoneId,
CompactionLevel, ParquetFile, PartitionId, SequenceNumber, ShardId, Tombstone, TombstoneId,
};
use std::{ops::Deref, sync::Arc};
@ -11,7 +11,7 @@ use std::{ops::Deref, sync::Arc};
/// This is mostly the same as [`IngesterPartition`] but allows easier mocking.
pub trait IngesterPartitionInfo {
fn partition_id(&self) -> PartitionId;
fn sequencer_id(&self) -> SequencerId;
fn shard_id(&self) -> ShardId;
fn parquet_max_sequence_number(&self) -> Option<SequenceNumber>;
fn tombstone_max_sequence_number(&self) -> Option<SequenceNumber>;
}
@ -21,8 +21,8 @@ impl IngesterPartitionInfo for IngesterPartition {
self.deref().partition_id()
}
fn sequencer_id(&self) -> SequencerId {
self.deref().sequencer_id()
fn shard_id(&self) -> ShardId {
self.deref().shard_id()
}
fn parquet_max_sequence_number(&self) -> Option<SequenceNumber> {
@ -42,8 +42,8 @@ where
self.deref().partition_id()
}
fn sequencer_id(&self) -> SequencerId {
self.deref().sequencer_id()
fn shard_id(&self) -> ShardId {
self.deref().shard_id()
}
fn parquet_max_sequence_number(&self) -> Option<SequenceNumber> {
@ -97,7 +97,7 @@ impl ParquetFileInfo for QuerierChunk {
/// This is mostly the same as [`Tombstone`] but allows easier mocking.
pub trait TombstoneInfo {
fn id(&self) -> TombstoneId;
fn sequencer_id(&self) -> SequencerId;
fn shard_id(&self) -> ShardId;
fn sequence_number(&self) -> SequenceNumber;
}
@ -106,8 +106,8 @@ impl TombstoneInfo for Tombstone {
self.id
}
fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
fn shard_id(&self) -> ShardId {
self.shard_id
}
fn sequence_number(&self) -> SequenceNumber {
@ -120,8 +120,8 @@ impl TombstoneInfo for Arc<Tombstone> {
self.id
}
fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
fn shard_id(&self) -> ShardId {
self.shard_id
}
fn sequence_number(&self) -> SequenceNumber {

View File

@ -131,7 +131,7 @@ impl IngesterPartitionBuilder {
Arc::clone(&self.ingester_name),
Arc::from(self.table.table.name.as_str()),
self.partition.partition.id,
self.sequencer.sequencer.id,
self.sequencer.shard.id,
parquet_max_sequence_number,
tombstone_max_sequence_number,
Arc::clone(&self.partition_sort_key),

View File

@ -1,4 +1,4 @@
use data_types::{DeletePredicate, SequenceNumber, SequencerId, Tombstone, TombstoneId};
use data_types::{DeletePredicate, SequenceNumber, ShardId, Tombstone, TombstoneId};
use predicate::delete_predicate::parse_delete_predicate;
use std::sync::Arc;
@ -8,8 +8,8 @@ pub struct QuerierTombstone {
/// Delete predicate associated with this tombstone.
delete_predicate: Arc<DeletePredicate>,
/// Sequencer that this tombstone affects.
sequencer_id: SequencerId,
/// Shard that this tombstone affects.
shard_id: ShardId,
/// The sequence number assigned to the tombstone from the sequencer.
sequence_number: SequenceNumber,
@ -24,12 +24,12 @@ impl QuerierTombstone {
&self.delete_predicate
}
/// Sequencer that this tombstone affects.
pub fn sequencer_id(&self) -> SequencerId {
self.sequencer_id
/// Shard that this tombstone affects.
pub fn shard_id(&self) -> ShardId {
self.shard_id
}
/// The sequence number assigned to the tombstone from the sequencer.
/// The sequence number assigned to the tombstone from the shard.
pub fn sequence_number(&self) -> SequenceNumber {
self.sequence_number
}
@ -53,7 +53,7 @@ impl From<&Tombstone> for QuerierTombstone {
Self {
delete_predicate,
sequencer_id: tombstone.sequencer_id,
shard_id: tombstone.shard_id,
sequence_number: tombstone.sequence_number,
tombstone_id: tombstone.id,
}

View File

@ -4,7 +4,7 @@ use async_trait::async_trait;
use backoff::BackoffConfig;
use data_types::{
DeletePredicate, IngesterMapping, KafkaPartition, NonEmptyString, ParquetFileId, PartitionId,
PartitionKey, Sequence, SequenceNumber, SequencerId, TombstoneId,
PartitionKey, Sequence, SequenceNumber, ShardId, TombstoneId,
};
use dml::{DmlDelete, DmlMeta, DmlOperation, DmlWrite};
use futures::StreamExt;
@ -15,7 +15,7 @@ use generated_types::{
use influxdb_iox_client::flight::{low_level::LowLevelMessage, Error as FlightError};
use ingester::{
data::{
FlatIngesterQueryResponse, IngesterData, IngesterQueryResponse, Persister, SequencerData,
FlatIngesterQueryResponse, IngesterData, IngesterQueryResponse, Persister, ShardData,
},
lifecycle::LifecycleHandle,
querier_handler::prepare_data_to_querier,
@ -692,17 +692,17 @@ impl MockIngester {
let ns = catalog.create_namespace("test_db").await;
let sequencer = ns.create_sequencer(1).await;
let sequencers = BTreeMap::from([(
sequencer.sequencer.id,
SequencerData::new(
sequencer.sequencer.kafka_partition,
let shards = BTreeMap::from([(
sequencer.shard.id,
ShardData::new(
sequencer.shard.kafka_partition,
catalog.metric_registry(),
),
)]);
let ingester_data = Arc::new(IngesterData::new(
catalog.object_store(),
catalog.catalog(),
sequencers,
shards,
catalog.exec(),
BackoffConfig::default(),
catalog.metric_registry(),
@ -731,7 +731,7 @@ impl MockIngester {
let should_pause = self
.ingester_data
.buffer_operation(
self.sequencer.sequencer.id,
self.sequencer.shard.id,
dml_operation,
&lifecycle_handle,
)
@ -841,7 +841,7 @@ impl MockIngester {
self.partition_keys
.insert(sequence_number, partition_key.to_string());
let meta = DmlMeta::sequenced(
Sequence::new(self.sequencer.sequencer.id.get() as u32, sequence_number),
Sequence::new(self.sequencer.shard.id.get() as u32, sequence_number),
self.catalog.time_provider().now(),
None,
0,
@ -866,7 +866,7 @@ impl MockIngester {
let sequence_number = self.next_sequence_number();
let meta = DmlMeta::sequenced(
Sequence::new(self.sequencer.sequencer.id.get() as u32, sequence_number),
Sequence::new(self.sequencer.shard.id.get() as u32, sequence_number),
self.catalog.time_provider().now(),
None,
0,
@ -971,7 +971,7 @@ impl LifecycleHandle for NoopLifecycleHandle {
fn log_write(
&self,
_partition_id: PartitionId,
_sequencer_id: SequencerId,
_shard_id: ShardId,
_sequence_number: SequenceNumber,
_bytes_written: usize,
) -> bool {

View File

@ -84,7 +84,7 @@ impl catalog_service_server::CatalogService for CatalogService {
fn to_parquet_file(p: data_types::ParquetFile) -> ParquetFile {
ParquetFile {
id: p.id.get(),
sequencer_id: p.sequencer_id.get(),
sequencer_id: p.shard_id.get(), // XXXJPG
namespace_id: p.namespace_id.get(),
table_id: p.table_id.get(),
partition_id: p.partition_id.get(),
@ -105,7 +105,7 @@ fn to_parquet_file(p: data_types::ParquetFile) -> ParquetFile {
fn to_partition(p: data_types::Partition) -> Partition {
Partition {
id: p.id.get(),
sequencer_id: p.sequencer_id.get(),
sequencer_id: p.shard_id.get(),
key: p.partition_key.to_string(),
table_id: p.table_id.get(),
array_sort_key: p.sort_key,
@ -143,8 +143,8 @@ mod tests {
.create_or_get("iox_shared")
.await
.unwrap();
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
@ -160,11 +160,11 @@ mod tests {
.unwrap();
let partition = repos
.partitions()
.create_or_get("foo".into(), sequencer.id, table.id)
.create_or_get("foo".into(), shard.id, table.id)
.await
.unwrap();
let p1params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: table.id,
partition_id: partition.id,
@ -224,8 +224,8 @@ mod tests {
.create_or_get("iox_shared")
.await
.unwrap();
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
@ -241,22 +241,22 @@ mod tests {
.unwrap();
partition1 = repos
.partitions()
.create_or_get("foo".into(), sequencer.id, table.id)
.create_or_get("foo".into(), shard.id, table.id)
.await
.unwrap();
partition2 = repos
.partitions()
.create_or_get("bar".into(), sequencer.id, table.id)
.create_or_get("bar".into(), shard.id, table.id)
.await
.unwrap();
let sequencer2 = repos
.sequencers()
let shard2 = repos
.shards()
.create_or_get(&kafka, KafkaPartition::new(2))
.await
.unwrap();
partition3 = repos
.partitions()
.create_or_get("foo".into(), sequencer2.id, table.id)
.create_or_get("foo".into(), shard2.id, table.id)
.await
.unwrap();

View File

@ -68,7 +68,7 @@ impl object_store_service_server::ObjectStoreService for ObjectStoreService {
let path = ParquetFilePath::new(
parquet_file.namespace_id,
parquet_file.table_id,
parquet_file.sequencer_id,
parquet_file.shard_id,
parquet_file.partition_id,
parquet_file.object_store_id,
);
@ -122,8 +122,8 @@ mod tests {
.create_or_get("iox_shared")
.await
.unwrap();
let sequencer = repos
.sequencers()
let shard = repos
.shards()
.create_or_get(&kafka, KafkaPartition::new(1))
.await
.unwrap();
@ -139,11 +139,11 @@ mod tests {
.unwrap();
let partition = repos
.partitions()
.create_or_get("foo".into(), sequencer.id, table.id)
.create_or_get("foo".into(), shard.id, table.id)
.await
.unwrap();
let p1params = ParquetFileParams {
sequencer_id: sequencer.id,
shard_id: shard.id,
namespace_id: namespace.id,
table_id: table.id,
partition_id: partition.id,
@ -167,7 +167,7 @@ mod tests {
let path = ParquetFilePath::new(
p1.namespace_id,
p1.table_id,
p1.sequencer_id,
p1.shard_id,
p1.partition_id,
p1.object_store_id,
);

View File

@ -7,7 +7,7 @@ use snafu::{OptionExt, Snafu};
use std::collections::BTreeMap;
mod progress;
pub use progress::SequencerProgress;
pub use progress::ShardProgress;
#[derive(Debug, Snafu, PartialEq, Eq)]
pub enum Error {
@ -110,7 +110,7 @@ impl WriteSummary {
pub fn write_status(
&self,
kafka_partition: KafkaPartition,
progress: &SequencerProgress,
progress: &ShardProgress,
) -> Result<KafkaPartitionWriteStatus> {
let sequence_numbers = self
.sequencers
@ -353,7 +353,7 @@ mod tests {
// if we have no info about this partition in the progress
let kafka_partition = KafkaPartition::new(1);
let progress = SequencerProgress::new();
let progress = ShardProgress::new();
assert_eq!(
summary.write_status(kafka_partition, &progress),
Ok(KafkaPartitionWriteStatus::KafkaPartitionUnknown)
@ -365,7 +365,7 @@ mod tests {
let summary = test_summary();
// No information on kafka partition 3
let kafka_partition = KafkaPartition::new(3);
let progress = SequencerProgress::new().with_buffered(SequenceNumber::new(2));
let progress = ShardProgress::new().with_buffered(SequenceNumber::new(2));
let err = summary
.write_status(kafka_partition, &progress)
.unwrap_err();
@ -378,7 +378,7 @@ mod tests {
// kafka partition 1 made it to 3
let kafka_partition = KafkaPartition::new(1);
let progress = SequencerProgress::new().with_buffered(SequenceNumber::new(3));
let progress = ShardProgress::new().with_buffered(SequenceNumber::new(3));
assert_eq!(
summary.write_status(kafka_partition, &progress),
Ok(KafkaPartitionWriteStatus::Readable)
@ -386,7 +386,7 @@ mod tests {
// if kafka partition 1 only made it to 2, but write includes 3
let kafka_partition = KafkaPartition::new(1);
let progress = SequencerProgress::new().with_buffered(SequenceNumber::new(2));
let progress = ShardProgress::new().with_buffered(SequenceNumber::new(2));
assert_eq!(
summary.write_status(kafka_partition, &progress),
Ok(KafkaPartitionWriteStatus::Durable)
@ -394,7 +394,7 @@ mod tests {
// kafka partition 2 made it to 2
let kafka_partition = KafkaPartition::new(2);
let progress = SequencerProgress::new().with_buffered(SequenceNumber::new(2));
let progress = ShardProgress::new().with_buffered(SequenceNumber::new(2));
assert_eq!(
summary.write_status(kafka_partition, &progress),
@ -408,7 +408,7 @@ mod tests {
// kafka partition 1 has persisted up to sequence 3
let kafka_partition = KafkaPartition::new(1);
let progress = SequencerProgress::new().with_persisted(SequenceNumber::new(3));
let progress = ShardProgress::new().with_persisted(SequenceNumber::new(3));
assert_eq!(
summary.write_status(kafka_partition, &progress),
Ok(KafkaPartitionWriteStatus::Persisted)
@ -416,7 +416,7 @@ mod tests {
// kafka partition 2 has persisted up to sequence 2
let kafka_partition = KafkaPartition::new(2);
let progress = SequencerProgress::new().with_persisted(SequenceNumber::new(2));
let progress = ShardProgress::new().with_persisted(SequenceNumber::new(2));
assert_eq!(
summary.write_status(kafka_partition, &progress),
Ok(KafkaPartitionWriteStatus::Persisted)
@ -424,7 +424,7 @@ mod tests {
// kafka partition 1 only persisted up to sequence number 2, have buffered data at 3
let kafka_partition = KafkaPartition::new(1);
let progress = SequencerProgress::new()
let progress = ShardProgress::new()
.with_buffered(SequenceNumber::new(3))
.with_persisted(SequenceNumber::new(2));

View File

@ -13,7 +13,7 @@ use data_types::SequenceNumber;
///
/// Note: min_readable_sequence_number <= min_totally_persisted_sequence_number
#[derive(Clone, Debug, PartialEq, Eq, Default)]
pub struct SequencerProgress {
pub struct ShardProgress {
/// Smallest sequence number of data that is buffered in memory
min_buffered: Option<SequenceNumber>,
@ -30,7 +30,7 @@ pub struct SequencerProgress {
actively_buffering: Option<SequenceNumber>,
}
impl SequencerProgress {
impl ShardProgress {
pub fn new() -> Self {
Default::default()
}
@ -169,7 +169,7 @@ mod tests {
#[test]
fn empty() {
let progress = SequencerProgress::new();
let progress = ShardProgress::new();
let sequence_number = SequenceNumber::new(0);
assert!(!progress.readable(sequence_number));
assert!(!progress.persisted(sequence_number));
@ -181,7 +181,7 @@ mod tests {
let eq = SequenceNumber::new(1);
let gt = SequenceNumber::new(2);
let progress = SequencerProgress::new().with_persisted(eq);
let progress = ShardProgress::new().with_persisted(eq);
assert!(progress.readable(lt));
assert!(progress.persisted(lt));
@ -200,7 +200,7 @@ mod tests {
let eq = SequenceNumber::new(1);
let gt = SequenceNumber::new(2);
let progress = SequencerProgress::new().with_buffered(eq);
let progress = ShardProgress::new().with_buffered(eq);
assert!(progress.readable(lt));
assert!(!progress.persisted(lt));
@ -218,7 +218,7 @@ mod tests {
let eq = SequenceNumber::new(1);
let gt = SequenceNumber::new(2);
let progress = SequencerProgress::new()
let progress = ShardProgress::new()
.with_buffered(eq)
.with_persisted(lt);
@ -238,7 +238,7 @@ mod tests {
let eq = SequenceNumber::new(1);
let gt = SequenceNumber::new(2);
let progress = SequencerProgress::new()
let progress = ShardProgress::new()
.with_buffered(eq)
.with_persisted(eq);
@ -259,7 +259,7 @@ mod tests {
let gt = SequenceNumber::new(2);
// data buffered between lt and eq
let progress = SequencerProgress::new()
let progress = ShardProgress::new()
.with_buffered(lt)
.with_buffered(eq)
.with_persisted(eq);
@ -280,13 +280,13 @@ mod tests {
let eq = SequenceNumber::new(1);
let gt = SequenceNumber::new(2);
let progress1 = SequencerProgress::new().with_buffered(gt);
let progress1 = ShardProgress::new().with_buffered(gt);
let progress2 = SequencerProgress::new()
let progress2 = ShardProgress::new()
.with_buffered(lt)
.with_persisted(eq);
let expected = SequencerProgress::new()
let expected = ShardProgress::new()
.with_buffered(lt)
.with_buffered(gt)
.with_persisted(eq);
@ -309,7 +309,7 @@ mod tests {
let cases = vec![
// No buffering
(
SequencerProgress::new()
ShardProgress::new()
.actively_buffering(None)
.with_buffered(num1)
.with_buffered(num2),
@ -320,7 +320,7 @@ mod tests {
),
// actively buffering num2
(
SequencerProgress::new()
ShardProgress::new()
.actively_buffering(Some(num2))
.with_buffered(num1)
.with_buffered(num2),
@ -331,7 +331,7 @@ mod tests {
),
// actively buffering only one
(
SequencerProgress::new()
ShardProgress::new()
.actively_buffering(Some(num1))
.with_buffered(num1),
Expected {
@ -341,7 +341,7 @@ mod tests {
),
// actively buffering, haven't buffed any yet
(
SequencerProgress::new()
ShardProgress::new()
.actively_buffering(Some(num1))
.with_buffered(num0),
Expected {
@ -351,7 +351,7 @@ mod tests {
),
// actively buffering, haven't buffered any
(
SequencerProgress::new().actively_buffering(Some(num0)),
ShardProgress::new().actively_buffering(Some(num0)),
Expected {
min_buffered: None,
max_buffered: None,
@ -359,7 +359,7 @@ mod tests {
),
// actively buffering partially buffered
(
SequencerProgress::new()
ShardProgress::new()
.actively_buffering(Some(num0))
.with_buffered(num0),
Expected {