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
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
use std::collections::HashMap;
use std::sync::Arc;
use std::time::Duration;

use async_lock::RwLock;

use dataplane::ReplicaKey;
use dataplane::record::Record;

#[cfg(feature = "smartengine")]
use fluvio_smartengine::SmartModuleInstance;
use fluvio_types::PartitionId;
use fluvio_types::event::StickyEvent;
use tracing::instrument;

mod accumulator;
mod config;
mod error;
mod event;
mod output;
mod partitioning;
mod record;
mod partition_producer;

pub use dataplane::record::{RecordKey, RecordData};

use crate::FluvioError;
use crate::spu::SpuPool;
use crate::producer::accumulator::{RecordAccumulator, PushRecord};
use crate::producer::partitioning::{Partitioner, PartitionerConfig};

use self::accumulator::{BatchHandler};
pub use self::config::{
    TopicProducerConfigBuilder, TopicProducerConfig, TopicProducerConfigBuilderError,
};
pub use self::error::ProducerError;
use self::event::EventHandler;
pub use self::output::ProduceOutput;
use self::partition_producer::PartitionProducer;
pub use self::record::FutureRecordMetadata;

use crate::error::Result;

/// An interface for producing events to a particular topic
///
/// A `TopicProducer` allows you to send events to the specific
/// topic it was initialized for. Once you have a `TopicProducer`,
/// you can send events to the topic, choosing which partition
/// each event should be delivered to.
pub struct TopicProducer {
    inner: Arc<InnerTopicProducer>,
    #[cfg(feature = "smartengine")]
    pub(crate) smartmodule_instance: Option<Arc<RwLock<Box<dyn SmartModuleInstance>>>>,
}

/// Pool of producers for a given topic. There is a producer per partition
struct ProducerPool {
    flush_events: Vec<(Arc<EventHandler>, Arc<EventHandler>)>,
    end_events: Vec<Arc<StickyEvent>>,
    errors: Vec<Arc<RwLock<Option<ProducerError>>>>,
}

impl ProducerPool {
    fn new(
        topic: String,
        spu_pool: Arc<SpuPool>,
        batches: Arc<HashMap<PartitionId, BatchHandler>>,
        linger: Duration,
    ) -> Self {
        let mut end_events = vec![];
        let mut flush_events = vec![];
        let mut errors = vec![];
        for (&partition_id, (batch_events, batch_list)) in batches.iter() {
            let end_event = StickyEvent::shared();
            let flush_event = (EventHandler::shared(), EventHandler::shared());
            let replica = ReplicaKey::new(topic.clone(), partition_id);
            let error = Arc::new(RwLock::new(None));

            PartitionProducer::start(
                replica,
                spu_pool.clone(),
                batch_list.clone(),
                batch_events.clone(),
                linger,
                error.clone(),
                end_event.clone(),
                flush_event.clone(),
            );
            errors.push(error);
            end_events.push(end_event);
            flush_events.push(flush_event);
        }
        Self {
            end_events,
            flush_events,
            errors,
        }
    }

    fn shared(
        topic: String,
        spu_pool: Arc<SpuPool>,
        batches: Arc<HashMap<PartitionId, BatchHandler>>,
        linger: Duration,
    ) -> Arc<Self> {
        Arc::new(ProducerPool::new(topic, spu_pool, batches, linger))
    }

    async fn flush_all_batches(&self) -> Result<()> {
        for ((manual_flush_notifier, batch_flushed_event), error) in
            self.flush_events.iter().zip(self.errors.iter())
        {
            let listener = batch_flushed_event.listen();
            manual_flush_notifier.notify().await;
            listener.await;
            {
                let error_handle = error.read().await;
                if let Some(error) = &*error_handle {
                    return Err(error.clone().into());
                }
            }
        }

        Ok(())
    }

    async fn last_error(&self, partition_id: PartitionId) -> Option<ProducerError> {
        let error = self.errors[partition_id as usize].read().await;
        error.clone()
    }

    async fn clear_errors(&self) {
        for error in self.errors.iter() {
            let mut error_handle = error.write().await;
            *error_handle = None;
        }
    }

