fluss-rs 0.1.0

The official rust client of Apache Fluss (Incubating)
Documentation
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
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you 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.

use crate::client::{RowBytes, WriteFormat, WriteRecord, WriteResultFuture, WriterClient};
use crate::error::Error::{IllegalArgument, UnexpectedError};
use crate::error::Result;
use crate::metadata::{RowType, TableInfo, TablePath};
use crate::row::InternalRow;
use crate::row::encode::{KeyEncoder, KeyEncoderFactory, RowEncoder, RowEncoderFactory};
use crate::row::field_getter::FieldGetter;
use std::sync::{Arc, Mutex};

use crate::client::table::partition_getter::{PartitionGetter, get_physical_path};
use bitvec::prelude::bitvec;
use bytes::Bytes;

#[allow(dead_code)]
pub struct TableUpsert {
    table_path: TablePath,
    table_info: TableInfo,
    writer_client: Arc<WriterClient>,
    target_columns: Option<Arc<Vec<usize>>>,
}

#[allow(dead_code)]
impl TableUpsert {
    pub fn new(
        table_path: TablePath,
        table_info: TableInfo,
        writer_client: Arc<WriterClient>,
    ) -> Self {
        Self {
            table_path,
            table_info,
            writer_client,
            target_columns: None,
        }
    }

    pub fn partial_update(&self, target_columns: Option<Vec<usize>>) -> Result<Self> {
        if let Some(columns) = &target_columns {
            let num_columns = self.table_info.row_type().fields().len();

            if let Some(&invalid_column) = columns.iter().find(|&&col| col >= num_columns) {
                return Err(IllegalArgument {
                    message: format!(
                        "Invalid target column index: {invalid_column} for table {}. The table only has {num_columns} columns.",
                        self.table_path
                    ),
                });
            }
        }

        Ok(Self {
            table_path: self.table_path.clone(),
            table_info: self.table_info.clone(),
            writer_client: self.writer_client.clone(),
            target_columns: target_columns.map(Arc::new),
        })
    }

    pub fn partial_update_with_column_names(&self, target_column_names: &[&str]) -> Result<Self> {
        let row_type = self.table_info.row_type();
        let col_indices: Vec<(&str, Option<usize>)> = target_column_names
            .iter()
            .map(|col_name| (*col_name, row_type.get_field_index(col_name)))
            .collect();

        if let Some((missing_name, _)) = col_indices.iter().find(|(_, ix)| ix.is_none()) {
            return Err(IllegalArgument {
                message: format!(
                    "Cannot find target column `{}` for table {}.",
                    missing_name, self.table_path
                ),
            });
        }

        let valid_col_indices: Vec<usize> = col_indices
            .into_iter()
            .map(|(_, index)| index.unwrap())
            .collect();

        self.partial_update(Some(valid_col_indices))
    }

    pub fn create_writer(&self) -> Result<UpsertWriter> {
        UpsertWriterFactory::create(
            Arc::new(self.table_path.clone()),
            Arc::new(self.table_info.clone()),
            self.target_columns.clone(),
            Arc::clone(&self.writer_client),
        )
    }
}

pub struct UpsertWriter {
    table_path: Arc<TablePath>,
    writer_client: Arc<WriterClient>,
    partition_field_getter: Option<PartitionGetter>,
    primary_key_encoder: Mutex<Box<dyn KeyEncoder>>,
    target_columns: Option<Arc<Vec<usize>>>,
    // Use primary key encoder as bucket key encoder when None
    bucket_key_encoder: Option<Mutex<Box<dyn KeyEncoder>>>,
    write_format: WriteFormat,
    row_encoder: Mutex<Box<dyn RowEncoder>>,
    field_getters: Box<[FieldGetter]>,
    table_info: Arc<TableInfo>,
}

struct UpsertWriterFactory;

