refactor: Rename the WriteBuffer trait to be WriteBufferWriting

pull/24376/head
Carol (Nichols || Goulding) 2021-06-17 17:00:05 -04:00
parent 83e50cfba4
commit fd4bcc2fa5
4 changed files with 13 additions and 13 deletions

View File

@ -15,7 +15,7 @@ use query::exec::Executor;
/// This module contains code for managing the configuration of the server.
use crate::{
db::{catalog::Catalog, DatabaseToCommit, Db},
write_buffer::WriteBuffer,
write_buffer::WriteBufferWriting,
Error, JobRegistry, Result,
};
use observability_deps::tracing::{self, error, info, warn, Instrument};
@ -628,7 +628,7 @@ impl<'a> DatabaseHandle<'a> {
&mut self,
preserved_catalog: PreservedCatalog,
catalog: Catalog,
write_buffer: Option<Arc<dyn WriteBuffer>>,
write_buffer: Option<Arc<dyn WriteBufferWriting>>,
) -> Result<()> {
match self.state().as_ref() {
DatabaseState::RulesLoaded {

View File

@ -11,7 +11,7 @@ use crate::{
catalog::{chunk::CatalogChunk, partition::Partition, Catalog, TableNameFilter},
lifecycle::{LockableCatalogChunk, LockableCatalogPartition},
},
write_buffer::WriteBuffer,
write_buffer::WriteBufferWriting,
JobRegistry,
};
use ::lifecycle::{LockableChunk, LockablePartition};
@ -230,7 +230,7 @@ pub struct Db {
metric_labels: Vec<KeyValue>,
/// Optionally buffer writes
write_buffer: Option<Arc<dyn WriteBuffer>>,
write_buffer: Option<Arc<dyn WriteBufferWriting>>,
/// Lock that prevents the cleanup job from deleting files that are written but not yet added to the preserved
/// catalog.
@ -249,7 +249,7 @@ pub(crate) struct DatabaseToCommit {
pub(crate) preserved_catalog: PreservedCatalog,
pub(crate) catalog: Catalog,
pub(crate) rules: DatabaseRules,
pub(crate) write_buffer: Option<Arc<dyn WriteBuffer>>,
pub(crate) write_buffer: Option<Arc<dyn WriteBufferWriting>>,
}
impl Db {

View File

@ -11,7 +11,7 @@ use query::{exec::Executor, QueryDatabase};
use crate::{
db::{load::load_or_create_preserved_catalog, DatabaseToCommit, Db},
write_buffer::WriteBuffer,
write_buffer::WriteBufferWriting,
JobRegistry,
};
@ -35,7 +35,7 @@ pub struct TestDbBuilder {
object_store: Option<Arc<ObjectStore>>,
db_name: Option<DatabaseName<'static>>,
worker_cleanup_avg_sleep: Option<Duration>,
write_buffer: Option<Arc<dyn WriteBuffer>>,
write_buffer: Option<Arc<dyn WriteBufferWriting>>,
catalog_transactions_until_checkpoint: Option<NonZeroU64>,
partition_template: Option<PartitionTemplate>,
}
@ -127,7 +127,7 @@ impl TestDbBuilder {
self
}
pub fn write_buffer(mut self, write_buffer: Arc<dyn WriteBuffer>) -> Self {
pub fn write_buffer(mut self, write_buffer: Arc<dyn WriteBufferWriting>) -> Self {
self.write_buffer = Some(write_buffer);
self
}

View File

@ -10,10 +10,10 @@ use std::{convert::TryInto, sync::Arc};
pub type WriteBufferError = Box<dyn std::error::Error + Sync + Send>;
pub fn new(rules: &DatabaseRules) -> Result<Option<Arc<dyn WriteBuffer>>, WriteBufferError> {
pub fn new(rules: &DatabaseRules) -> Result<Option<Arc<dyn WriteBufferWriting>>, WriteBufferError> {
let name = rules.db_name();
// Right now, `KafkaBuffer` is the only production implementation of the `WriteBuffer`
// Right now, `KafkaBuffer` is the only production implementation of the `WriteBufferWriting`
// trait, so always use `KafkaBuffer` when there is a write buffer connection string
// specified. If/when there are other kinds of write buffers, additional configuration will
// be needed to determine what kind of write buffer to use here.
@ -30,7 +30,7 @@ pub fn new(rules: &DatabaseRules) -> Result<Option<Arc<dyn WriteBuffer>>, WriteB
/// A Write Buffer takes an `Entry` and returns `Sequence` data that facilitates reading entries
/// from the Write Buffer at a later time.
#[async_trait]
pub trait WriteBuffer: Sync + Send + std::fmt::Debug + 'static {
pub trait WriteBufferWriting: Sync + Send + std::fmt::Debug + 'static {
/// Send an `Entry` to the write buffer and return information that can be used to restore
/// entries at a later time.
async fn store_entry(&self, entry: &Entry) -> Result<Sequence, WriteBufferError>;
@ -56,7 +56,7 @@ impl std::fmt::Debug for KafkaBuffer {
}
#[async_trait]
impl WriteBuffer for KafkaBuffer {
impl WriteBufferWriting for KafkaBuffer {
/// Send an `Entry` to Kafka and return the partition ID as the sequencer ID and the offset
/// as the sequence number.
async fn store_entry(&self, entry: &Entry) -> Result<Sequence, WriteBufferError> {
@ -115,7 +115,7 @@ pub mod test_helpers {
}
#[async_trait]
impl WriteBuffer for MockBuffer {
impl WriteBufferWriting for MockBuffer {
async fn store_entry(&self, entry: &Entry) -> Result<Sequence, WriteBufferError> {
let mut entries = self.entries.lock().unwrap();
let offset = entries.len() as u64;