log_store/kafka/
worker.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
// 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.

pub(crate) mod dump_index;
pub(crate) mod flush;
pub(crate) mod produce;

use std::sync::Arc;

use common_telemetry::debug;
use futures::future::try_join_all;
use rskafka::client::partition::Compression;
use rskafka::record::Record;
use snafu::{OptionExt, ResultExt};
use store_api::logstore::provider::KafkaProvider;
use store_api::logstore::EntryId;
use store_api::storage::RegionId;
use tokio::sync::mpsc::Receiver;
use tokio::sync::oneshot::{self};

use crate::error::{self, NoMaxValueSnafu, Result};
use crate::kafka::index::{IndexCollector, IndexEncoder};
use crate::kafka::producer::ProducerClient;

pub(crate) enum WorkerRequest {
    Produce(ProduceRequest),
    TruncateIndex(TruncateIndexRequest),
    DumpIndex(DumpIndexRequest),
}

impl WorkerRequest {
    pub(crate) fn new_produce_request(
        region_id: RegionId,
        batch: Vec<Record>,
    ) -> (WorkerRequest, ProduceResultHandle) {
        let (tx, rx) = oneshot::channel();

        (
            WorkerRequest::Produce(ProduceRequest {
                region_id,
                batch,
                sender: tx,
            }),
            ProduceResultHandle { receiver: rx },
        )
    }
}

pub(crate) struct DumpIndexRequest {
    encoder: Arc<dyn IndexEncoder>,
    sender: oneshot::Sender<()>,
}

impl DumpIndexRequest {
    pub fn new(encoder: Arc<dyn IndexEncoder>) -> (DumpIndexRequest, oneshot::Receiver<()>) {
        let (tx, rx) = oneshot::channel();
        (
            DumpIndexRequest {
                encoder,
                sender: tx,
            },
            rx,
        )
    }
}

pub(crate) struct TruncateIndexRequest {
    region_id: RegionId,
    entry_id: EntryId,
}

impl TruncateIndexRequest {
    pub fn new(region_id: RegionId, entry_id: EntryId) -> Self {
        Self {
            region_id,
            entry_id,
        }
    }
}

pub(crate) struct ProduceRequest {
    region_id: RegionId,
    batch: Vec<Record>,
    sender: oneshot::Sender<ProduceResultReceiver>,
}

/// Receives the committed offsets when data has been committed to Kafka
/// or an unrecoverable error has been encountered.
pub(crate) struct ProduceResultHandle {
    receiver: oneshot::Receiver<ProduceResultReceiver>,
}

impl ProduceResultHandle {
    /// Waits for the data has been committed to Kafka.
    /// Returns the **max** committed offsets.
    pub(crate) async fn wait(self) -> Result<u64> {
        self.receiver
            .await
            .context(error::WaitProduceResultReceiverSnafu)?
            .wait()
            .await
    }
}

#[derive(Default)]
pub(crate) struct ProduceResultReceiver {
    receivers: Vec<oneshot::Receiver<Result<Vec<i64>>>>,
}

impl ProduceResultReceiver {
    fn add_receiver(&mut self, receiver: oneshot::Receiver<Result<Vec<i64>>>) {
        self.receivers.push(receiver)
    }

    async fn wait(self) -> Result<u64> {
        Ok(try_join_all(self.receivers)
            .await
            .into_iter()
            .flatten()
            .collect::<Result<Vec<_>>>()?
            .into_iter()
            .flatten()
            .max()
            .context(NoMaxValueSnafu)? as u64)
    }
}

pub(crate) struct PendingRequest {
    batch: Vec<Record>,
    region_ids: Vec<RegionId>,
    size: usize,
    sender: oneshot::Sender<Result<Vec<i64>>>,
}

pub(crate) struct BackgroundProducerWorker {
    pub(crate) provider: Arc<KafkaProvider>,
    /// The [`ProducerClient`].
    pub(crate) client: Arc<dyn ProducerClient>,
    // The compression configuration.
    pub(crate) compression: Compression,
    /// Receiver of [ProduceRequest].
    pub(crate) receiver: Receiver<WorkerRequest>,
    /// Max batch size for a worker to handle requests.
    pub(crate) request_batch_size: usize,
    /// Max bytes size for a single flush.
    pub(crate) max_batch_bytes: usize,
    /// Collecting ids of WAL entries.
    pub(crate) index_collector: Box<dyn IndexCollector>,
}

impl BackgroundProducerWorker {
    pub(crate) async fn run(&mut self) {
        let mut buffer = Vec::with_capacity(self.request_batch_size);
        loop {
            match self.receiver.recv().await {
                Some(req) => {
                    buffer.clear();
                    buffer.push(req);
                    for _ in 1..self.request_batch_size {
                        match self.receiver.try_recv() {
                            Ok(req) => buffer.push(req),
                            Err(_) => break,
                        }
                    }
                    self.handle_requests(&mut buffer).await;
                }
                None => {
                    debug!("The sender is dropped, BackgroundProducerWorker exited");
                    // Exits the loop if the `sender` is dropped.
                    break;
                }
            }
        }
    }

    async fn handle_requests(&mut self, buffer: &mut Vec<WorkerRequest>) {
        let mut produce_requests = Vec::with_capacity(buffer.len());
        for req in buffer.drain(..) {
            match req {
                WorkerRequest::Produce(req) => produce_requests.push(req),
                WorkerRequest::TruncateIndex(TruncateIndexRequest {
                    region_id,
                    entry_id,
                }) => self.index_collector.truncate(region_id, entry_id),
                WorkerRequest::DumpIndex(req) => self.dump_index(req).await,
            }
        }

        let pending_requests = self.aggregate_records(&mut produce_requests, self.max_batch_bytes);
        self.try_flush_pending_requests(pending_requests).await;
    }
}