    fn end(&self) {
        for event in &self.end_events {
            event.notify();
        }
    }
}

impl Drop for ProducerPool {
    fn drop(&mut self) {
        self.end();
    }
}
struct InnerTopicProducer {
    topic: String,
    spu_pool: Arc<SpuPool>,
    partitioner: Box<dyn Partitioner + Send + Sync>,
    record_accumulator: RecordAccumulator,
    producer_pool: Arc<ProducerPool>,
}

impl InnerTopicProducer {
    /// Flush all the PartitionProducers and wait for them.
    async fn flush(&self) -> Result<()> {
        self.producer_pool.flush_all_batches().await?;
        Ok(())
    }

    async fn push_record(self: Arc<Self>, record: Record) -> Result<PushRecord> {
        let topics = self.spu_pool.metadata.topics();
        let topic_spec = topics
            .lookup_by_key(&self.topic)
            .await?
            .ok_or_else(|| FluvioError::TopicNotFound(self.topic.to_string()))?
            .spec;
        let partition_count = topic_spec.partitions();
        let partition_config = PartitionerConfig { partition_count };

        let key = record.key.as_ref().map(|k| k.as_ref());
        let value = record.value.as_ref();
        let partition = self.partitioner.partition(&partition_config, key, value);

        if let Some(error) = self.producer_pool.last_error(partition).await {
            return Err(error.into());
        }

        let push_record = self
            .record_accumulator
            .push_record(record, partition)
            .await?;

        Ok(push_record)
    }

    async fn clear_errors(&self) {
        self.producer_pool.clear_errors().await;
    }
}

cfg_if::cfg_if! {
    if #[cfg(feature = "smartengine")] {
        use fluvio_spu_schema::server::stream_fetch::{SmartModuleWasmCompressed, SmartModuleKind, LegacySmartModulePayload};
        use std::collections::BTreeMap;
        use fluvio_smartengine::SmartEngine;

        impl TopicProducer {
            fn init_engine(&mut self, smart_payload: LegacySmartModulePayload) -> Result<(), FluvioError> {
                let engine = SmartEngine::default();
                let  smartmodule = engine.create_module_from_payload(
                    smart_payload,
                    None).map_err(|e| FluvioError::Other(format!("SmartEngine - {:?}", e)))?;
                self.smartmodule_instance = Some(Arc::new(RwLock::new(smartmodule)));
                Ok(())
            }
            /// Adds a SmartModule filter to this TopicProducer
            pub fn with_filter<T: Into<Vec<u8>>>(
                mut self,
                filter: T,
                params: BTreeMap<String, String>,
            ) -> Result<Self, FluvioError> {
                let smart_payload = LegacySmartModulePayload {
                    wasm: SmartModuleWasmCompressed::Raw(filter.into()),
                    kind: SmartModuleKind::Filter,
                    params: params.into(),
                };
                self.init_engine(smart_payload)?;
                Ok(self)
            }

            /// Adds a SmartModule map to this TopicProducer
            pub fn with_map<T: Into<Vec<u8>>>(
                mut self,
                map: T,
                params: BTreeMap<String, String>,
            ) -> Result<Self, FluvioError> {
                let smart_payload = LegacySmartModulePayload {
                    wasm: SmartModuleWasmCompressed::Raw(map.into()),
                    kind: SmartModuleKind::Map,
                    params: params.into(),
                };
                self.init_engine(smart_payload)?;
                Ok(self)
            }

            /// Adds a SmartModule array_map to this TopicProducer
            pub fn with_array_map<T: Into<Vec<u8>>>(
                mut self,
                map: T,
                params: BTreeMap<String, String>,
            ) -> Result<Self, FluvioError> {
                let smart_payload = LegacySmartModulePayload {
                    wasm: SmartModuleWasmCompressed::Raw(map.into()),
                    kind: SmartModuleKind::ArrayMap,
                    params: params.into(),
                };

                self.init_engine(smart_payload)?;
                Ok(self)
            }

            /// Adds a SmartModule aggregate to this TopicProducer
            pub fn with_aggregate<T: Into<Vec<u8>>>(
                mut self,
                map: T,
                params: BTreeMap<String, String>,
                accumulator: Vec<u8>,
            ) -> Result<Self, FluvioError> {
                let smart_payload = LegacySmartModulePayload {
                    wasm: SmartModuleWasmCompressed::Raw(map.into()),
                    kind: SmartModuleKind::Aggregate { accumulator },
                    params: params.into(),
                };
                self.init_engine(smart_payload)?;
                Ok(self)
            }
        }
    }
}

