leekeiabstraction commented on code in PR #452: URL: https://github.com/apache/fluss-rust/pull/452#discussion_r3068072836
########## bindings/elixir/native/fluss_nif/src/row_convert.rs: ########## @@ -0,0 +1,263 @@ +// 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 std::str::FromStr; + +use fluss::metadata::{Column, DataType}; +use fluss::row::{Date, Decimal, GenericRow, InternalRow, Time, TimestampLtz, TimestampNtz}; +use rustler::types::binary::NewBinary; +use rustler::{Encoder, Env, Term}; + +use crate::atoms; + +/// Convert column names to BEAM atoms for use as map keys. +/// +/// Note: BEAM atoms are never garbage-collected. This is safe because column +/// names come from server-defined table schemas (bounded set), not arbitrary +/// user input. The BEAM deduplicates atoms, so repeated calls with the same +/// column names do not grow the atom table. +pub fn intern_column_atoms<'a>(env: Env<'a>, columns: &[Column]) -> Vec<rustler::Atom> { + columns + .iter() + .map(|col| rustler::Atom::from_str(env, col.name()).expect("valid atom")) + .collect() +} + +pub fn row_to_term<'a>( + env: Env<'a>, + row: &dyn InternalRow, + columns: &[Column], + column_atoms: &[rustler::Atom], +) -> Result<Term<'a>, String> { + let pairs: Vec<(Term<'a>, Term<'a>)> = columns + .iter() + .enumerate() + .map(|(i, col)| { + let key = column_atoms[i].encode(env); + let value = field_to_term(env, row, i, col.data_type())?; + Ok((key, value)) + }) + .collect::<Result<_, String>>()?; + Term::map_from_pairs(env, &pairs).map_err(|_| "failed to create map".to_string()) +} + +fn field_to_term<'a>( + env: Env<'a>, + row: &dyn InternalRow, + pos: usize, + data_type: &DataType, +) -> Result<Term<'a>, String> { + if row.is_null_at(pos).map_err(|e| e.to_string())? { + return Ok(atoms::nil().encode(env)); + } + + match data_type { + DataType::Boolean(_) => { + let v = row.get_boolean(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::TinyInt(_) => { + let v = row.get_byte(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::SmallInt(_) => { + let v = row.get_short(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Int(_) => { + let v = row.get_int(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::BigInt(_) => { + let v = row.get_long(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Float(_) => { + let v = row.get_float(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Double(_) => { + let v = row.get_double(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::String(_) => { + let v = row.get_string(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Char(ct) => { + let v = row + .get_char(pos, ct.length() as usize) + .map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Bytes(_) => { + let v = row.get_bytes(pos).map_err(|e| e.to_string())?; + let mut bin = NewBinary::new(env, v.len()); + bin.as_mut_slice().copy_from_slice(v); + let binary: rustler::Binary = bin.into(); + Ok(binary.encode(env)) + } + DataType::Binary(bt) => { + let v = row + .get_binary(pos, bt.length()) + .map_err(|e| e.to_string())?; + let mut bin = NewBinary::new(env, v.len()); + bin.as_mut_slice().copy_from_slice(v); + let binary: rustler::Binary = bin.into(); + Ok(binary.encode(env)) + } + DataType::Date(_) => { + let v = row.get_date(pos).map_err(|e| e.to_string())?; + Ok(v.get_inner().encode(env)) + } + DataType::Time(_) => { + let v = row.get_time(pos).map_err(|e| e.to_string())?; + Ok(v.get_inner().encode(env)) + } + DataType::Timestamp(ts) => { + let v = row + .get_timestamp_ntz(pos, ts.precision()) + .map_err(|e| e.to_string())?; + Ok((v.get_millisecond(), v.get_nano_of_millisecond()).encode(env)) + } + DataType::TimestampLTz(ts) => { + let v = row + .get_timestamp_ltz(pos, ts.precision()) + .map_err(|e| e.to_string())?; + Ok((v.get_epoch_millisecond(), v.get_nano_of_millisecond()).encode(env)) + } + DataType::Decimal(dt) => { + let v = row + .get_decimal(pos, dt.precision() as usize, dt.scale() as usize) + .map_err(|e| e.to_string())?; + Ok(v.to_string().encode(env)) + } + _ => Err(format!("unsupported data type: {data_type:?}")), + } +} + +pub fn term_to_row<'a>( + env: Env<'a>, + values: Term<'a>, + columns: &[Column], +) -> Result<GenericRow<'static>, String> { + let list: Vec<Term<'a>> = values + .decode() + .map_err(|_| "expected a list of values".to_string())?; + if list.len() != columns.len() { + return Err(format!( + "expected {} values, got {}", + columns.len(), + list.len() + )); + } + + let mut row = GenericRow::new(columns.len()); + for (i, (term, col)) in list.iter().zip(columns.iter()).enumerate() { + if term.is_atom() + && let Ok(atom) = term.decode::<rustler::Atom>() + && atom == atoms::nil() + { + continue; // leave as null + } + set_field_from_term(env, &mut row, i, *term, col.data_type())?; + } + Ok(row) +} + +fn set_field_from_term<'a>( + _env: Env<'a>, + row: &mut GenericRow<'static>, + pos: usize, + term: Term<'a>, + data_type: &DataType, +) -> Result<(), String> { + match data_type { + DataType::Boolean(_) => { + let v: bool = term.decode().map_err(|_| "expected boolean")?; + row.set_field(pos, v); + } + DataType::TinyInt(_) => { + let v: i8 = term.decode().map_err(|_| "expected integer for tinyint")?; + row.set_field(pos, v); + } + DataType::SmallInt(_) => { + let v: i16 = term.decode().map_err(|_| "expected integer for smallint")?; + row.set_field(pos, v); + } Review Comment: Should we decode as i64 first and then perform range check? Otherwise the message `expected integer` can be confusing for user a they may have passed in integer that is out of i8 or i16 range. ########## bindings/elixir/native/fluss_nif/src/log_scanner.rs: ########## @@ -0,0 +1,169 @@ +// 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::RUNTIME; +use crate::atoms::{self, to_nif_err}; +use crate::row_convert; +use crate::table::TableResource; +use fluss::client::LogScanner; +use fluss::metadata::Column; +use fluss::record::ChangeType; +use rustler::env::OwnedEnv; +use rustler::types::LocalPid; +use rustler::{Atom, Encoder, Env, ResourceArc}; +use std::collections::HashMap; +use std::time::Duration; + +pub struct LogScannerResource { + pub scanner: LogScanner, + pub columns: Vec<Column>, +} + +impl std::panic::RefUnwindSafe for LogScannerResource {} + +#[rustler::resource_impl] +impl rustler::Resource for LogScannerResource {} + +#[rustler::nif] +fn log_scanner_new( + table: ResourceArc<TableResource>, +) -> Result<ResourceArc<LogScannerResource>, rustler::Error> { + let _guard = RUNTIME.enter(); + let (scanner, columns) = table.with_table(|t| { + let scanner = t.new_scan().create_log_scanner().map_err(to_nif_err)?; + Ok((scanner, t.get_table_info().schema.columns().to_vec())) + })?; + Ok(ResourceArc::new(LogScannerResource { scanner, columns })) +} + +#[rustler::nif(schedule = "DirtyIo")] +fn log_scanner_subscribe( + scanner: ResourceArc<LogScannerResource>, + bucket: i32, + offset: i64, +) -> Result<Atom, rustler::Error> { + RUNTIME + .block_on(scanner.scanner.subscribe(bucket, offset)) + .map_err(to_nif_err)?; + Ok(atoms::ok()) +} + +#[rustler::nif(schedule = "DirtyIo")] +fn log_scanner_subscribe_buckets( + scanner: ResourceArc<LogScannerResource>, + bucket_offsets: Vec<(i32, i64)>, +) -> Result<Atom, rustler::Error> { + let map: HashMap<i32, i64> = bucket_offsets.into_iter().collect(); + RUNTIME + .block_on(scanner.scanner.subscribe_buckets(&map)) + .map_err(to_nif_err)?; + Ok(atoms::ok()) +} + +#[rustler::nif(schedule = "DirtyIo")] +fn log_scanner_unsubscribe( + scanner: ResourceArc<LogScannerResource>, + bucket: i32, +) -> Result<Atom, rustler::Error> { + RUNTIME + .block_on(scanner.scanner.unsubscribe(bucket)) + .map_err(to_nif_err)?; + Ok(atoms::ok()) +} + +#[rustler::nif] +fn log_scanner_poll(env: Env, scanner: ResourceArc<LogScannerResource>, timeout_ms: u64) -> Atom { + let pid = env.pid(); + let scanner = scanner.clone(); + + RUNTIME.spawn(async move { + let result = scanner + .scanner + .poll(Duration::from_millis(timeout_ms)) + .await; + send_poll_result(&pid, result, &scanner.columns); + }); + + atoms::ok() +} + +fn send_poll_result( + pid: &LocalPid, + result: Result<fluss::record::ScanRecords, fluss::error::Error>, + columns: &[Column], +) { + let mut msg_env = OwnedEnv::new(); + + match result { + Ok(scan_records) => { + let _ = msg_env.send_and_clear(pid, |env| { + match encode_scan_records(env, scan_records, columns) { + Ok(records) => (atoms::fluss_records(), records).encode(env), + Err(e) => (atoms::fluss_poll_error(), e).encode(env), + } + }); + } + Err(e) => { + let _ = msg_env.send_and_clear(pid, |env| { + (atoms::fluss_poll_error(), e.to_string()).encode(env) + }); + } + } +} + +fn encode_scan_records<'a>( + env: Env<'a>, + scan_records: fluss::record::ScanRecords, + columns: &[Column], +) -> Result<rustler::Term<'a>, String> { + let column_atoms = row_convert::intern_column_atoms(env, columns); + let mut result = Vec::new(); + + for record in scan_records { + let row_map = row_convert::row_to_term(env, record.row(), columns, &column_atoms) + .map_err(|e| format!("failed to convert row at offset {}: {e}", record.offset()))?; Review Comment: Looking at this, we are also abandoning the remaining of the batch. I think we should call that out in the error as well? This is so that user is aware that batch is discarded, otherwise when they see this error they might handle it with the assumption that only one row has failed. Does this get translated to `fluss_poll_error`? Is this distinguishable between vs. retryable error e.g. timeout? ########## bindings/elixir/native/fluss_nif/src/row_convert.rs: ########## @@ -0,0 +1,263 @@ +// 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 std::str::FromStr; + +use fluss::metadata::{Column, DataType}; +use fluss::row::{Date, Decimal, GenericRow, InternalRow, Time, TimestampLtz, TimestampNtz}; +use rustler::types::binary::NewBinary; +use rustler::{Encoder, Env, Term}; + +use crate::atoms; + +/// Convert column names to BEAM atoms for use as map keys. +/// +/// Note: BEAM atoms are never garbage-collected. This is safe because column +/// names come from server-defined table schemas (bounded set), not arbitrary +/// user input. The BEAM deduplicates atoms, so repeated calls with the same +/// column names do not grow the atom table. +pub fn intern_column_atoms<'a>(env: Env<'a>, columns: &[Column]) -> Vec<rustler::Atom> { + columns + .iter() + .map(|col| rustler::Atom::from_str(env, col.name()).expect("valid atom")) + .collect() +} + +pub fn row_to_term<'a>( + env: Env<'a>, + row: &dyn InternalRow, + columns: &[Column], + column_atoms: &[rustler::Atom], +) -> Result<Term<'a>, String> { + let pairs: Vec<(Term<'a>, Term<'a>)> = columns + .iter() + .enumerate() + .map(|(i, col)| { + let key = column_atoms[i].encode(env); + let value = field_to_term(env, row, i, col.data_type())?; + Ok((key, value)) + }) + .collect::<Result<_, String>>()?; + Term::map_from_pairs(env, &pairs).map_err(|_| "failed to create map".to_string()) +} + +fn field_to_term<'a>( + env: Env<'a>, + row: &dyn InternalRow, + pos: usize, + data_type: &DataType, +) -> Result<Term<'a>, String> { + if row.is_null_at(pos).map_err(|e| e.to_string())? { + return Ok(atoms::nil().encode(env)); + } + + match data_type { + DataType::Boolean(_) => { + let v = row.get_boolean(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::TinyInt(_) => { + let v = row.get_byte(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::SmallInt(_) => { + let v = row.get_short(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Int(_) => { + let v = row.get_int(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::BigInt(_) => { + let v = row.get_long(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Float(_) => { + let v = row.get_float(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Double(_) => { + let v = row.get_double(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::String(_) => { + let v = row.get_string(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Char(ct) => { + let v = row + .get_char(pos, ct.length() as usize) + .map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Bytes(_) => { + let v = row.get_bytes(pos).map_err(|e| e.to_string())?; + let mut bin = NewBinary::new(env, v.len()); + bin.as_mut_slice().copy_from_slice(v); + let binary: rustler::Binary = bin.into(); + Ok(binary.encode(env)) + } + DataType::Binary(bt) => { + let v = row + .get_binary(pos, bt.length()) + .map_err(|e| e.to_string())?; + let mut bin = NewBinary::new(env, v.len()); + bin.as_mut_slice().copy_from_slice(v); + let binary: rustler::Binary = bin.into(); + Ok(binary.encode(env)) + } + DataType::Date(_) => { + let v = row.get_date(pos).map_err(|e| e.to_string())?; + Ok(v.get_inner().encode(env)) + } + DataType::Time(_) => { + let v = row.get_time(pos).map_err(|e| e.to_string())?; + Ok(v.get_inner().encode(env)) + } + DataType::Timestamp(ts) => { + let v = row + .get_timestamp_ntz(pos, ts.precision()) + .map_err(|e| e.to_string())?; + Ok((v.get_millisecond(), v.get_nano_of_millisecond()).encode(env)) + } + DataType::TimestampLTz(ts) => { + let v = row + .get_timestamp_ltz(pos, ts.precision()) + .map_err(|e| e.to_string())?; + Ok((v.get_epoch_millisecond(), v.get_nano_of_millisecond()).encode(env)) + } + DataType::Decimal(dt) => { + let v = row + .get_decimal(pos, dt.precision() as usize, dt.scale() as usize) + .map_err(|e| e.to_string())?; + Ok(v.to_string().encode(env)) + } + _ => Err(format!("unsupported data type: {data_type:?}")), + } +} + +pub fn term_to_row<'a>( + env: Env<'a>, + values: Term<'a>, + columns: &[Column], +) -> Result<GenericRow<'static>, String> { + let list: Vec<Term<'a>> = values + .decode() + .map_err(|_| "expected a list of values".to_string())?; + if list.len() != columns.len() { + return Err(format!( + "expected {} values, got {}", + columns.len(), + list.len() + )); + } + + let mut row = GenericRow::new(columns.len()); + for (i, (term, col)) in list.iter().zip(columns.iter()).enumerate() { + if term.is_atom() + && let Ok(atom) = term.decode::<rustler::Atom>() + && atom == atoms::nil() + { + continue; // leave as null + } + set_field_from_term(env, &mut row, i, *term, col.data_type())?; + } + Ok(row) +} + +fn set_field_from_term<'a>( + _env: Env<'a>, + row: &mut GenericRow<'static>, + pos: usize, + term: Term<'a>, + data_type: &DataType, +) -> Result<(), String> { + match data_type { + DataType::Boolean(_) => { + let v: bool = term.decode().map_err(|_| "expected boolean")?; + row.set_field(pos, v); + } + DataType::TinyInt(_) => { + let v: i8 = term.decode().map_err(|_| "expected integer for tinyint")?; + row.set_field(pos, v); + } + DataType::SmallInt(_) => { + let v: i16 = term.decode().map_err(|_| "expected integer for smallint")?; + row.set_field(pos, v); + } + DataType::Int(_) => { + let v: i32 = term.decode().map_err(|_| "expected integer")?; + row.set_field(pos, v); + } + DataType::BigInt(_) => { + let v: i64 = term.decode().map_err(|_| "expected integer")?; + row.set_field(pos, v); + } + DataType::Date(_) => { + let v: i32 = term + .decode() + .map_err(|_| "expected integer (days since epoch)")?; + row.set_field(pos, Date::new(v)); + } + DataType::Time(_) => { + let v: i32 = term + .decode() + .map_err(|_| "expected integer (millis since midnight)")?; + row.set_field(pos, Time::new(v)); + } + DataType::Timestamp(_) => { + let (millis, nanos): (i64, i32) = term + .decode() + .map_err(|_| "expected {millis, nanos} tuple for timestamp")?; + let ts = TimestampNtz::from_millis_nanos(millis, nanos).map_err(|e| e.to_string())?; + row.set_field(pos, ts); + } + DataType::TimestampLTz(_) => { + let (millis, nanos): (i64, i32) = term + .decode() + .map_err(|_| "expected {millis, nanos} tuple for timestamp_ltz")?; + let ts = TimestampLtz::from_millis_nanos(millis, nanos).map_err(|e| e.to_string())?; + row.set_field(pos, ts); + } + DataType::Float(_) => { + let v: f64 = term.decode().map_err(|_| "expected number for float")?; + row.set_field(pos, v as f32); + } + DataType::Double(_) => { + let v: f64 = term.decode().map_err(|_| "expected number for double")?; + row.set_field(pos, v); + } + DataType::String(_) | DataType::Char(_) => { + let v: String = term.decode().map_err(|_| "expected string")?; + row.set_field(pos, v); + } + DataType::Decimal(dt) => { + let v: String = term.decode().map_err(|_| "expected string for decimal")?; + let bd = bigdecimal::BigDecimal::from_str(&v) + .map_err(|e| format!("failed to parse decimal '{v}': {e}"))?; + let decimal = Decimal::from_big_decimal(bd, dt.precision(), dt.scale()) + .map_err(|e| e.to_string())?; + row.set_field(pos, decimal); + } + DataType::Bytes(_) | DataType::Binary(_) => { + let bin: rustler::Binary = term.decode().map_err(|_| "expected binary")?; + row.set_field(pos, bin.as_slice().to_vec()); + } Review Comment: Do we need to perform length check here? ########## bindings/elixir/test/integration/log_table_test.exs: ########## @@ -0,0 +1,420 @@ +# 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. + +defmodule Fluss.Integration.LogTableTest do + use ExUnit.Case, async: false + + alias Fluss.Test.Cluster + + @moduletag :integration + + @database "fluss" + + setup_all do + case Cluster.ensure_started() do + {:ok, servers} -> + config = Fluss.Config.new(servers) + + # Wait for cluster to be fully ready (connection + admin working) + {conn, admin} = connect_with_retry(config, 90) + + %{conn: conn, admin: admin, config: config} + + {:error, reason} -> + raise "Failed to start Fluss cluster: #{reason}" + end + end + + describe "append and scan" do + test "append rows and scan with log scanner", %{conn: conn, admin: admin} do + table_name = "ex_test_append_and_scan_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("c1", :int) + |> Fluss.Schema.column("c2", :string) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + writer = Fluss.AppendWriter.new!(table) + + # Append 6 rows + for {c1, c2} <- [{1, "a1"}, {2, "a2"}, {3, "a3"}, {4, "a4"}, {5, "a5"}, {6, "a6"}] do + {:ok, _} = Fluss.AppendWriter.append(writer, [c1, c2]) + end + + :ok = Fluss.AppendWriter.flush(writer) + + # Scan all records + scanner = Fluss.LogScanner.new!(table) + :ok = Fluss.LogScanner.subscribe(scanner, 0, Fluss.earliest_offset()) + + records = poll_records(scanner, 6) + + assert length(records) == 6 + + sorted = Enum.sort_by(records, fn r -> r[:row][:c1] end) + + for {record, i} <- Enum.with_index(sorted, 1) do + assert record[:row][:c1] == i + assert record[:row][:c2] == "a#{i}" + assert record[:change_type] == :append_only + end + + # Unsubscribe should not error + :ok = Fluss.LogScanner.unsubscribe(scanner, 0) + + cleanup_table(admin, table_name) + end + + test "append with nil values", %{conn: conn, admin: admin} do + table_name = "ex_test_append_nil_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("id", :int) + |> Fluss.Schema.column("name", :string) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + writer = Fluss.AppendWriter.new!(table) + + {:ok, _} = Fluss.AppendWriter.append(writer, [1, nil]) + {:ok, _} = Fluss.AppendWriter.append(writer, [2, "present"]) + :ok = Fluss.AppendWriter.flush(writer) + + scanner = Fluss.LogScanner.new!(table) + :ok = Fluss.LogScanner.subscribe(scanner, 0, Fluss.earliest_offset()) + + records = poll_records(scanner, 2) + assert length(records) == 2 + + sorted = Enum.sort_by(records, fn r -> r[:row][:id] end) + assert Enum.at(sorted, 0)[:row][:name] == nil + assert Enum.at(sorted, 1)[:row][:name] == "present" + + cleanup_table(admin, table_name) + end + end + + describe "multiple data types" do + test "tinyint, smallint, int, bigint, float, double, string, boolean", %{ + conn: conn, + admin: admin + } do + table_name = "ex_test_data_types_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("a_tinyint", :tinyint) + |> Fluss.Schema.column("b_smallint", :smallint) + |> Fluss.Schema.column("c_int", :int) + |> Fluss.Schema.column("d_bigint", :bigint) + |> Fluss.Schema.column("e_float", :float) + |> Fluss.Schema.column("f_double", :double) + |> Fluss.Schema.column("g_string", :string) + |> Fluss.Schema.column("h_bool", :boolean) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + writer = Fluss.AppendWriter.new!(table) + + {:ok, _} = + Fluss.AppendWriter.append(writer, [ + 127, + 32_000, + 42, + 1_000_000_000_000, + 3.14, + 2.718281828, + "hello", + true + ]) + + {:ok, _} = + Fluss.AppendWriter.append(writer, [-128, -32_000, -1, -999, 0.0, -1.5, "", false]) + + :ok = Fluss.AppendWriter.flush(writer) + + scanner = Fluss.LogScanner.new!(table) + :ok = Fluss.LogScanner.subscribe(scanner, 0, Fluss.earliest_offset()) + + records = poll_records(scanner, 2) + assert length(records) == 2 + + sorted = Enum.sort_by(records, fn r -> r[:row][:c_int] end) + row1 = Enum.at(sorted, 0)[:row] + row2 = Enum.at(sorted, 1)[:row] + + assert row1[:a_tinyint] == -128 + assert row1[:b_smallint] == -32_000 + assert row1[:c_int] == -1 + assert row1[:d_bigint] == -999 + assert row1[:g_string] == "" + assert row1[:h_bool] == false + + assert row2[:a_tinyint] == 127 + assert row2[:b_smallint] == 32_000 + assert row2[:c_int] == 42 + assert row2[:d_bigint] == 1_000_000_000_000 + assert row2[:g_string] == "hello" + assert row2[:h_bool] == true + + cleanup_table(admin, table_name) + end + end + + describe "subscribe_buckets" do + test "subscribe to multiple buckets at once", %{conn: conn, admin: admin} do + table_name = "ex_test_subscribe_buckets_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("id", :int) + |> Fluss.Schema.column("val", :string) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema, bucket_count: 3) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + writer = Fluss.AppendWriter.new!(table) + + for i <- 1..9 do + {:ok, _} = Fluss.AppendWriter.append(writer, [i, "v#{i}"]) + end + + :ok = Fluss.AppendWriter.flush(writer) + + scanner = Fluss.LogScanner.new!(table) + earliest = Fluss.earliest_offset() + + :ok = + Fluss.LogScanner.subscribe_buckets(scanner, [ + {0, earliest}, + {1, earliest}, + {2, earliest} + ]) + + records = poll_records(scanner, 9) + assert length(records) == 9 + + ids = records |> Enum.map(fn r -> r[:row][:id] end) |> Enum.sort() + assert ids == Enum.to_list(1..9) + + cleanup_table(admin, table_name) + end + end + + describe "admin operations" do + test "create and drop database", %{admin: admin} do + db_name = "ex_test_db_#{:rand.uniform(100_000)}" + :ok = Fluss.Admin.create_database(admin, db_name, true) + + {:ok, databases} = Fluss.Admin.list_databases(admin) + assert db_name in databases + + :ok = Fluss.Admin.drop_database(admin, db_name, true) + end + + test "list tables", %{admin: admin} do + table_name = "ex_test_list_tables_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("id", :int) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + {:ok, tables} = Fluss.Admin.list_tables(admin, @database) + assert table_name in tables + + cleanup_table(admin, table_name) + end + + test "table metadata", %{conn: conn, admin: admin} do + table_name = "ex_test_table_meta_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("id", :int) + |> Fluss.Schema.column("name", :string) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + assert Fluss.Table.has_primary_key?(table) == false + assert Fluss.Table.column_names(table) == ["id", "name"] + + cleanup_table(admin, table_name) + end + end + + describe "scan from offset" do + test "subscribe from specific offset skips earlier records", %{conn: conn, admin: admin} do + table_name = "ex_test_scan_offset_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("id", :int) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + writer = Fluss.AppendWriter.new!(table) + + for i <- 1..5 do + {:ok, _} = Fluss.AppendWriter.append(writer, [i]) + end + + :ok = Fluss.AppendWriter.flush(writer) + + # Subscribe from offset 3, should skip first 3 records + scanner = Fluss.LogScanner.new!(table) + :ok = Fluss.LogScanner.subscribe(scanner, 0, 3) + + records = poll_records(scanner, 2) + assert length(records) == 2 + + ids = records |> Enum.map(fn r -> r[:row][:id] end) |> Enum.sort() + assert ids == [4, 5] + + cleanup_table(admin, table_name) + end + end + + describe "multiple flushes" do + test "append, flush, append more, flush, scan all", %{conn: conn, admin: admin} do + table_name = "ex_test_multi_flush_#{:rand.uniform(100_000)}" + cleanup_table(admin, table_name) + + schema = + Fluss.Schema.build() + |> Fluss.Schema.column("id", :int) + |> Fluss.Schema.column("batch", :string) + |> Fluss.Schema.build!() + + descriptor = Fluss.TableDescriptor.new!(schema) + :ok = Fluss.Admin.create_table(admin, @database, table_name, descriptor, false) + + table = Fluss.Table.get!(conn, @database, table_name) + writer = Fluss.AppendWriter.new!(table) + + # First batch + {:ok, _} = Fluss.AppendWriter.append(writer, [1, "first"]) + {:ok, _} = Fluss.AppendWriter.append(writer, [2, "first"]) + :ok = Fluss.AppendWriter.flush(writer) + + # Second batch + {:ok, _} = Fluss.AppendWriter.append(writer, [3, "second"]) + {:ok, _} = Fluss.AppendWriter.append(writer, [4, "second"]) + :ok = Fluss.AppendWriter.flush(writer) + + scanner = Fluss.LogScanner.new!(table) + :ok = Fluss.LogScanner.subscribe(scanner, 0, Fluss.earliest_offset()) + + records = poll_records(scanner, 4) + assert length(records) == 4 + + sorted = Enum.sort_by(records, fn r -> r[:row][:id] end) + assert Enum.at(sorted, 0)[:row][:batch] == "first" + assert Enum.at(sorted, 1)[:row][:batch] == "first" + assert Enum.at(sorted, 2)[:row][:batch] == "second" + assert Enum.at(sorted, 3)[:row][:batch] == "second" + + cleanup_table(admin, table_name) + end + end + + defp poll_records(scanner, expected_count, timeout_ms \\ 10_000) do + deadline = System.monotonic_time(:millisecond) + timeout_ms + do_poll(scanner, expected_count, deadline, []) + end + + defp do_poll(_scanner, expected_count, _deadline, acc) when length(acc) >= expected_count do + acc + end + + defp do_poll(scanner, expected_count, deadline, acc) do + remaining = deadline - System.monotonic_time(:millisecond) + + if remaining <= 0 do + acc + else + :ok = Fluss.LogScanner.poll(scanner, min(5_000, remaining)) + + receive do + {:fluss_records, records} -> + do_poll(scanner, expected_count, deadline, acc ++ records) + + {:fluss_poll_error, _reason} -> + do_poll(scanner, expected_count, deadline, acc) Review Comment: Seems like we're just getting rid of fluss_poll_error here. I think we need to at least log this out (or even surface the error up the stack) so that if error occurs during tests, there are information to aid debugging. ########## bindings/elixir/native/fluss_nif/src/row_convert.rs: ########## @@ -0,0 +1,263 @@ +// 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 std::str::FromStr; + +use fluss::metadata::{Column, DataType}; +use fluss::row::{Date, Decimal, GenericRow, InternalRow, Time, TimestampLtz, TimestampNtz}; +use rustler::types::binary::NewBinary; +use rustler::{Encoder, Env, Term}; + +use crate::atoms; + +/// Convert column names to BEAM atoms for use as map keys. +/// +/// Note: BEAM atoms are never garbage-collected. This is safe because column +/// names come from server-defined table schemas (bounded set), not arbitrary +/// user input. The BEAM deduplicates atoms, so repeated calls with the same +/// column names do not grow the atom table. +pub fn intern_column_atoms<'a>(env: Env<'a>, columns: &[Column]) -> Vec<rustler::Atom> { + columns + .iter() + .map(|col| rustler::Atom::from_str(env, col.name()).expect("valid atom")) + .collect() +} + +pub fn row_to_term<'a>( + env: Env<'a>, + row: &dyn InternalRow, + columns: &[Column], + column_atoms: &[rustler::Atom], +) -> Result<Term<'a>, String> { + let pairs: Vec<(Term<'a>, Term<'a>)> = columns + .iter() + .enumerate() + .map(|(i, col)| { + let key = column_atoms[i].encode(env); + let value = field_to_term(env, row, i, col.data_type())?; + Ok((key, value)) + }) + .collect::<Result<_, String>>()?; + Term::map_from_pairs(env, &pairs).map_err(|_| "failed to create map".to_string()) +} + +fn field_to_term<'a>( + env: Env<'a>, + row: &dyn InternalRow, + pos: usize, + data_type: &DataType, +) -> Result<Term<'a>, String> { + if row.is_null_at(pos).map_err(|e| e.to_string())? { + return Ok(atoms::nil().encode(env)); + } + + match data_type { + DataType::Boolean(_) => { + let v = row.get_boolean(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::TinyInt(_) => { + let v = row.get_byte(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::SmallInt(_) => { + let v = row.get_short(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Int(_) => { + let v = row.get_int(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::BigInt(_) => { + let v = row.get_long(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Float(_) => { + let v = row.get_float(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Double(_) => { + let v = row.get_double(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::String(_) => { + let v = row.get_string(pos).map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Char(ct) => { + let v = row + .get_char(pos, ct.length() as usize) + .map_err(|e| e.to_string())?; + Ok(v.encode(env)) + } + DataType::Bytes(_) => { + let v = row.get_bytes(pos).map_err(|e| e.to_string())?; + let mut bin = NewBinary::new(env, v.len()); + bin.as_mut_slice().copy_from_slice(v); + let binary: rustler::Binary = bin.into(); + Ok(binary.encode(env)) + } + DataType::Binary(bt) => { + let v = row + .get_binary(pos, bt.length()) + .map_err(|e| e.to_string())?; + let mut bin = NewBinary::new(env, v.len()); + bin.as_mut_slice().copy_from_slice(v); + let binary: rustler::Binary = bin.into(); + Ok(binary.encode(env)) + } + DataType::Date(_) => { + let v = row.get_date(pos).map_err(|e| e.to_string())?; + Ok(v.get_inner().encode(env)) + } + DataType::Time(_) => { + let v = row.get_time(pos).map_err(|e| e.to_string())?; + Ok(v.get_inner().encode(env)) + } + DataType::Timestamp(ts) => { + let v = row + .get_timestamp_ntz(pos, ts.precision()) + .map_err(|e| e.to_string())?; + Ok((v.get_millisecond(), v.get_nano_of_millisecond()).encode(env)) + } + DataType::TimestampLTz(ts) => { + let v = row + .get_timestamp_ltz(pos, ts.precision()) + .map_err(|e| e.to_string())?; + Ok((v.get_epoch_millisecond(), v.get_nano_of_millisecond()).encode(env)) + } + DataType::Decimal(dt) => { + let v = row + .get_decimal(pos, dt.precision() as usize, dt.scale() as usize) + .map_err(|e| e.to_string())?; + Ok(v.to_string().encode(env)) + } + _ => Err(format!("unsupported data type: {data_type:?}")), + } +} + +pub fn term_to_row<'a>( + env: Env<'a>, + values: Term<'a>, + columns: &[Column], +) -> Result<GenericRow<'static>, String> { + let list: Vec<Term<'a>> = values + .decode() + .map_err(|_| "expected a list of values".to_string())?; + if list.len() != columns.len() { + return Err(format!( + "expected {} values, got {}", + columns.len(), + list.len() + )); + } + + let mut row = GenericRow::new(columns.len()); + for (i, (term, col)) in list.iter().zip(columns.iter()).enumerate() { + if term.is_atom() + && let Ok(atom) = term.decode::<rustler::Atom>() + && atom == atoms::nil() + { + continue; // leave as null + } + set_field_from_term(env, &mut row, i, *term, col.data_type())?; + } + Ok(row) +} + +fn set_field_from_term<'a>( + _env: Env<'a>, + row: &mut GenericRow<'static>, + pos: usize, + term: Term<'a>, + data_type: &DataType, +) -> Result<(), String> { + match data_type { + DataType::Boolean(_) => { + let v: bool = term.decode().map_err(|_| "expected boolean")?; + row.set_field(pos, v); + } + DataType::TinyInt(_) => { + let v: i8 = term.decode().map_err(|_| "expected integer for tinyint")?; + row.set_field(pos, v); + } + DataType::SmallInt(_) => { + let v: i16 = term.decode().map_err(|_| "expected integer for smallint")?; + row.set_field(pos, v); + } + DataType::Int(_) => { + let v: i32 = term.decode().map_err(|_| "expected integer")?; + row.set_field(pos, v); + } + DataType::BigInt(_) => { + let v: i64 = term.decode().map_err(|_| "expected integer")?; + row.set_field(pos, v); + } + DataType::Date(_) => { + let v: i32 = term + .decode() + .map_err(|_| "expected integer (days since epoch)")?; + row.set_field(pos, Date::new(v)); + } + DataType::Time(_) => { + let v: i32 = term + .decode() + .map_err(|_| "expected integer (millis since midnight)")?; + row.set_field(pos, Time::new(v)); + } + DataType::Timestamp(_) => { + let (millis, nanos): (i64, i32) = term + .decode() + .map_err(|_| "expected {millis, nanos} tuple for timestamp")?; + let ts = TimestampNtz::from_millis_nanos(millis, nanos).map_err(|e| e.to_string())?; + row.set_field(pos, ts); + } + DataType::TimestampLTz(_) => { + let (millis, nanos): (i64, i32) = term + .decode() + .map_err(|_| "expected {millis, nanos} tuple for timestamp_ltz")?; + let ts = TimestampLtz::from_millis_nanos(millis, nanos).map_err(|e| e.to_string())?; + row.set_field(pos, ts); + } + DataType::Float(_) => { + let v: f64 = term.decode().map_err(|_| "expected number for float")?; + row.set_field(pos, v as f32); Review Comment: We are casting from f64 to f32, is there risk of silent saturating cast to infinities? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
