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
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
//! Fluent builder for `LaminarDB` construction.
#![allow(clippy::disallowed_types)] // cold path
use std::collections::HashMap;
use std::path::PathBuf;
use datafusion_expr::{AggregateUDF, ScalarUDF};
use laminar_core::streaming::{BackpressureStrategy, StreamCheckpointConfig};
use crate::config::LaminarConfig;
use crate::db::LaminarDB;
use crate::error::DbError;
use crate::profile::Profile;
/// Callback for registering custom connectors.
type ConnectorCallback = Box<dyn FnOnce(&laminar_connectors::registry::ConnectorRegistry) + Send>;
/// Fluent builder for constructing a [`LaminarDB`] instance.
///
/// # Example
///
/// ```rust,ignore
/// let db = LaminarDB::builder()
/// .config_var("KAFKA_BROKERS", "localhost:9092")
/// .buffer_size(131072)
/// .build()
/// .await?;
/// ```
pub struct LaminarDbBuilder {
config: LaminarConfig,
config_vars: HashMap<String, String>,
connector_callbacks: Vec<ConnectorCallback>,
profile: Profile,
profile_explicit: bool,
object_store_url: Option<String>,
object_store_options: HashMap<String, String>,
custom_udfs: Vec<ScalarUDF>,
custom_udafs: Vec<AggregateUDF>,
}
impl LaminarDbBuilder {
/// Create a new builder with default settings.
#[must_use]
pub fn new() -> Self {
Self {
config: LaminarConfig::default(),
config_vars: HashMap::new(),
connector_callbacks: Vec::new(),
profile: Profile::default(),
profile_explicit: false,
object_store_url: None,
object_store_options: HashMap::new(),
custom_udfs: Vec::new(),
custom_udafs: Vec::new(),
}
}
/// Set a config variable for `${VAR}` substitution in SQL.
#[must_use]
pub fn config_var(mut self, key: &str, value: &str) -> Self {
self.config_vars.insert(key.to_string(), value.to_string());
self
}
/// Set the default buffer size for streaming channels.
#[must_use]
pub fn buffer_size(mut self, size: usize) -> Self {
self.config.default_buffer_size = size;
self
}
/// Set the default backpressure strategy.
#[must_use]
pub fn backpressure(mut self, strategy: BackpressureStrategy) -> Self {
self.config.default_backpressure = strategy;
self
}
/// Set the storage directory for WAL and checkpoints.
#[must_use]
pub fn storage_dir(mut self, path: impl Into<PathBuf>) -> Self {
self.config.storage_dir = Some(path.into());
self
}
/// Set checkpoint configuration.
#[must_use]
pub fn checkpoint(mut self, config: StreamCheckpointConfig) -> Self {
self.config.checkpoint = Some(config);
self
}
/// Set the deployment profile.
///
/// See [`Profile`] for the available tiers.
#[must_use]
pub fn profile(mut self, profile: Profile) -> Self {
self.profile = profile;
self.profile_explicit = true;
self
}
/// Set the object-store URL for durable checkpoints.
///
/// Required when using [`Profile::Durable`] or
/// [`Profile::Delta`].
#[must_use]
pub fn object_store_url(mut self, url: impl Into<String>) -> Self {
self.object_store_url = Some(url.into());
self
}
/// Set explicit credential/config overrides for the object store.
///
/// Keys are backend-specific (e.g., `aws_access_key_id`, `aws_region`).
/// These supplement environment-variable-based credential resolution.
#[must_use]
pub fn object_store_options(mut self, opts: HashMap<String, String>) -> Self {
self.object_store_options = opts;
self
}
/// Set the S3 storage class tiering configuration.
#[must_use]
pub fn tiering(mut self, tiering: crate::config::TieringConfig) -> Self {
self.config.tiering = Some(tiering);
self
}
/// Set the end-to-end delivery guarantee for the pipeline.
#[must_use]
pub fn delivery_guarantee(
mut self,
guarantee: laminar_connectors::connector::DeliveryGuarantee,
) -> Self {
self.config.delivery_guarantee = guarantee;
self
}
/// Register a custom scalar UDF with the database.
///
/// The UDF will be available in SQL queries after `build()`.
///
/// # Example
///
/// ```rust,ignore
/// use datafusion_expr::ScalarUDF;
///
/// let db = LaminarDB::builder()
/// .register_udf(my_scalar_udf)
/// .build()
/// .await?;
/// ```
#[must_use]
pub fn register_udf(mut self, udf: ScalarUDF) -> Self {
self.custom_udfs.push(udf);
self
}
/// Register a custom aggregate UDF (UDAF) with the database.
///
/// The UDAF will be available in SQL queries after `build()`.
///
/// # Example
///
/// ```rust,ignore
/// use datafusion_expr::AggregateUDF;
///
/// let db = LaminarDB::builder()
/// .register_udaf(my_aggregate_udf)
/// .build()
/// .await?;
/// ```
#[must_use]
pub fn register_udaf(mut self, udaf: AggregateUDF) -> Self {
self.custom_udafs.push(udaf);
self
}
/// Source → coordinator channel capacity (default 64). Increase for
/// burst absorption at the cost of memory.
#[must_use]
pub fn pipeline_channel_capacity(mut self, capacity: usize) -> Self {
self.config.pipeline_channel_capacity = Some(capacity);
self
}
/// Micro-batch coalescing window (default 5ms for connectors, 0 for
/// embedded). Larger values amortize per-cycle SQL overhead.
#[must_use]
pub fn pipeline_batch_window(mut self, window: std::time::Duration) -> Self {
self.config.pipeline_batch_window = Some(window);
self
}
/// Max time draining the source channel per cycle, in nanoseconds
/// (default 1ms). Increase to process more messages per SQL execution.
#[must_use]
pub fn pipeline_drain_budget_ns(mut self, ns: u64) -> Self {
self.config.pipeline_drain_budget_ns = Some(ns);
self
}
/// Per-query execution budget in nanoseconds (default 8ms). When
/// exceeded, remaining queries are deferred to the next cycle.
#[must_use]
pub fn pipeline_query_budget_ns(mut self, ns: u64) -> Self {
self.config.pipeline_query_budget_ns = Some(ns);
self
}
/// Register custom connectors with the `ConnectorRegistry`.
///
/// The callback is invoked after the database is created and built-in
/// connectors are registered. Use it to add user-defined source/sink
/// implementations.
///
/// # Example
///
/// ```rust,ignore
/// let db = LaminarDB::builder()
/// .register_connector(|registry| {
/// registry.register_source("my-source", info, factory);
/// })
/// .build()
/// .await?;
/// ```
#[must_use]
pub fn register_connector(
mut self,
f: impl FnOnce(&laminar_connectors::registry::ConnectorRegistry) + Send + 'static,
) -> Self {
self.connector_callbacks.push(Box::new(f));
self
}
/// Build the `LaminarDB` instance.
///
/// # Errors
///
/// Returns `DbError` if database creation fails.
#[allow(clippy::unused_async)]
pub async fn build(mut self) -> Result<LaminarDB, DbError> {
// Forward object store settings into the config before profile detection.
self.config.object_store_url = self.object_store_url;
self.config.object_store_options = self.object_store_options;
// Auto-detect profile from config if not explicitly set.
if !self.profile_explicit {
self.profile = Profile::from_config(&self.config, false);
}
// Validate profile feature gates and config requirements.
self.profile
.validate_features()
.map_err(|e| DbError::Config(e.to_string()))?;
self.profile
.validate_config(&self.config, self.config.object_store_url.as_deref())
.map_err(|e| DbError::Config(e.to_string()))?;
// Apply profile defaults for fields the user hasn't set.
self.profile.apply_defaults(&mut self.config);
let db = LaminarDB::open_with_config_and_vars(self.config, self.config_vars)?;
for callback in self.connector_callbacks {
callback(db.connector_registry());
}
for udf in self.custom_udfs {
db.register_custom_udf(udf);
}
for udaf in self.custom_udafs {
db.register_custom_udaf(udaf);
}
Ok(db)
}
}
impl Default for LaminarDbBuilder {
fn default() -> Self {
Self::new()
}
}
impl std::fmt::Debug for LaminarDbBuilder {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("LaminarDbBuilder")
.field("config", &self.config)
.field("profile", &self.profile)
.field("profile_explicit", &self.profile_explicit)
.field("object_store_url", &self.object_store_url)
.field(
"object_store_options_count",
&self.object_store_options.len(),
)
.field("config_vars_count", &self.config_vars.len())
.field("connector_callbacks", &self.connector_callbacks.len())
.field("custom_udfs", &self.custom_udfs.len())
.field("custom_udafs", &self.custom_udafs.len())
.finish()
}
}
#[cfg(test)]
mod tests {
use super::*;
#[tokio::test]
async fn test_default_builder() {
let db = LaminarDbBuilder::new().build().await.unwrap();
assert!(!db.is_closed());
}
#[tokio::test]
async fn test_builder_with_config_vars() {
let db = LaminarDbBuilder::new()
.config_var("KAFKA_BROKERS", "localhost:9092")
.config_var("GROUP_ID", "test-group")
.build()
.await
.unwrap();
assert!(!db.is_closed());
}
#[tokio::test]
async fn test_builder_with_options() {
let db = LaminarDbBuilder::new()
.buffer_size(131_072)
.build()
.await
.unwrap();
assert!(!db.is_closed());
}
#[tokio::test]
async fn test_builder_from_laminardb() {
let db = LaminarDB::builder().build().await.unwrap();
assert!(!db.is_closed());
}
#[test]
fn test_builder_debug() {
let builder = LaminarDbBuilder::new().config_var("K", "V");
let debug = format!("{builder:?}");
assert!(debug.contains("LaminarDbBuilder"));
assert!(debug.contains("config_vars_count: 1"));
}
#[tokio::test]
async fn test_builder_register_udf() {
use std::any::Any;
use std::hash::{Hash, Hasher};
use arrow::datatypes::DataType;
use datafusion_expr::{
ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, TypeSignature, Volatility,
};
/// Trivial UDF that returns 42.
#[derive(Debug)]
struct FortyTwo {
signature: Signature,
}
impl FortyTwo {
fn new() -> Self {
Self {
signature: Signature::new(TypeSignature::Nullary, Volatility::Immutable),
}
}
}
impl PartialEq for FortyTwo {
fn eq(&self, _: &Self) -> bool {
true
}
}
impl Eq for FortyTwo {}
impl Hash for FortyTwo {
fn hash<H: Hasher>(&self, state: &mut H) {
"forty_two".hash(state);
}
}
impl ScalarUDFImpl for FortyTwo {
fn as_any(&self) -> &dyn Any {
self
}
fn name(&self) -> &'static str {
"forty_two"
}
fn signature(&self) -> &Signature {
&self.signature
}
fn return_type(&self, _: &[DataType]) -> datafusion_common::Result<DataType> {
Ok(DataType::Int64)
}
fn invoke_with_args(
&self,
_args: ScalarFunctionArgs,
) -> datafusion_common::Result<ColumnarValue> {
Ok(ColumnarValue::Scalar(
datafusion_common::ScalarValue::Int64(Some(42)),
))
}
}
let udf = ScalarUDF::new_from_impl(FortyTwo::new());
let db = LaminarDB::builder()
.register_udf(udf)
.build()
.await
.unwrap();
// Verify the UDF is queryable
let result = db.execute("SELECT forty_two()").await;
assert!(result.is_ok(), "UDF should be callable: {result:?}");
}
}