impl UpsertWriterFactory {
    pub fn create(
        table_path: Arc<TablePath>,
        table_info: Arc<TableInfo>,
        partial_update_columns: Option<Arc<Vec<usize>>>,
        writer_client: Arc<WriterClient>,
    ) -> Result<UpsertWriter> {
        let data_lake_format = &table_info.table_config.get_datalake_format()?;
        let row_type = table_info.row_type();
        let physical_pks = table_info.get_physical_primary_keys();

        let names = table_info.get_schema().auto_increment_col_names();

        Self::sanity_check(
            row_type,
            &table_info.primary_keys,
            names,
            &partial_update_columns,
        )?;

        let primary_key_encoder = KeyEncoderFactory::of(row_type, physical_pks, data_lake_format)?;
        let bucket_key_encoder = if !table_info.is_default_bucket_key() {
            Some(KeyEncoderFactory::of(
                row_type,
                table_info.get_bucket_keys(),
                data_lake_format,
            )?)
        } else {
            // Defaults to using primary key encoder when None for bucket key
            None
        };

        let kv_format = table_info.get_table_config().get_kv_format()?;
        let write_format = WriteFormat::from_kv_format(&kv_format)?;

        let field_getters = FieldGetter::create_field_getters(row_type);

        let partition_field_getter = if table_info.is_partitioned() {
            Some(PartitionGetter::new(
                row_type,
                Arc::clone(table_info.get_partition_keys()),
            )?)
        } else {
            None
        };

        Ok(UpsertWriter {
            table_path,
            partition_field_getter,
            writer_client,
            primary_key_encoder: Mutex::new(primary_key_encoder),
            target_columns: partial_update_columns,
            bucket_key_encoder: bucket_key_encoder.map(Mutex::new),
            write_format,
            row_encoder: Mutex::new(Box::new(RowEncoderFactory::create(
                kv_format,
                row_type.clone(),
            )?)),
            field_getters,
            table_info: table_info.clone(),
        })
    }

    #[allow(dead_code)]
    fn sanity_check(
        row_type: &RowType,
        primary_keys: &Vec<String>,
        auto_increment_col_names: &Vec<String>,
        target_columns: &Option<Arc<Vec<usize>>>,
    ) -> Result<()> {
        if target_columns.is_none() {
            if !auto_increment_col_names.is_empty() {
                return Err(IllegalArgument {
                    message: format!(
                        "This table has auto increment column {}. Explicitly specifying values for an auto increment column is not allowed. Please Specify non-auto-increment columns as target columns using partialUpdate first.",
                        auto_increment_col_names.join(", ")
                    ),
                });
            }
            return Ok(());
        }

        let field_count = row_type.fields().len();

        let mut target_column_set = bitvec![0; field_count];

        let columns = target_columns.as_ref().unwrap().as_ref();

        for &target_index in columns {
            target_column_set.set(target_index, true);
        }

        let mut pk_column_set = bitvec![0; field_count];

        // check the target columns contains the primary key
        for primary_key in primary_keys {
            let pk_index = row_type.get_field_index(primary_key.as_str());
            match pk_index {
                Some(pk_index) => {
                    if !target_column_set[pk_index] {
                        return Err(IllegalArgument {
                            message: format!(
                                "The target write columns {} must contain the primary key columns {}",
                                row_type.project(columns)?.get_field_names().join(", "),
                                primary_keys.join(", ")
                            ),
                        });
                    }
                    pk_column_set.set(pk_index, true);
                }
                None => {
                    return Err(IllegalArgument {
                        message: format!(
                            "The specified primary key {primary_key} is not in row type {row_type}"
                        ),
                    });
                }
            }
        }

        let mut auto_increment_column_set = bitvec![0; field_count];
        // explicitly specifying values for an auto increment column is not allowed
        for auto_increment_col_name in auto_increment_col_names {
            let auto_increment_field_index =
                row_type.get_field_index(auto_increment_col_name.as_str());

            if let Some(index) = auto_increment_field_index {
                if target_column_set[index] {
                    return Err(IllegalArgument {
                        message: format!(
                            "Explicitly specifying values for the auto increment column {auto_increment_col_name} is not allowed."
                        ),
                    });
                }

                auto_increment_column_set.set(index, true);
            }
        }

        // check the columns not in targetColumns should be nullable
        for i in 0..field_count {
            // column not in primary key and not in auto increment column
            if !pk_column_set[i] && !auto_increment_column_set[i] {
                // the column should be nullable
                if !row_type.fields().get(i).unwrap().data_type.is_nullable() {
                    return Err(IllegalArgument {
                        message: format!(
                            "Partial Update requires all columns except primary key to be nullable, but column {} is NOT NULL.",
                            row_type.fields().get(i).unwrap().name()
                        ),
                    });
                }
            }
        }

        Ok(())
    }
}

