mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-05-17 13:30:38 +00:00
refactor: simplify the PaginationStream (#3787)
* refactor: simplify the `PaginationStream` * refactor: refactor decode fn * fix: fix clippy
This commit is contained in:
@@ -192,10 +192,10 @@ impl MigrateTableMetadata {
|
||||
let key = v1SchemaKey::parse(key_str)
|
||||
.unwrap_or_else(|e| panic!("schema key is corrupted: {e}, key: {key_str}"));
|
||||
|
||||
Ok((key, ()))
|
||||
Ok(key)
|
||||
}),
|
||||
);
|
||||
while let Some((key, _)) = stream.try_next().await.context(error::IterStreamSnafu)? {
|
||||
while let Some(key) = stream.try_next().await.context(error::IterStreamSnafu)? {
|
||||
let _ = self.migrate_schema_key(&key).await;
|
||||
keys.push(key.to_string().as_bytes().to_vec());
|
||||
}
|
||||
@@ -244,10 +244,10 @@ impl MigrateTableMetadata {
|
||||
let key = v1CatalogKey::parse(key_str)
|
||||
.unwrap_or_else(|e| panic!("catalog key is corrupted: {e}, key: {key_str}"));
|
||||
|
||||
Ok((key, ()))
|
||||
Ok(key)
|
||||
}),
|
||||
);
|
||||
while let Some((key, _)) = stream.try_next().await.context(error::IterStreamSnafu)? {
|
||||
while let Some(key) = stream.try_next().await.context(error::IterStreamSnafu)? {
|
||||
let _ = self.migrate_catalog_key(&key).await;
|
||||
keys.push(key.to_string().as_bytes().to_vec());
|
||||
}
|
||||
|
||||
@@ -17,7 +17,6 @@ use std::sync::Arc;
|
||||
|
||||
use common_catalog::consts::DEFAULT_CATALOG_NAME;
|
||||
use futures::stream::BoxStream;
|
||||
use futures::StreamExt;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
|
||||
@@ -84,11 +83,11 @@ impl<'a> TryFrom<&'a str> for CatalogNameKey<'a> {
|
||||
}
|
||||
|
||||
/// Decoder `KeyValue` to ({catalog},())
|
||||
pub fn catalog_decoder(kv: KeyValue) -> Result<(String, ())> {
|
||||
pub fn catalog_decoder(kv: KeyValue) -> Result<String> {
|
||||
let str = std::str::from_utf8(&kv.key).context(error::ConvertRawKeySnafu)?;
|
||||
let catalog_name = CatalogNameKey::try_from(str)?;
|
||||
|
||||
Ok((catalog_name.catalog.to_string(), ()))
|
||||
Ok(catalog_name.catalog.to_string())
|
||||
}
|
||||
|
||||
pub struct CatalogManager {
|
||||
@@ -134,7 +133,7 @@ impl CatalogManager {
|
||||
Arc::new(catalog_decoder),
|
||||
);
|
||||
|
||||
Box::pin(stream.map(|kv| kv.map(|kv| kv.0)))
|
||||
Box::pin(stream)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -16,7 +16,6 @@ use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use futures::stream::BoxStream;
|
||||
use futures::StreamExt;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::OptionExt;
|
||||
use store_api::storage::RegionNumber;
|
||||
@@ -126,10 +125,8 @@ impl DatanodeTableValue {
|
||||
}
|
||||
|
||||
/// Decodes `KeyValue` to ((),`DatanodeTableValue`)
|
||||
pub fn datanode_table_value_decoder(kv: KeyValue) -> Result<((), DatanodeTableValue)> {
|
||||
let value = DatanodeTableValue::try_from_raw_value(&kv.value)?;
|
||||
|
||||
Ok(((), value))
|
||||
pub fn datanode_table_value_decoder(kv: KeyValue) -> Result<DatanodeTableValue> {
|
||||
DatanodeTableValue::try_from_raw_value(&kv.value)
|
||||
}
|
||||
|
||||
pub struct DatanodeTableManager {
|
||||
@@ -163,7 +160,7 @@ impl DatanodeTableManager {
|
||||
Arc::new(datanode_table_value_decoder),
|
||||
);
|
||||
|
||||
Box::pin(stream.map(|kv| kv.map(|kv| kv.1)))
|
||||
Box::pin(stream)
|
||||
}
|
||||
|
||||
/// Builds the create datanode table transactions. It only executes while the primary keys comparing successes.
|
||||
|
||||
@@ -19,7 +19,6 @@ use std::time::Duration;
|
||||
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use futures::stream::BoxStream;
|
||||
use futures::StreamExt;
|
||||
use humantime_serde::re::humantime;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
@@ -103,11 +102,11 @@ impl TableMetaKey for SchemaNameKey<'_> {
|
||||
}
|
||||
|
||||
/// Decodes `KeyValue` to ({schema},())
|
||||
pub fn schema_decoder(kv: KeyValue) -> Result<(String, ())> {
|
||||
pub fn schema_decoder(kv: KeyValue) -> Result<String> {
|
||||
let str = std::str::from_utf8(&kv.key).context(error::ConvertRawKeySnafu)?;
|
||||
let schema_name = SchemaNameKey::try_from(str)?;
|
||||
|
||||
Ok((schema_name.schema.to_string(), ()))
|
||||
Ok(schema_name.schema.to_string())
|
||||
}
|
||||
|
||||
impl<'a> TryFrom<&'a str> for SchemaNameKey<'a> {
|
||||
@@ -193,7 +192,7 @@ impl SchemaManager {
|
||||
Arc::new(schema_decoder),
|
||||
);
|
||||
|
||||
Box::pin(stream.map(|kv| kv.map(|kv| kv.0)))
|
||||
Box::pin(stream)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -28,13 +28,13 @@ use crate::rpc::store::{RangeRequest, RangeResponse};
|
||||
use crate::rpc::KeyValue;
|
||||
use crate::util::get_next_prefix_key;
|
||||
|
||||
pub type KeyValueDecoderFn<K, V> = dyn Fn(KeyValue) -> Result<(K, V)> + Send + Sync;
|
||||
pub type KeyValueDecoderFn<T> = dyn Fn(KeyValue) -> Result<T> + Send + Sync;
|
||||
|
||||
enum PaginationStreamState<K, V> {
|
||||
enum PaginationStreamState<T> {
|
||||
/// At the start of reading.
|
||||
Init,
|
||||
/// Decoding key value pairs.
|
||||
Decoding(SimpleKeyValueDecoder<K, V>),
|
||||
Decoding(SimpleKeyValueDecoder<T>),
|
||||
/// Retrieving data from backend.
|
||||
Reading(BoxFuture<'static, Result<(PaginationStreamFactory, Option<RangeResponse>)>>),
|
||||
/// Error
|
||||
@@ -77,7 +77,7 @@ struct PaginationStreamFactory {
|
||||
}
|
||||
|
||||
impl PaginationStreamFactory {
|
||||
pub fn new(
|
||||
fn new(
|
||||
kv: &KvBackendRef,
|
||||
key: Vec<u8>,
|
||||
range_end: Vec<u8>,
|
||||
@@ -137,7 +137,7 @@ impl PaginationStreamFactory {
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn read_next(mut self) -> Result<(Self, Option<RangeResponse>)> {
|
||||
async fn read_next(mut self) -> Result<(Self, Option<RangeResponse>)> {
|
||||
if self.more {
|
||||
let resp = self
|
||||
.adaptive_range(RangeRequest {
|
||||
@@ -174,18 +174,19 @@ impl PaginationStreamFactory {
|
||||
}
|
||||
}
|
||||
|
||||
pub struct PaginationStream<K, V> {
|
||||
state: PaginationStreamState<K, V>,
|
||||
decoder_fn: Arc<KeyValueDecoderFn<K, V>>,
|
||||
pub struct PaginationStream<T> {
|
||||
state: PaginationStreamState<T>,
|
||||
decoder_fn: Arc<KeyValueDecoderFn<T>>,
|
||||
factory: Option<PaginationStreamFactory>,
|
||||
}
|
||||
|
||||
impl<K, V> PaginationStream<K, V> {
|
||||
impl<T> PaginationStream<T> {
|
||||
/// Returns a new [PaginationStream].
|
||||
pub fn new(
|
||||
kv: KvBackendRef,
|
||||
req: RangeRequest,
|
||||
page_size: usize,
|
||||
decoder_fn: Arc<KeyValueDecoderFn<K, V>>,
|
||||
decoder_fn: Arc<KeyValueDecoderFn<T>>,
|
||||
) -> Self {
|
||||
Self {
|
||||
state: PaginationStreamState::Init,
|
||||
@@ -202,13 +203,13 @@ impl<K, V> PaginationStream<K, V> {
|
||||
}
|
||||
}
|
||||
|
||||
struct SimpleKeyValueDecoder<K, V> {
|
||||
struct SimpleKeyValueDecoder<T> {
|
||||
kv: VecDeque<KeyValue>,
|
||||
decoder: Arc<KeyValueDecoderFn<K, V>>,
|
||||
decoder: Arc<KeyValueDecoderFn<T>>,
|
||||
}
|
||||
|
||||
impl<K, V> Iterator for SimpleKeyValueDecoder<K, V> {
|
||||
type Item = Result<(K, V)>;
|
||||
impl<T> Iterator for SimpleKeyValueDecoder<T> {
|
||||
type Item = Result<T>;
|
||||
|
||||
fn next(&mut self) -> Option<Self::Item> {
|
||||
if let Some(kv) = self.kv.pop_front() {
|
||||
@@ -219,8 +220,8 @@ impl<K, V> Iterator for SimpleKeyValueDecoder<K, V> {
|
||||
}
|
||||
}
|
||||
|
||||
impl<K, V> Stream for PaginationStream<K, V> {
|
||||
type Item = Result<(K, V)>;
|
||||
impl<T> Stream for PaginationStream<T> {
|
||||
type Item = Result<T>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
loop {
|
||||
|
||||
@@ -102,15 +102,10 @@ impl LeaderCachedKvBackend {
|
||||
self.store.clone(),
|
||||
RangeRequest::new().with_prefix(prefix.as_bytes()),
|
||||
DEFAULT_PAGE_SIZE,
|
||||
Arc::new(|kv| Ok((kv, ()))),
|
||||
Arc::new(Ok),
|
||||
);
|
||||
|
||||
let kvs = stream
|
||||
.try_collect::<Vec<_>>()
|
||||
.await?
|
||||
.into_iter()
|
||||
.map(|(kv, _)| kv)
|
||||
.collect();
|
||||
let kvs = stream.try_collect::<Vec<_>>().await?.into_iter().collect();
|
||||
|
||||
self.cache
|
||||
.batch_put(BatchPutRequest {
|
||||
|
||||
Reference in New Issue
Block a user