Skip to content

Commit

Permalink
feat(log_store): introduce the IndexCollector (#4461)
Browse files Browse the repository at this point in the history
* feat: introduce the IndexCollector

* refactor: separate BackgroundProducerWorker code into files

* feat: introduce index related operations

* feat: introduce the `GlobalIndexCollector`

* refactor: move collector to index mod

* refactor: refactor `GlobalIndexCollector`

* chore: remove unused collector.rs

* chore: add comments

* chore: add comments

* chore: apply suggestions from CR

* chore: apply suggestions from CR
  • Loading branch information
WenyXu authored Aug 13, 2024
1 parent 2e2eacf commit c821d21
Show file tree
Hide file tree
Showing 13 changed files with 697 additions and 269 deletions.
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions src/log-store/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ workspace = true
[dependencies]
async-stream.workspace = true
async-trait.workspace = true
bytes.workspace = true
chrono.workspace = true
common-base.workspace = true
common-error.workspace = true
Expand Down
8 changes: 2 additions & 6 deletions src/log-store/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@ use serde_json::error::Error as JsonError;
use snafu::{Location, Snafu};
use store_api::storage::RegionId;

use crate::kafka::producer::ProduceRequest;

#[derive(Snafu)]
#[snafu(visibility(pub))]
#[stack_trace_debug]
Expand Down Expand Up @@ -268,12 +266,10 @@ pub enum Error {
attempt_index: u64,
},

#[snafu(display("Failed to send produce request"))]
SendProduceRequest {
#[snafu(display("OrderedBatchProducer is stopped",))]
OrderedBatchProducerStopped {
#[snafu(implicit)]
location: Location,
#[snafu(source)]
error: tokio::sync::mpsc::error::SendError<ProduceRequest>,
},

#[snafu(display("Failed to send produce request"))]
Expand Down
5 changes: 5 additions & 0 deletions src/log-store/src/kafka.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,17 @@
// limitations under the License.

pub(crate) mod client_manager;
// TODO(weny): remove it
#[allow(dead_code)]
pub(crate) mod consumer;
#[allow(unused)]
pub(crate) mod index;
pub mod log_store;
pub(crate) mod producer;
pub(crate) mod util;
// TODO(weny): remove it
#[allow(dead_code)]
pub(crate) mod worker;

use serde::{Deserialize, Serialize};
use store_api::logstore::entry::Id as EntryId;
Expand Down
24 changes: 13 additions & 11 deletions src/log-store/src/kafka/client_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,19 +23,16 @@ use snafu::ResultExt;
use store_api::logstore::provider::KafkaProvider;
use tokio::sync::{Mutex, RwLock};

use super::producer::OrderedBatchProducer;
use crate::error::{
BuildClientSnafu, BuildPartitionClientSnafu, ResolveKafkaEndpointSnafu, Result, TlsConfigSnafu,
};
use crate::kafka::producer::OrderedBatchProducerRef;
use crate::kafka::index::{GlobalIndexCollector, NoopCollector};
use crate::kafka::producer::{OrderedBatchProducer, OrderedBatchProducerRef};

// Each topic only has one partition for now.
// The `DEFAULT_PARTITION` refers to the index of the partition.
const DEFAULT_PARTITION: i32 = 0;

// Max batch size for a `OrderedBatchProducer` to handle requests.
const REQUEST_BATCH_SIZE: usize = 64;

/// Arc wrapper of ClientManager.
pub(crate) type ClientManagerRef = Arc<ClientManager>;

Expand Down Expand Up @@ -63,9 +60,8 @@ pub(crate) struct ClientManager {
/// Used to initialize a new [Client].
mutex: Mutex<()>,
instances: RwLock<HashMap<Arc<KafkaProvider>, Client>>,
global_index_collector: Option<GlobalIndexCollector>,

producer_channel_size: usize,
producer_request_batch_size: usize,
flush_batch_size: usize,
compression: Compression,
}
Expand Down Expand Up @@ -99,10 +95,9 @@ impl ClientManager {
client,
mutex: Mutex::new(()),
instances: RwLock::new(HashMap::new()),
producer_channel_size: REQUEST_BATCH_SIZE * 2,
producer_request_batch_size: REQUEST_BATCH_SIZE,
flush_batch_size: config.max_batch_bytes.as_bytes() as usize,
compression: Compression::Lz4,
global_index_collector: None,
})
}

Expand Down Expand Up @@ -151,12 +146,19 @@ impl ClientManager {
})
.map(Arc::new)?;

let (tx, rx) = OrderedBatchProducer::channel();
let index_collector = if let Some(global_collector) = self.global_index_collector.as_ref() {
global_collector.provider_level_index_collector(provider.clone(), tx.clone())
} else {
Box::new(NoopCollector)
};
let producer = Arc::new(OrderedBatchProducer::new(
(tx, rx),
provider.clone(),
client.clone(),
self.compression,
self.producer_channel_size,
self.producer_request_batch_size,
self.flush_batch_size,
index_collector,
));

Ok(Client { client, producer })
Expand Down
4 changes: 4 additions & 0 deletions src/log-store/src/kafka/index.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,8 +12,12 @@
// See the License for the specific language governing permissions and
// limitations under the License.

mod collector;
mod iterator;