impl UpsertWriter {
    fn check_field_count<R: InternalRow>(&self, row: &R) -> Result<()> {
        let expected = self.table_info.get_row_type().fields().len();
        if row.get_field_count() != expected {
            return Err(IllegalArgument {
                message: format!(
                    "The field count of the row does not match the table schema. Expected: {}, Actual: {}",
                    expected,
                    row.get_field_count()
                ),
            });
        }
        Ok(())
    }

    fn get_keys(&self, row: &dyn InternalRow) -> Result<(Bytes, Option<Bytes>)> {
        let key = self
            .primary_key_encoder
            .lock()
            .map_err(|e| UnexpectedError {
                message: format!("primary_key_encoder lock poisoned: {e}"),
                source: None,
            })?
            .encode_key(row)?;
        let bucket_key = match &self.bucket_key_encoder {
            Some(encoder) => Some(
                encoder
                    .lock()
                    .map_err(|e| UnexpectedError {
                        message: format!("bucket_key_encoder lock poisoned: {e}"),
                        source: None,
                    })?
                    .encode_key(row)?,
            ),
            None => Some(key.clone()),
        };
        Ok((key, bucket_key))
    }

    fn encode_row<R: InternalRow>(&self, row: &R) -> Result<Bytes> {
        let mut encoder = self.row_encoder.lock().map_err(|e| UnexpectedError {
            message: format!("row_encoder lock poisoned: {e}"),
            source: None,
        })?;
        encoder.start_new_row()?;
        for (pos, field_getter) in self.field_getters.iter().enumerate() {
            let datum = field_getter.get_field(row)?;
            encoder.encode_field(pos, datum)?;
        }
        encoder.finish_row()
    }

    /// Flush data written that have not yet been sent to the server, forcing the client to send the
    /// requests to server and blocks on the completion of the requests associated with these
    /// records. A request is considered completed when it is successfully acknowledged according to
    /// the CLIENT_WRITER_ACKS configuration option you have specified or else it
    /// results in an error.
    pub async fn flush(&self) -> Result<()> {
        self.writer_client.flush().await
    }

    /// Inserts row into Fluss table if they do not already exist, or updates them if they do exist.
    ///
    /// This method returns a [`WriteResultFuture`] immediately after queueing the write,
    /// enabling fire-and-forget semantics for efficient batching.
    ///
    /// # Arguments
    /// * row - the row to upsert.
    ///
    /// # Returns
    /// A [`WriteResultFuture`] that can be awaited to wait for server acknowledgment,
    /// or dropped for fire-and-forget behavior (use `flush()` to ensure delivery).
    pub fn upsert<R: InternalRow>(&self, row: &R) -> Result<WriteResultFuture> {
        self.check_field_count(row)?;

        let (key, bucket_key) = self.get_keys(row)?;

        let row_bytes: RowBytes<'_> = match row.as_encoded_bytes(self.write_format) {
            Some(bytes) => RowBytes::Borrowed(bytes),
            None => RowBytes::Owned(self.encode_row(row)?),
        };

        let write_record = WriteRecord::for_upsert(
            Arc::clone(&self.table_info),
            Arc::new(get_physical_path(
                &self.table_path,
                self.partition_field_getter.as_ref(),
                row,
            )?),
            self.table_info.schema_id,
            key,
            bucket_key,
            self.write_format,
            self.target_columns.clone(),
            Some(row_bytes),
        );

