diff --git a/bindings/python/example/example.py b/bindings/python/example/example.py index 52cefe1e..57a3313d 100644 --- a/bindings/python/example/example.py +++ b/bindings/python/example/example.py @@ -40,9 +40,8 @@ async def main(): config = fluss.Config(config_spec) # Create connection using the static create method - conn = await fluss.FlussConnection.create(config) - - # Define fields for PyArrow + async with await fluss.FlussConnection.create(config) as conn: + # Define fields for PyArrow fields = [ pa.field("id", pa.int32()), pa.field("name", pa.string()), @@ -105,8 +104,8 @@ async def main(): print(f"Got table: {table}") # Create a writer for the table - append_writer = table.new_append().create_writer() - print(f"Created append writer: {append_writer}") + async with table.new_append().create_writer() as append_writer: + print(f"Created append writer: {append_writer}") try: # Demo: Write PyArrow Table @@ -240,10 +239,10 @@ async def main(): append_writer.write_pandas(df) print("Successfully wrote Pandas DataFrame") - # Flush all pending data - print("\n--- Flushing data ---") - await append_writer.flush() - print("Successfully flushed data") + # Note: flush() and close() are automatically called by the 'async with' block on successful exit. + # To manually flush before the context ends, you can still call: + # await append_writer.flush() + print("\n--- Ending append writer context (auto-flushing) ---") # Demo: Check offsets after writes print("\n--- Checking offsets after writes ---") @@ -264,8 +263,8 @@ async def main(): print("\n--- Scanning table (batch scanner) ---") try: # Use new_scan().create_record_batch_log_scanner() for batch-based operations - batch_scanner = await table.new_scan().create_record_batch_log_scanner() - print(f"Created batch scanner: {batch_scanner}") + async with await table.new_scan().create_record_batch_log_scanner() as batch_scanner: + print(f"Created batch scanner: {batch_scanner}") # Subscribe to buckets (required before to_arrow/to_pandas) # Use subscribe_buckets to subscribe all buckets from EARLIEST_OFFSET @@ -284,15 +283,15 @@ async def main(): print(f"Could not convert to PyArrow: {e}") # Create a new batch scanner for to_pandas() test - batch_scanner2 = await table.new_scan().create_record_batch_log_scanner() - batch_scanner2.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) + async with await table.new_scan().create_record_batch_log_scanner() as batch_scanner2: + batch_scanner2.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) - # Try to get as Pandas DataFrame - try: - df_result = batch_scanner2.to_pandas() - print(f"\nAs Pandas DataFrame:\n{df_result}") - except Exception as e: - print(f"Could not convert to Pandas: {e}") + # Try to get as Pandas DataFrame + try: + df_result = batch_scanner2.to_pandas() + print(f"\nAs Pandas DataFrame:\n{df_result}") + except Exception as e: + print(f"Could not convert to Pandas: {e}") # TODO: support to_arrow_batch_reader() # which is reserved for streaming use cases @@ -301,43 +300,43 @@ async def main(): # Test poll_arrow() method for incremental reading as Arrow Table print("\n--- Testing poll_arrow() method ---") - batch_scanner3 = await table.new_scan().create_record_batch_log_scanner() - batch_scanner3.subscribe(bucket_id=0, start_offset=fluss.EARLIEST_OFFSET) - print(f"Subscribed to bucket 0 at EARLIEST_OFFSET ({fluss.EARLIEST_OFFSET})") - - # Poll with a timeout of 5000ms (5 seconds) - # Note: poll_arrow() returns an empty table (not an error) on timeout - try: - poll_result = batch_scanner3.poll_arrow(5000) - print(f"Number of rows: {poll_result.num_rows}") - - if poll_result.num_rows > 0: - poll_df = poll_result.to_pandas() - print(f"Polled data:\n{poll_df}") - else: - print("Empty result (no records available)") - # Empty table still has schema - this is useful! - print(f"Schema: {poll_result.schema}") - - except Exception as e: - print(f"Error during poll_arrow: {e}") + async with await table.new_scan().create_record_batch_log_scanner() as batch_scanner3: + batch_scanner3.subscribe(bucket_id=0, start_offset=fluss.EARLIEST_OFFSET) + print(f"Subscribed to bucket 0 at EARLIEST_OFFSET ({fluss.EARLIEST_OFFSET})") + + # Poll with a timeout of 5000ms (5 seconds) + # Note: poll_arrow() returns an empty table (not an error) on timeout + try: + poll_result = batch_scanner3.poll_arrow(5000) + print(f"Number of rows: {poll_result.num_rows}") + + if poll_result.num_rows > 0: + poll_df = poll_result.to_pandas() + print(f"Polled data:\n{poll_df}") + else: + print("Empty result (no records available)") + # Empty table still has schema - this is useful! + print(f"Schema: {poll_result.schema}") + + except Exception as e: + print(f"Error during poll_arrow: {e}") # Test poll_record_batch() method for batches with metadata print("\n--- Testing poll_record_batch() method ---") - batch_scanner4 = await table.new_scan().create_record_batch_log_scanner() - batch_scanner4.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) + async with await table.new_scan().create_record_batch_log_scanner() as batch_scanner4: + batch_scanner4.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) - try: - batches = batch_scanner4.poll_record_batch(5000) - print(f"Number of batches: {len(batches)}") + try: + batches = batch_scanner4.poll_record_batch(5000) + print(f"Number of batches: {len(batches)}") - for i, batch in enumerate(batches): - print(f" Batch {i}: bucket={batch.bucket}, " - f"offsets={batch.base_offset}-{batch.last_offset}, " - f"rows={batch.batch.num_rows}") + for i, batch in enumerate(batches): + print(f" Batch {i}: bucket={batch.bucket}, " + f"offsets={batch.base_offset}-{batch.last_offset}, " + f"rows={batch.batch.num_rows}") - except Exception as e: - print(f"Error during poll_record_batch: {e}") + except Exception as e: + print(f"Error during poll_record_batch: {e}") except Exception as e: print(f"Error during batch scanning: {e}") @@ -346,34 +345,34 @@ async def main(): print("\n--- Scanning table (record scanner) ---") try: # Use new_scan().create_log_scanner() for record-based operations - record_scanner = await table.new_scan().create_log_scanner() - print(f"Created record scanner: {record_scanner}") - - record_scanner.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) - - # Poll returns ScanRecords — records grouped by bucket - print("\n--- Testing poll() method (record-by-record) ---") - try: - scan_records = record_scanner.poll(5000) - print(f"Total records: {scan_records.count()}, buckets: {len(scan_records.buckets())}") - - # Flat iteration over all records (regardless of bucket) - print(f" Flat iteration: {scan_records.count()} records") - for record in scan_records: - print(f" offset={record.offset}, timestamp={record.timestamp}") - - # Per-bucket access - for bucket in scan_records.buckets(): - bucket_recs = scan_records.records(bucket) - print(f" Bucket {bucket}: {len(bucket_recs)} records") - for record in bucket_recs[:3]: - print(f" offset={record.offset}, " - f"timestamp={record.timestamp}, " - f"change_type={record.change_type}, " - f"row={record.row}") - - except Exception as e: - print(f"Error during poll: {e}") + async with await table.new_scan().create_log_scanner() as record_scanner: + print(f"Created record scanner: {record_scanner}") + + record_scanner.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) + + # Poll returns ScanRecords — records grouped by bucket + print("\n--- Testing poll() method (record-by-record) ---") + try: + scan_records = record_scanner.poll(5000) + print(f"Total records: {scan_records.count()}, buckets: {len(scan_records.buckets())}") + + # Flat iteration over all records (regardless of bucket) + print(f" Flat iteration: {scan_records.count()} records") + for record in scan_records: + print(f" offset={record.offset}, timestamp={record.timestamp}") + + # Per-bucket access + for bucket in scan_records.buckets(): + bucket_recs = scan_records.records(bucket) + print(f" Bucket {bucket}: {len(bucket_recs)} records") + for record in bucket_recs[:3]: + print(f" offset={record.offset}, " + f"timestamp={record.timestamp}, " + f"change_type={record.change_type}, " + f"row={record.row}") + + except Exception as e: + print(f"Error during poll: {e}") except Exception as e: print(f"Error during record scanning: {e}") @@ -381,14 +380,14 @@ async def main(): # Demo: unsubscribe — unsubscribe from a bucket (non-partitioned tables) print("\n--- Testing unsubscribe ---") try: - unsub_scanner = await table.new_scan().create_record_batch_log_scanner() - unsub_scanner.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) - print(f"Subscribed to {num_buckets} buckets") - # Unsubscribe from bucket 0 — future polls will skip this bucket - unsub_scanner.unsubscribe(bucket_id=0) - print("Unsubscribed from bucket 0") - remaining = unsub_scanner.poll_arrow(5000) - print(f"After unsubscribe, got {remaining.num_rows} records (from remaining buckets)") + async with await table.new_scan().create_record_batch_log_scanner() as unsub_scanner: + unsub_scanner.subscribe_buckets({i: fluss.EARLIEST_OFFSET for i in range(num_buckets)}) + print(f"Subscribed to {num_buckets} buckets") + # Unsubscribe from bucket 0 — future polls will skip this bucket + unsub_scanner.unsubscribe(bucket_id=0) + print("Unsubscribed from bucket 0") + remaining = unsub_scanner.poll_arrow(5000) + print(f"After unsubscribe, got {remaining.num_rows} records (from remaining buckets)") except Exception as e: print(f"Error during unsubscribe test: {e}") @@ -437,8 +436,8 @@ async def main(): # --- Test Upsert --- print("\n--- Testing Upsert (fire-and-forget) ---") try: - upsert_writer = pk_table.new_upsert().create_writer() - print(f"Created upsert writer: {upsert_writer}") + async with pk_table.new_upsert().create_writer() as upsert_writer: + print(f"Created upsert writer: {upsert_writer}") # Fire-and-forget: queue writes synchronously, flush at end. # Records are batched internally for efficiency. @@ -489,9 +488,10 @@ async def main(): ) print("Queued user_id=3 (Charlie)") - # flush() waits for all queued writes to be acknowledged by the server - await upsert_writer.flush() - print("Flushed — all 3 rows acknowledged by server") + # flush() and close() are automatically called by the 'async with' block on successful exit. + # Bypass manual flush: + # await upsert_writer.flush() + print("Ending upsert writer context (auto-flushing)") # Per-record acknowledgment: await the returned handle to block until # the server confirms this specific write, useful when you need to diff --git a/bindings/python/fluss/__init__.pyi b/bindings/python/fluss/__init__.pyi index 02edcdb3..0ad12c69 100644 --- a/bindings/python/fluss/__init__.pyi +++ b/bindings/python/fluss/__init__.pyi @@ -253,6 +253,13 @@ class FlussConnection: exc_value: Optional[BaseException], traceback: Optional[TracebackType], ) -> bool: ... + async def __aenter__(self) -> FlussConnection: ... + async def __aexit__( + self, + exc_type: Optional[type], + exc_value: Optional[BaseException], + traceback: Optional[TracebackType], + ) -> bool: ... def __repr__(self) -> str: ... class ServerNode: @@ -611,6 +618,37 @@ class AppendWriter: def write_arrow_batch(self, batch: pa.RecordBatch) -> WriteResultHandle: ... def write_pandas(self, df: pd.DataFrame) -> None: ... async def flush(self) -> None: ... + async def __aenter__(self) -> AppendWriter: + """ + Enter the async context manager. + + Returns: + The AppendWriter instance. + """ + ... + async def __aexit__( + self, + exc_type: Optional[type], + exc_value: Optional[BaseException], + traceback: Optional[TracebackType], + ) -> bool: + """ + Exit the async context manager. + + On successful exit, the writer is automatically flushed to ensure + all pending records are sent and acknowledged. + + Note on Exceptions: + If an exception occurs inside the `async with` block, `flush()` is + bypassed to return control to the event loop immediately. However, + any records already passed to `append()` prior to the exception + reside in a shared background buffer and will still be transmitted + to the server. + + To achieve true atomicity, buffer your records in a Python list and + write them in a single batch at the end of your logic. + """ + ... def __repr__(self) -> str: ... class UpsertWriter: @@ -644,6 +682,37 @@ class UpsertWriter: async def flush(self) -> None: """Flush all pending upsert/delete operations to the server.""" ... + async def __aenter__(self) -> UpsertWriter: + """ + Enter the async context manager. + + Returns: + The UpsertWriter instance. + """ + ... + async def __aexit__( + self, + exc_type: Optional[type], + exc_value: Optional[BaseException], + traceback: Optional[TracebackType], + ) -> bool: + """ + Exit the async context manager. + + On successful exit, the writer is automatically flushed to ensure + all pending records are sent and acknowledged. + + Note on Exceptions: + If an exception occurs inside the `async with` block, `flush()` is + bypassed to return control to the event loop immediately. However, + any records already passed to `upsert()` or `delete()` prior to the + exception reside in a shared background buffer and will still be + transmitted to the server. + + To achieve true atomicity, buffer your records in a Python list and + write them in a single batch at the end of your logic. + """ + ... def __repr__(self) -> str: ... @@ -807,6 +876,15 @@ class LogScanner: You must call subscribe(), subscribe_buckets(), or subscribe_partition() first. """ + ... + def close(self) -> None: ... + async def __aenter__(self) -> LogScanner: ... + async def __aexit__( + self, + exc_type: Optional[type], + exc_value: Optional[BaseException], + traceback: Optional[TracebackType], + ) -> bool: ... def __repr__(self) -> str: ... def __aiter__(self) -> AsyncIterator[Union[ScanRecord, RecordBatch]]: ... diff --git a/bindings/python/src/connection.rs b/bindings/python/src/connection.rs index a8d2d9e3..8112e2d5 100644 --- a/bindings/python/src/connection.rs +++ b/bindings/python/src/connection.rs @@ -104,6 +104,28 @@ impl FlussConnection { Ok(false) } + // Enter the async runtime context (for 'async with' statement) + fn __aenter__<'py>(slf: PyRef<'py, Self>, py: Python<'py>) -> PyResult> { + let py_slf = slf.into_pyobject(py)?.unbind(); + future_into_py(py, async move { Ok(py_slf) }) + } + + // Exit the async runtime context (for 'async with' statement) + #[pyo3(signature = (_exc_type=None, _exc_value=None, _traceback=None))] + fn __aexit__<'py>( + &mut self, + py: Python<'py>, + _exc_type: Option>, + _exc_value: Option>, + _traceback: Option>, + ) -> PyResult> { + future_into_py(py, async move { + // In the future, we could call an async close on the core connection here + // e.g., client.close().await; + Ok(false) + }) + } + fn __repr__(&self) -> String { "FlussConnection()".to_string() } diff --git a/bindings/python/src/table.rs b/bindings/python/src/table.rs index 8d92aba1..338a9339 100644 --- a/bindings/python/src/table.rs +++ b/bindings/python/src/table.rs @@ -989,6 +989,37 @@ impl AppendWriter { }) } + // Enter the async runtime context (for 'async with' statement) + fn __aenter__<'py>(slf: PyRef<'py, Self>, py: Python<'py>) -> PyResult> { + let py_slf = slf.into_pyobject(py)?.unbind(); + future_into_py(py, async move { Ok(py_slf) }) + } + + // Exit the async runtime context (for 'async with' statement) + /// On successful exit, the writer is automatically flushed. + /// If an exception occurs, the flush is skipped to allow immediate error + /// propagation, though pending records may still be sent in the background. + #[pyo3(signature = (exc_type=None, _exc_value=None, _traceback=None))] + fn __aexit__<'py>( + &self, + py: Python<'py>, + exc_type: Option>, + _exc_value: Option>, + _traceback: Option>, + ) -> PyResult> { + let has_error = exc_type.is_some(); + let inner = self.inner.clone(); + future_into_py(py, async move { + if !has_error { + inner + .flush() + .await + .map_err(|e| FlussError::from_core_error(&e))?; + } + Ok(false) + }) + } + fn __repr__(&self) -> String { "AppendWriter()".to_string() } @@ -2331,6 +2362,32 @@ async def _async_scan_generic(scanner, method_name): fn __repr__(&self) -> String { format!("LogScanner(table={})", self.table_info.table_path) } + + /// Close the scanner + pub fn close(&self) -> PyResult<()> { + Ok(()) + } + + // Enter the async runtime context (for 'async with' statement) + fn __aenter__<'py>(slf: PyRef<'py, Self>, py: Python<'py>) -> PyResult> { + let py_slf = slf.into_pyobject(py)?.unbind(); + future_into_py(py, async move { Ok(py_slf) }) + } + + // Exit the async runtime context (for 'async with' statement) + #[pyo3(signature = (_exc_type=None, _exc_value=None, _traceback=None))] + fn __aexit__<'py>( + &self, + py: Python<'py>, + _exc_type: Option>, + _exc_value: Option>, + _traceback: Option>, + ) -> PyResult> { + future_into_py(py, async move { + // In the future, we can call an async close on the core scanner here + Ok(false) + }) + } } impl LogScanner { diff --git a/bindings/python/src/upsert.rs b/bindings/python/src/upsert.rs index 02ad7fa4..f597737d 100644 --- a/bindings/python/src/upsert.rs +++ b/bindings/python/src/upsert.rs @@ -97,6 +97,7 @@ impl UpsertWriter { /// /// Returns: /// None on success + /// Flush any pending data pub fn flush<'py>(&self, py: Python<'py>) -> PyResult> { let writer = self.writer.clone(); @@ -108,6 +109,37 @@ impl UpsertWriter { }) } + // Enter the async runtime context (for 'async with' statement) + fn __aenter__<'py>(slf: PyRef<'py, Self>, py: Python<'py>) -> PyResult> { + let py_slf = slf.into_pyobject(py)?.unbind(); + future_into_py(py, async move { Ok(py_slf) }) + } + + // Exit the async runtime context (for 'async with' statement) + /// On successful exit, the writer is automatically flushed. + /// If an exception occurs, the flush is skipped to allow immediate error + /// propagation, though pending records may still be sent in the background. + #[pyo3(signature = (exc_type=None, _exc_value=None, _traceback=None))] + fn __aexit__<'py>( + &self, + py: Python<'py>, + exc_type: Option>, + _exc_value: Option>, + _traceback: Option>, + ) -> PyResult> { + let has_error = exc_type.is_some(); + let writer = self.writer.clone(); + future_into_py(py, async move { + if !has_error { + writer + .flush() + .await + .map_err(|e| FlussError::from_core_error(&e))?; + } + Ok(false) + }) + } + fn __repr__(&self) -> String { "UpsertWriter()".to_string() } diff --git a/bindings/python/test/test_context_manager.py b/bindings/python/test/test_context_manager.py new file mode 100644 index 00000000..0e4f6cc1 --- /dev/null +++ b/bindings/python/test/test_context_manager.py @@ -0,0 +1,186 @@ +# 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. + +import asyncio +import pytest +import pyarrow as pa +import time +import fluss + +def _poll_records(scanner, expected_count, timeout_s=10): + """Poll a record-based scanner until expected_count records are collected.""" + collected = [] + deadline = time.monotonic() + timeout_s + while len(collected) < expected_count and time.monotonic() < deadline: + records = scanner.poll(5000) + collected.extend(records) + return collected + +@pytest.mark.asyncio +async def test_connection_context_manager(plaintext_bootstrap_servers): + config = fluss.Config({"bootstrap.servers": plaintext_bootstrap_servers}) + async with await fluss.FlussConnection.create(config) as conn: + admin = conn.get_admin() + nodes = await admin.get_server_nodes() + assert len(nodes) > 0 + # conn should be closed (though currently close is a no-op in python side, but verifies syntax) + +@pytest.mark.asyncio +async def test_append_writer_success_flush(connection, admin): + table_path = fluss.TablePath("fluss", "test_append_ctx_success") + await admin.drop_table(table_path, ignore_if_not_exists=True) + + schema = fluss.Schema(pa.schema([pa.field("a", pa.int32())])) + await admin.create_table(table_path, fluss.TableDescriptor(schema)) + + table = await connection.get_table(table_path) + + async with table.new_append().create_writer() as writer: + writer.append({"a": 1}) + writer.append({"a": 2}) + # No explicit flush here + + # After context exit, data should be flushed + scanner = await table.new_scan().create_log_scanner() + scanner.subscribe(0, fluss.EARLIEST_OFFSET) + records = _poll_records(scanner, expected_count=2) + assert len(records) == 2 + assert sorted([r.row["a"] for r in records]) == [1, 2] + +@pytest.mark.asyncio +async def test_append_writer_exception_no_flush(connection, admin): + table_path = fluss.TablePath("fluss", "test_append_ctx_fail") + await admin.drop_table(table_path, ignore_if_not_exists=True) + + schema = fluss.Schema(pa.schema([pa.field("a", pa.int32())])) + await admin.create_table(table_path, fluss.TableDescriptor(schema)) + table = await connection.get_table(table_path) + + class TestException(Exception): pass + + start_time = time.perf_counter() + try: + async with table.new_append().create_writer() as writer: + writer.append({"a": 100}) + raise TestException("abort") + except TestException: + pass + duration = time.perf_counter() - start_time + + # Verification: + # 1. The exception was propagated immediately. + # 2. The block exited nearly instantly because it bypassed the network flush. + assert duration < 0.1, f"Context exit took too long ({duration:.3f}s), likely performed a flush" + + # NOTE: Records may still eventually arrive because of the background sender threads. + # We don't assert 0 records here because Fluss does not support true transactional rollback. + +@pytest.mark.asyncio +async def test_upsert_writer_context_manager(connection, admin): + table_path = fluss.TablePath("fluss", "test_upsert_ctx") + await admin.drop_table(table_path, ignore_if_not_exists=True) + + schema = fluss.Schema(pa.schema([pa.field("id", pa.int32()), pa.field("v", pa.string())]), primary_keys=["id"]) + await admin.create_table(table_path, fluss.TableDescriptor(schema)) + + table = await connection.get_table(table_path) + + # Success path: verify it flushes + async with table.new_upsert().create_writer() as writer: + writer.upsert({"id": 1, "v": "a"}) + + lookuper = table.new_lookup().create_lookuper() + res = await lookuper.lookup({"id": 1}) + assert res is not None + assert res["v"] == "a" + + # Failure path: verify it bypasses flush + class TestException(Exception): pass + start_time = time.perf_counter() + try: + async with table.new_upsert().create_writer() as writer: + writer.upsert({"id": 2, "v": "b"}) + raise TestException("abort") + except TestException: + pass + duration = time.perf_counter() - start_time + assert duration < 0.1, f"Context exit took too long ({duration:.3f}s), likely performed a flush" + +@pytest.mark.asyncio +async def test_log_scanner_context_manager(connection, admin): + table_path = fluss.TablePath("fluss", "test_scanner_ctx") + await admin.drop_table(table_path, ignore_if_not_exists=True) + schema = fluss.Schema(pa.schema([pa.field("a", pa.int32())])) + await admin.create_table(table_path, fluss.TableDescriptor(schema)) + table = await connection.get_table(table_path) + + async with await table.new_scan().create_log_scanner() as scanner: + scanner.subscribe(0, fluss.EARLIEST_OFFSET) + # Verifies it works and closes properly + res = scanner.poll(100) + assert len(res) == 0 + +@pytest.mark.asyncio +async def test_connection_context_manager_exception(plaintext_bootstrap_servers): + config = fluss.Config({"bootstrap.servers": plaintext_bootstrap_servers}) + class TestException(Exception): pass + + try: + async with await fluss.FlussConnection.create(config) as conn: + raise TestException("connection error") + except TestException: + pass + # If we reach here without hanging, the connection __aexit__ gracefully handled the error + +@pytest.mark.asyncio +async def test_record_batch_scanner_context_manager(connection, admin): + table_path = fluss.TablePath("fluss", "test_batch_scanner_ctx") + await admin.drop_table(table_path, ignore_if_not_exists=True) + schema = fluss.Schema(pa.schema([pa.field("a", pa.int32())])) + await admin.create_table(table_path, fluss.TableDescriptor(schema)) + table = await connection.get_table(table_path) + + async with await table.new_scan().create_record_batch_log_scanner() as scanner: + scanner.subscribe(0, fluss.EARLIEST_OFFSET) + res = scanner.poll_arrow(100) + assert res.num_rows == 0 + +@pytest.mark.asyncio +async def test_scanner_exception_propagation(connection, admin): + table_path = fluss.TablePath("fluss", "test_scanner_ctx_fail") + await admin.drop_table(table_path, ignore_if_not_exists=True) + schema = fluss.Schema(pa.schema([pa.field("a", pa.int32())])) + await admin.create_table(table_path, fluss.TableDescriptor(schema)) + table = await connection.get_table(table_path) + + class TestException(Exception): pass + + # Test record scanner exception + try: + async with await table.new_scan().create_log_scanner() as scanner: + scanner.subscribe(0, fluss.EARLIEST_OFFSET) + raise TestException("scanner error") + except TestException: + pass + + # Test batch scanner exception + try: + async with await table.new_scan().create_record_batch_log_scanner() as scanner: + scanner.subscribe(0, fluss.EARLIEST_OFFSET) + raise TestException("batch scanner error") + except TestException: + pass \ No newline at end of file diff --git a/bindings/python/test/test_log_table.py b/bindings/python/test/test_log_table.py index eb118748..48567d41 100644 --- a/bindings/python/test/test_log_table.py +++ b/bindings/python/test/test_log_table.py @@ -143,6 +143,7 @@ async def test_list_offsets(connection, admin): assert latest[0] == 0 before_append_ms = int(time.time() * 1000) + await asyncio.sleep(0.1) # Append some records table = await connection.get_table(table_path)