impl TopicProducer {
    pub(crate) async fn new(
        topic: String,
        spu_pool: Arc<SpuPool>,
        config: TopicProducerConfig,
    ) -> Result<Self> {
        let partitioner = config.partitioner;
        let topics = spu_pool.metadata.topics();
        let topic_spec = topics
            .lookup_by_key(&topic)
            .await?
            .ok_or_else(|| FluvioError::TopicNotFound(topic.to_string()))?
            .spec;
        let partition_count = topic_spec.partitions();
        let record_accumulator = RecordAccumulator::new(config.batch_size, partition_count);
        let producer_pool = ProducerPool::shared(
            topic.clone(),
            spu_pool.clone(),
            record_accumulator.batches(),
            config.linger,
        );

        Ok(Self {
            inner: Arc::new(InnerTopicProducer {
                topic,
                spu_pool,
                partitioner,
                producer_pool,
                record_accumulator,
            }),
            #[cfg(feature = "smartengine")]
            smartmodule_instance: Default::default(),
        })
    }

    pub async fn flush(&self) -> Result<(), FluvioError> {
        self.inner.flush().await
    }

    /// Sends a key/value record to this producer's Topic.
    ///
    /// The partition that the record will be sent to is derived from the Key.
    ///
    /// # Example
    ///
    /// ```
    /// # use fluvio::{TopicProducer, FluvioError};
    /// # async fn example(producer: &TopicProducer) -> Result<(), FluvioError> {
    /// producer.send("Key", "Value").await?;
    /// # Ok(())
    /// # }
    /// ```
    #[instrument(
        skip(self, key, value),
        fields(topic = %self.inner.topic),
    )]
    pub async fn send<K, V>(&self, key: K, value: V) -> Result<ProduceOutput, FluvioError>
    where
        K: Into<RecordKey>,
        V: Into<RecordData>,
    {
        let record_key = key.into();
        let record_value = value.into();
        let record = Record::from((record_key, record_value));

        cfg_if::cfg_if! {
            if #[cfg(feature = "smartengine")] {
                let mut entries = vec![record];
                use dataplane::smartmodule::SmartModuleInput;
                use std::convert::TryFrom;

                if let Some(
                    smartmodule_instance_ref
                ) = &self.smartmodule_instance {
                    let mut smartengine = smartmodule_instance_ref.write().await;
                    let output = smartengine.process(SmartModuleInput::try_from(entries)?).map_err(|e| FluvioError::Other(format!("SmartEngine - {:?}", e)))?;
                    entries = output.successes;
                }
            } else {
                let  entries = vec![record];
                }
        }

        let mut results = ProduceOutput::default();
        for record in entries {
            let push_record = self.inner.clone().push_record(record).await?;
            results.add(push_record.future);
        }
        Ok(results)
    }

    #[instrument(
        skip(self, records),
        fields(topic = %self.inner.topic),
    )]
    pub async fn send_all<K, V, I>(&self, records: I) -> Result<Vec<ProduceOutput>, FluvioError>
    where
        K: Into<RecordKey>,
        V: Into<RecordData>,
        I: IntoIterator<Item = (K, V)>,
    {
        let mut results = vec![];
        for (key, value) in records {
            let produce_output = self.send(key, value).await?;
            results.push(produce_output);
        }

        Ok(results)
    }

    /// Clear partition producers errors in order to make partition producers available.
    /// This is needed once an error is present in order to send new records again.
    pub async fn clear_errors(&self) {
        self.inner.clear_errors().await;
    }
}