        let result_handle = self.writer_client.send(&write_record)?;
        Ok(WriteResultFuture::new(result_handle))
    }

    /// Delete certain row by the input row in Fluss table, the input row must contain the primary
    /// key.
    ///
    /// This method returns a [`WriteResultFuture`] immediately after queueing the delete,
    /// enabling fire-and-forget semantics for efficient batching.
    ///
    /// # Arguments
    /// * row - the row to delete (must contain the primary key fields).
    ///
    /// # Returns
    /// A [`WriteResultFuture`] that can be awaited to wait for server acknowledgment,
    /// or dropped for fire-and-forget behavior (use `flush()` to ensure delivery).
    pub fn delete<R: InternalRow>(&self, row: &R) -> Result<WriteResultFuture> {
        self.check_field_count(row)?;

        let (key, bucket_key) = self.get_keys(row)?;

        let write_record = WriteRecord::for_upsert(
            Arc::clone(&self.table_info),
            Arc::new(get_physical_path(
                &self.table_path,
                self.partition_field_getter.as_ref(),
                row,
            )?),
            self.table_info.schema_id,
            key,
            bucket_key,
            self.write_format,
            self.target_columns.clone(),
            None,
        );

        let result_handle = self.writer_client.send(&write_record)?;
        Ok(WriteResultFuture::new(result_handle))
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::metadata::{DataField, DataTypes};

    #[test]
    fn sanity_check() {
        // No target columns specified but table has auto-increment column
        let fields = vec![
            DataField::new("id", DataTypes::int().as_non_nullable(), None),
            DataField::new("name", DataTypes::string(), None),
        ];
        let row_type = RowType::new(fields);
        let primary_keys = vec!["id".to_string()];
        let auto_increment_col_names = vec!["id".to_string()];
        let target_columns = None;

        let result = UpsertWriterFactory::sanity_check(
            &row_type,
            &primary_keys,
            &auto_increment_col_names,
            &target_columns,
        );

        assert!(result.unwrap_err().to_string().contains(
            "This table has auto increment column id. Explicitly specifying values for an auto increment column is not allowed. Please Specify non-auto-increment columns as target columns using partialUpdate first."
        ));

        // Target columns do not contain primary key
        let fields = vec![
            DataField::new("id", DataTypes::int().as_non_nullable(), None),
            DataField::new("name", DataTypes::string(), None),
            DataField::new("value", DataTypes::int(), None),
        ];
        let row_type = RowType::new(fields);
        let primary_keys = vec!["id".to_string()];
        let auto_increment_col_names = vec![];
        let target_columns = Some(Arc::new(vec![1usize]));

        let result = UpsertWriterFactory::sanity_check(
            &row_type,
            &primary_keys,
            &auto_increment_col_names,
            &target_columns,
        );

        assert!(
            result
                .unwrap_err()
                .to_string()
                .contains("The target write columns name must contain the primary key columns id")
        );

        // Primary key column not found in row type
        let fields = vec![
            DataField::new("id", DataTypes::int().as_non_nullable(), None),
            DataField::new("name", DataTypes::string(), None),
        ];
        let row_type = RowType::new(fields);
        let primary_keys = vec!["nonexistent_pk".to_string()];
        let auto_increment_col_names = vec![];
        let target_columns = Some(Arc::new(vec![0usize, 1]));

        let result = UpsertWriterFactory::sanity_check(
            &row_type,
            &primary_keys,
            &auto_increment_col_names,
            &target_columns,
        );

        assert!(
            result
                .unwrap_err()
                .to_string()
                .contains("The specified primary key nonexistent_pk is not in row type")
        );

        // Target columns include auto-increment column
        let fields = vec![
            DataField::new("id", DataTypes::int().as_non_nullable(), None),
            DataField::new("seq", DataTypes::bigint().as_non_nullable(), None),
            DataField::new("name", DataTypes::string(), None),
        ];
        let row_type = RowType::new(fields);
        let primary_keys = vec!["id".to_string()];
        let auto_increment_col_names = vec!["seq".to_string()];
        let target_columns = Some(Arc::new(vec![0usize, 1, 2]));

        let result = UpsertWriterFactory::sanity_check(
            &row_type,
            &primary_keys,
            &auto_increment_col_names,
            &target_columns,
        );

        assert!(result.unwrap_err().to_string().contains(
            "Explicitly specifying values for the auto increment column seq is not allowed."
        ));

        // Non-nullable column not in target columns (partial update requires nullable)
        let fields = vec![
            DataField::new("id", DataTypes::int().as_non_nullable(), None),
            DataField::new(
                "required_field",
                DataTypes::string().as_non_nullable(),
                None,
            ),
            DataField::new("optional_field", DataTypes::int(), None),
        ];
        let row_type = RowType::new(fields);
        let primary_keys = vec!["id".to_string()];
        let auto_increment_col_names = vec![];
        let target_columns = Some(Arc::new(vec![0usize]));

        let result = UpsertWriterFactory::sanity_check(
            &row_type,
            &primary_keys,
            &auto_increment_col_names,
            &target_columns,
        );

        assert!(result.unwrap_err().to_string().contains(
            "Partial Update requires all columns except primary key to be nullable, but column required_field is NOT NULL."
        ));
    }
}

/// The result of upserting a record
/// Currently this is an empty struct to allow for compatible evolution in the future
#[derive(Default)]
#[allow(dead_code)]
pub struct UpsertResult;

/// The result of deleting a record
/// Currently this is an empty struct to allow for compatible evolution in the future
#[derive(Default)]
#[allow(dead_code)]
pub struct DeleteResult;