pub(crate) use collector::{
GlobalIndexCollector, IndexCollector, IndexEncoder, NoopCollector, ProviderLevelIndexCollector,
};
pub(crate) use iterator::{
MultipleRegionWalIndexIterator, NextBatchHint, RegionWalIndexIterator, RegionWalRange,
RegionWalVecIndex,
Expand Down
149 changes: 149 additions & 0 deletions src/log-store/src/kafka/index/collector.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::{BTreeSet, HashMap};
use std::io::Write;
use std::sync::{Arc, Mutex};
use std::time::Duration;

use bytes::buf::Writer;
use bytes::{BufMut, Bytes, BytesMut};
use common_telemetry::tracing::error;
use futures::future::try_join_all;
use serde::{Deserialize, Serialize};
use snafu::ResultExt;
use store_api::logstore::provider::KafkaProvider;
use store_api::logstore::EntryId;
use store_api::storage::RegionId;
use tokio::select;
use tokio::sync::mpsc::Sender;
use tokio::sync::Mutex as TokioMutex;

use crate::error::{self, Result};
use crate::kafka::worker::{DumpIndexRequest, WorkerRequest};

pub trait IndexEncoder: Send + Sync {
fn encode(&self, provider: &KafkaProvider, region_index: &RegionIndexes);

fn finish(&self) -> Result<Vec<u8>>;
}

/// The [`IndexCollector`] trait defines the operations for managing and collecting index entries.
pub trait IndexCollector: Send + Sync {
/// Appends an [`EntryId`] for a specific region.
fn append(&mut self, region_id: RegionId, entry_id: EntryId);

/// Truncates the index for a specific region up to a given [`EntryId`].
///
/// It removes all [`EntryId`]s smaller than `entry_id`.
fn truncate(&mut self, region_id: RegionId, entry_id: EntryId);

/// Sets the latest [`EntryId`].
fn set_latest_entry_id(&mut self, entry_id: EntryId);

/// Dumps the index.
fn dump(&mut self, encoder: &dyn IndexEncoder);
}

/// The [`GlobalIndexCollector`] struct is responsible for managing index entries
/// across multiple providers.
#[derive(Debug, Clone, Default)]
pub struct GlobalIndexCollector {
providers: Arc<TokioMutex<HashMap<Arc<KafkaProvider>, Sender<WorkerRequest>>>>,
}

impl GlobalIndexCollector {
/// Creates a new [`ProviderLevelIndexCollector`] for a specified provider.
pub fn provider_level_index_collector(
&self,
provider: Arc<KafkaProvider>,
sender: Sender<WorkerRequest>,
) -> Box<dyn IndexCollector> {
Box::new(ProviderLevelIndexCollector {
indexes: Default::default(),
provider,
})
}
}

/// The [`RegionIndexes`] struct maintains indexes for a collection of regions.
/// Each region is identified by a `RegionId` and maps to a set of [`EntryId`]s,
/// representing the entries within that region. It also keeps track of the
/// latest [`EntryId`] across all regions.
#[derive(Debug, Clone, Default, Serialize, Deserialize)]
pub struct RegionIndexes {
regions: HashMap<RegionId, BTreeSet<EntryId>>,
latest_entry_id: EntryId,
}

impl RegionIndexes {
fn append(&mut self, region_id: RegionId, entry_id: EntryId) {
self.regions.entry(region_id).or_default().insert(entry_id);
self.latest_entry_id = self.latest_entry_id.max(entry_id);
}

fn truncate(&mut self, region_id: RegionId, entry_id: EntryId) {
if let Some(entry_ids) = self.regions.get_mut(&region_id) {
*entry_ids = entry_ids.split_off(&entry_id);
// The `RegionIndexes` can be empty, keeps to track the latest entry id.
self.latest_entry_id = self.latest_entry_id.max(entry_id);
}
}

fn set_latest_entry_id(&mut self, entry_id: EntryId) {
self.latest_entry_id = entry_id;
}
}

/// The [`ProviderLevelIndexCollector`] struct is responsible for managing index entries
/// specific to a particular provider.
#[derive(Debug, Clone)]
pub struct ProviderLevelIndexCollector {
indexes: RegionIndexes,
provider: Arc<KafkaProvider>,
}

impl IndexCollector for ProviderLevelIndexCollector {
fn append(&mut self, region_id: RegionId, entry_id: EntryId) {
self.indexes.append(region_id, entry_id)
}

fn truncate(&mut self, region_id: RegionId, entry_id: EntryId) {
self.indexes.truncate(region_id, entry_id)
}

fn set_latest_entry_id(&mut self, entry_id: EntryId) {
self.indexes.set_latest_entry_id(entry_id);
}

fn dump(&mut self, encoder: &dyn IndexEncoder) {
encoder.encode(&self.provider, &self.indexes)
}
}

/// The [`NoopCollector`] struct implements the [`IndexCollector`] trait with no-op methods.
///
/// This collector effectively ignores all operations, making it suitable for cases
/// where index collection is not required or should be disabled.
pub struct NoopCollector;

impl IndexCollector for NoopCollector {
fn append(&mut self, _region_id: RegionId, _entry_id: EntryId) {}

fn truncate(&mut self, _region_id: RegionId, _entry_id: EntryId) {}

fn set_latest_entry_id(&mut self, _entry_id: EntryId) {}

fn dump(&mut self, encoder: &dyn IndexEncoder) {}
}
3 changes: 2 additions & 1 deletion src/log-store/src/kafka/log_store.rs
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,8 @@ impl LogStore for KafkaLogStore {
for (region_id, (producer, records)) in region_grouped_records {
// Safety: `KafkaLogStore::entry` will ensure that the
// `Record`'s `approximate_size` must be less or equal to `max_batch_bytes`.
region_grouped_result_receivers.push((region_id, producer.produce(records).await?))
region_grouped_result_receivers
.push((region_id, producer.produce(region_id, records).await?))
}

let region_grouped_max_offset =
Expand Down
Loading

0 comments on commit c821d21

Please sign in to comment.