mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-14 01:02:55 +00:00
refactor: Entry should be a trait (#37)
This commit is contained in:
@@ -16,15 +16,16 @@ pub mod namespace;
|
||||
pub trait LogStore {
|
||||
type Error: ErrorExt + Send + Sync;
|
||||
type Namespace: Namespace;
|
||||
type Entry: Entry;
|
||||
|
||||
/// Append an `Entry` to WAL with given namespace
|
||||
async fn append(&mut self, ns: Self::Namespace, e: Entry) -> Result<Offset, Self::Error>;
|
||||
async fn append(&mut self, ns: Self::Namespace, e: Self::Entry) -> Result<Offset, Self::Error>;
|
||||
|
||||
// Append a batch of entries atomically and return the offset of first entry.
|
||||
async fn append_batch(
|
||||
&mut self,
|
||||
ns: Self::Namespace,
|
||||
e: Vec<Entry>,
|
||||
e: Vec<Self::Entry>,
|
||||
) -> Result<Offset, Self::Error>;
|
||||
|
||||
// Create a new `EntryStream` to asynchronously generates `Entry`.
|
||||
@@ -32,7 +33,7 @@ pub trait LogStore {
|
||||
&self,
|
||||
ns: Self::Namespace,
|
||||
offset: Offset,
|
||||
) -> Result<SendableEntryStream, Self::Error>;
|
||||
) -> Result<SendableEntryStream<Self::Entry>, Self::Error>;
|
||||
|
||||
// Create a new `Namespace`.
|
||||
async fn create_namespace(&mut self, ns: Self::Namespace) -> Result<(), Self::Error>;
|
||||
|
||||
@@ -1,43 +1,14 @@
|
||||
pub type Offset = u64;
|
||||
pub type Epoch = u64;
|
||||
|
||||
pub struct Entry {
|
||||
/// Offset of current entry
|
||||
offset: Offset,
|
||||
/// Epoch of current entry
|
||||
epoch: Epoch,
|
||||
/// Binary data of current entry
|
||||
data: Vec<u8>,
|
||||
}
|
||||
|
||||
impl Entry {
|
||||
pub fn new(data: impl AsRef<[u8]>, offset: Offset, epoch: u64) -> Self {
|
||||
let data = data.as_ref().to_vec();
|
||||
Self {
|
||||
data,
|
||||
offset,
|
||||
epoch,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn data(&self) -> &[u8] {
|
||||
self.data.as_slice()
|
||||
}
|
||||
|
||||
pub fn offset(&self) -> Offset {
|
||||
self.offset
|
||||
}
|
||||
|
||||
pub fn epoch(&self) -> Epoch {
|
||||
self.epoch
|
||||
}
|
||||
|
||||
/// Return total length of entry after serialization(maybe CRC and length field)
|
||||
pub fn len(&self) -> usize {
|
||||
self.data.len() + 8
|
||||
}
|
||||
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.data.len() == 0
|
||||
}
|
||||
/// Entry is the minimal data storage unit in `LogStore`.
|
||||
pub trait Entry {
|
||||
/// Return contained data of entry.
|
||||
fn data(&self) -> &[u8];
|
||||
|
||||
/// Return offset of entry.
|
||||
fn offset(&self) -> Offset;
|
||||
|
||||
/// Returns epoch of entry.
|
||||
fn epoch(&self) -> Epoch;
|
||||
}
|
||||
|
||||
@@ -1,51 +1,90 @@
|
||||
use std::pin::Pin;
|
||||
use std::task::{Context, Poll};
|
||||
|
||||
use futures::Stream;
|
||||
|
||||
use crate::logstore::entry::Entry;
|
||||
use crate::logstore::Offset;
|
||||
|
||||
pub trait EntryStream: Stream<Item = Vec<Entry>> {
|
||||
pub trait EntryStream: Stream<Item = Vec<Self::Entry>> {
|
||||
type Entry: Entry;
|
||||
fn start_offset(&self) -> Offset;
|
||||
}
|
||||
|
||||
pub type SendableEntryStream<'a> = Pin<Box<dyn Stream<Item = Vec<Entry>> + Send + 'a>>;
|
||||
|
||||
pub struct EntryStreamImpl<'a> {
|
||||
inner: SendableEntryStream<'a>,
|
||||
start_offset: Offset,
|
||||
}
|
||||
|
||||
impl<'a> EntryStream for EntryStreamImpl<'a> {
|
||||
fn start_offset(&self) -> Offset {
|
||||
self.start_offset
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for EntryStreamImpl<'_> {
|
||||
type Item = Vec<Entry>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
match Pin::new(&mut self.inner).poll_next(cx) {
|
||||
Poll::Ready(Some(v)) => Poll::Ready(Some(v)),
|
||||
Poll::Ready(None) => Poll::Ready(None),
|
||||
Poll::Pending => Poll::Pending,
|
||||
}
|
||||
}
|
||||
}
|
||||
pub type SendableEntryStream<'a, E> = Pin<Box<dyn Stream<Item = Vec<E>> + Send + 'a>>;
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::task::{Context, Poll};
|
||||
|
||||
use futures::StreamExt;
|
||||
|
||||
use super::*;
|
||||
use crate::logstore::entry::Epoch;
|
||||
|
||||
pub struct SimpleEntry {
|
||||
/// Offset of current entry
|
||||
offset: Offset,
|
||||
/// Epoch of current entry
|
||||
epoch: Epoch,
|
||||
/// Binary data of current entry
|
||||
data: Vec<u8>,
|
||||
}
|
||||
|
||||
impl Entry for SimpleEntry {
|
||||
fn data(&self) -> &[u8] {
|
||||
self.data.as_slice()
|
||||
}
|
||||
|
||||
fn offset(&self) -> Offset {
|
||||
self.offset
|
||||
}
|
||||
|
||||
fn epoch(&self) -> Epoch {
|
||||
self.epoch
|
||||
}
|
||||
}
|
||||
|
||||
impl SimpleEntry {
|
||||
pub fn new(data: impl AsRef<[u8]>, offset: Offset, epoch: u64) -> Self {
|
||||
let data = data.as_ref().to_vec();
|
||||
Self {
|
||||
data,
|
||||
offset,
|
||||
epoch,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct EntryStreamImpl<'a> {
|
||||
inner: SendableEntryStream<'a, SimpleEntry>,
|
||||
start_offset: Offset,
|
||||
}
|
||||
|
||||
impl<'a> EntryStream for EntryStreamImpl<'a> {
|
||||
type Entry = SimpleEntry;
|
||||
|
||||
fn start_offset(&self) -> Offset {
|
||||
self.start_offset
|
||||
}
|
||||
}
|
||||
|
||||
impl Stream for EntryStreamImpl<'_> {
|
||||
type Item = Vec<SimpleEntry>;
|
||||
|
||||
fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
|
||||
match Pin::new(&mut self.inner).poll_next(cx) {
|
||||
Poll::Ready(Some(v)) => Poll::Ready(Some(v)),
|
||||
Poll::Ready(None) => Poll::Ready(None),
|
||||
Poll::Pending => Poll::Pending,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
pub async fn test_entry_stream() {
|
||||
let stream =
|
||||
async_stream::stream!({ yield vec![Entry::new("test_entry".as_bytes(), 0, 128)] });
|
||||
let stream = async_stream::stream!({
|
||||
yield vec![SimpleEntry::new("test_entry".as_bytes(), 0, 128)]
|
||||
});
|
||||
|
||||
let mut stream_impl = EntryStreamImpl {
|
||||
inner: Box::pin(stream),
|
||||
|
||||
Reference in New Issue
Block a user