Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix(storage): fast compactor judge whether using streaming uploader #13733

Merged
merged 4 commits into from
Nov 29, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 4 additions & 3 deletions src/storage/src/hummock/compactor/fast_compactor_runner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ use crate::hummock::value::HummockValue;
use crate::hummock::{
Block, BlockBuilder, BlockHolder, BlockIterator, BlockMeta, BlockedXor16FilterBuilder,
CachePolicy, CompressionAlgorithm, GetObjectId, HummockResult, SstableBuilderOptions,
StreamingSstableWriterFactory, TableHolder,
TableHolder, UnifiedSstableWriterFactory,
};
use crate::monitor::{CompactorMetrics, StoreLocalStatistic};

Expand Down Expand Up @@ -279,7 +279,7 @@ pub struct CompactorRunner {
right: Box<ConcatSstableIterator>,
task_id: u64,
executor: CompactTaskExecutor<
RemoteBuilderFactory<StreamingSstableWriterFactory, BlockedXor16FilterBuilder>,
RemoteBuilderFactory<UnifiedSstableWriterFactory, BlockedXor16FilterBuilder>,
>,
compression_algorithm: CompressionAlgorithm,
metrics: Arc<CompactorMetrics>,
Expand Down Expand Up @@ -313,7 +313,8 @@ impl CompactorRunner {
split_weight_by_vnode: task.split_weight_by_vnode,
use_block_based_filter: true,
};
let factory = StreamingSstableWriterFactory::new(context.sstable_store.clone());
let factory = UnifiedSstableWriterFactory::new(context.sstable_store.clone());
wcy-fdu marked this conversation as resolved.
Show resolved Hide resolved

let builder_factory = RemoteBuilderFactory::<_, BlockedXor16FilterBuilder> {
object_id_getter,
limiter: context.memory_limiter.clone(),
Expand Down
87 changes: 87 additions & 0 deletions src/storage/src/hummock/sstable_store.rs
Original file line number Diff line number Diff line change
Expand Up @@ -829,6 +829,11 @@ impl StreamingUploadWriter {
}
}

pub enum UnifiedSstableWriter {
StreamingSstableWriter(StreamingUploadWriter),
BatchSstableWriter(BatchUploadWriter),
}

#[async_trait::async_trait]
impl SstableWriter for StreamingUploadWriter {
type Output = JoinHandle<HummockResult<()>>;
Expand Down Expand Up @@ -917,6 +922,47 @@ impl StreamingSstableWriterFactory {
StreamingSstableWriterFactory { sstable_store }
}
}
pub struct UnifiedSstableWriterFactory {
sstable_store: SstableStoreRef,
}

impl UnifiedSstableWriterFactory {
pub fn new(sstable_store: SstableStoreRef) -> Self {
UnifiedSstableWriterFactory { sstable_store }
}
}

#[async_trait::async_trait]
impl SstableWriterFactory for UnifiedSstableWriterFactory {
type Writer = UnifiedSstableWriter;

async fn create_sst_writer(
&mut self,
object_id: HummockSstableObjectId,
options: SstableWriterOptions,
) -> HummockResult<Self::Writer> {
if self.sstable_store.store().support_streaming_upload() {
let path = self.sstable_store.get_sst_data_path(object_id);
let uploader = self.sstable_store.store.streaming_upload(&path).await?;
let streaming_uploader_writer = StreamingUploadWriter::new(
object_id,
self.sstable_store.clone(),
uploader,
options,
);

Ok(UnifiedSstableWriter::StreamingSstableWriter(
streaming_uploader_writer,
))
} else {
let batch_uploader_writer =
BatchUploadWriter::new(object_id, self.sstable_store.clone(), options);
Ok(UnifiedSstableWriter::BatchSstableWriter(
batch_uploader_writer,
))
}
}
}

#[async_trait::async_trait]
impl SstableWriterFactory for StreamingSstableWriterFactory {
Expand All @@ -938,6 +984,47 @@ impl SstableWriterFactory for StreamingSstableWriterFactory {
}
}

#[async_trait::async_trait]
impl SstableWriter for UnifiedSstableWriter {
type Output = JoinHandle<HummockResult<()>>;

async fn write_block(&mut self, block_data: &[u8], meta: &BlockMeta) -> HummockResult<()> {
match self {
UnifiedSstableWriter::StreamingSstableWriter(stream) => {
stream.write_block(block_data, meta).await
}
UnifiedSstableWriter::BatchSstableWriter(batch) => {
batch.write_block(block_data, meta).await
}
}
}

async fn write_block_bytes(&mut self, block: Bytes, meta: &BlockMeta) -> HummockResult<()> {
match self {
UnifiedSstableWriter::StreamingSstableWriter(stream) => {
stream.write_block_bytes(block, meta).await
}
UnifiedSstableWriter::BatchSstableWriter(batch) => {
batch.write_block_bytes(block, meta).await
}
}
}

async fn finish(self, meta: SstableMeta) -> HummockResult<UploadJoinHandle> {
match self {
UnifiedSstableWriter::StreamingSstableWriter(stream) => stream.finish(meta).await,
UnifiedSstableWriter::BatchSstableWriter(batch) => batch.finish(meta).await,
}
}

fn data_len(&self) -> usize {
match self {
UnifiedSstableWriter::StreamingSstableWriter(stream) => stream.data_len(),
UnifiedSstableWriter::BatchSstableWriter(batch) => batch.data_len(),
}
}
}

pub struct BlockStreamReader {
/// The stream that provides raw data.
byte_stream: MonitoredStreamingReader,
Expand Down
Loading