From 581d574ca694160a2b29fdbb62969e63a332bc09 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Mon, 4 May 2026 23:00:03 -0700 Subject: [PATCH 01/23] feat(amber): materialize per-port state to Iceberg storage Adds a state-materialization path alongside the existing tuple-result storage. State produced by an operator's processState is written to a companion Iceberg table whose URI is derived from the result URI. The input-port materialization reader replays both tuples and states into downstream workers. Key pieces: - New STATE resource type and a state-namespace storage config entry on both Python and Scala sides; namespaces are read from StorageConfig instead of hardcoded strings. - RegionExecutionCoordinator provisions a state document next to every result document at scheduling time, so readers and writers can rely on its presence without try/catch. - One long-lived BufferedItemWriter per output port, opened at port setup and closed at port completion, so a single Iceberg snapshot is produced per port instead of one per state. - DataProcessor.processInputState (Scala) and MainLoop.process_input_state (Python) persist the executor's *output* state, matching the state that is also emitted downstream. - New Python and Scala unit tests covering the State JSON wire format, the OutputManager state-writer lifecycle, the reader's state-replay block, and DocumentFactory namespace routing. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../architecture/packaging/output_manager.py | 30 ++- .../packaging/test_output_manager.py | 127 ++++++++++++ amber/src/main/python/core/models/state.py | 4 + .../main/python/core/runnables/main_loop.py | 1 + .../python/core/runnables/test_main_loop.py | 76 +++++++ .../python/core/storage/document_factory.py | 107 +++++----- .../storage/iceberg/test_iceberg_document.py | 84 ++++++++ ...ut_port_materialization_reader_runnable.py | 28 ++- ...ut_port_materialization_reader_runnable.py | 190 ++++++++++++++++++ .../python/core/storage/storage_config.py | 3 + .../core/storage/test_document_factory.py | 134 ++++++++++++ .../python/core/storage/vfs_uri_factory.py | 1 + .../storage/test_large_binary_manager.py | 1 + .../main/python/texera_run_python_worker.py | 2 + .../messaginglayer/OutputManager.scala | 19 +- .../pythonworker/PythonWorkflowWorker.scala | 1 + .../RegionExecutionCoordinator.scala | 3 + .../architecture/worker/DataProcessor.scala | 1 + ...InputPortMaterializationReaderThread.scala | 26 ++- common/config/src/main/resources/storage.conf | 3 + .../amber/config/EnvironmentalVariable.scala | 1 + .../texera/amber/config/StorageConfig.scala | 3 + .../texera/amber/core/state/State.scala | 4 + .../amber/core/storage/DocumentFactory.scala | 2 + .../amber/core/storage/VFSURIFactory.scala | 1 + .../result/iceberg/IcebergDocumentSpec.scala | 79 ++++++++ 26 files changed, 877 insertions(+), 54 deletions(-) create mode 100644 amber/src/main/python/core/architecture/packaging/test_output_manager.py create mode 100644 amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py create mode 100644 amber/src/main/python/core/storage/test_document_factory.py diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index bf4afbf396f..08fa210eca5 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -87,6 +87,8 @@ def __init__(self, worker_id: str): PortIdentity, typing.Tuple[Queue, PortStorageWriter, Thread] ] = dict() + self._state_writers: typing.Dict[PortIdentity, typing.Any] = dict() + def is_missing_output_ports(self): """ This method is only used for ensuring correct region execution. @@ -124,7 +126,8 @@ def add_output_port( def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): """ Create a separate thread for saving output tuples of a port - to storage in batch. + to storage in batch, and open a long-lived buffered writer for + state materialization on the same port. """ document, _ = DocumentFactory.open_document(storage_uri) buffered_item_writer = document.writer(str(get_worker_index(self.worker_id))) @@ -144,6 +147,13 @@ def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): writer_thread, ) + state_document, _ = DocumentFactory.open_document( + State.uri_from_result_uri(storage_uri) + ) + state_writer = state_document.writer(str(get_worker_index(self.worker_id))) + state_writer.open() + self._state_writers[port_id] = state_writer + def get_port(self, port_id=None) -> WorkerPort: return list(self._ports.values())[0] @@ -171,6 +181,19 @@ def save_tuple_to_storage_if_needed(self, tuple_: Tuple, port_id=None) -> None: PortStorageWriterElement(data_tuple=tuple_) ) + def save_state_to_storage_if_needed(self, state: State, port_id=None) -> None: + # Buffer the state on each long-lived writer; the writer flushes + # itself when its buffer fills, and the remaining buffer is + # flushed in close_port_storage_writers. + if port_id is None: + writers = self._state_writers.values() + elif port_id in self._state_writers: + writers = [self._state_writers[port_id]] + else: + return + for writer in writers: + writer.put_one(state.to_tuple()) + def close_port_storage_writers(self) -> None: """ Flush the buffers of port storage writers and wait for all the @@ -184,6 +207,11 @@ def close_port_storage_writers(self) -> None: for _, _, writer_thread in self._port_storage_writers.values(): # This blocking call will wait for all the writer to finish commit writer_thread.join() + # Close the long-lived state writers so the remaining buffered + # states are committed in a single Iceberg snapshot per port. + for state_writer in self._state_writers.values(): + state_writer.close() + self._state_writers.clear() def add_partitioning(self, tag: PhysicalLink, partitioning: Partitioning) -> None: """ diff --git a/amber/src/main/python/core/architecture/packaging/test_output_manager.py b/amber/src/main/python/core/architecture/packaging/test_output_manager.py new file mode 100644 index 00000000000..8f1daf8052e --- /dev/null +++ b/amber/src/main/python/core/architecture/packaging/test_output_manager.py @@ -0,0 +1,127 @@ +# 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. + +from unittest.mock import MagicMock, patch + +import pytest + +from core.architecture.packaging.output_manager import OutputManager +from core.models.state import State +from proto.org.apache.texera.amber.core import PortIdentity + + +class TestSaveStateToStorageIfNeeded: + @pytest.fixture + def output_manager(self): + return OutputManager(worker_id="Worker:WF0-test-main-0") + + @pytest.fixture + def port_a(self): + return PortIdentity(id=0, internal=False) + + @pytest.fixture + def port_b(self): + return PortIdentity(id=1, internal=False) + + @pytest.fixture + def state(self): + return State({"loop_counter": 1, "i": 2}) + + def test_no_state_writers_is_a_noop(self, output_manager, state): + # With no port set up, save_state_to_storage_if_needed must not + # touch any writer. + output_manager.save_state_to_storage_if_needed(state) # no-op, no exception + + def test_unknown_port_id_is_a_noop(self, output_manager, state, port_a): + output_manager.save_state_to_storage_if_needed(state, port_id=port_a) + # No assertion needed -- the absence of any writer means nothing + # was attempted. + + def test_writes_to_every_port_when_port_id_omitted( + self, output_manager, state, port_a, port_b + ): + writer_a = MagicMock() + writer_b = MagicMock() + output_manager._state_writers[port_a] = writer_a + output_manager._state_writers[port_b] = writer_b + + output_manager.save_state_to_storage_if_needed(state) + + writer_a.put_one.assert_called_once() + writer_b.put_one.assert_called_once() + # Long-lived writers must NOT be closed per state -- otherwise + # we'd be back to one Iceberg snapshot per state. + writer_a.close.assert_not_called() + writer_b.close.assert_not_called() + + def test_writes_only_to_selected_port_when_port_id_specified( + self, output_manager, state, port_a, port_b + ): + writer_a = MagicMock() + writer_b = MagicMock() + output_manager._state_writers[port_a] = writer_a + output_manager._state_writers[port_b] = writer_b + + output_manager.save_state_to_storage_if_needed(state, port_id=port_a) + + writer_a.put_one.assert_called_once() + writer_b.put_one.assert_not_called() + + def test_state_writer_is_opened_at_port_setup(self, output_manager, port_a): + # set_up_port_storage_writer should open the result document AND + # the state document, then cache the state writer for reuse. + result_doc = MagicMock() + state_doc = MagicMock() + state_writer = MagicMock() + state_doc.writer.return_value = state_writer + + with patch( + "core.architecture.packaging.output_manager.DocumentFactory" + ) as mock_factory: + mock_factory.open_document.side_effect = [ + (result_doc, MagicMock()), + (state_doc, MagicMock()), + ] + + output_manager.set_up_port_storage_writer( + port_a, "vfs:///wf/0/exec/0/result/op-a" + ) + + opened = [c.args[0] for c in mock_factory.open_document.call_args_list] + assert opened == [ + "vfs:///wf/0/exec/0/result/op-a", + "vfs:///wf/0/exec/0/state/op-a", + ] + state_writer.open.assert_called_once() + assert output_manager._state_writers[port_a] is state_writer + + def test_close_port_storage_writers_flushes_state_writers( + self, output_manager, port_a, port_b + ): + # After the port completes, the long-lived state writer's buffer + # must be flushed and the writer closed (one Iceberg commit per + # port instead of one per state). + writer_a = MagicMock() + writer_b = MagicMock() + output_manager._state_writers[port_a] = writer_a + output_manager._state_writers[port_b] = writer_b + + output_manager.close_port_storage_writers() + + writer_a.close.assert_called_once() + writer_b.close.assert_called_once() + assert output_manager._state_writers == {} diff --git a/amber/src/main/python/core/models/state.py b/amber/src/main/python/core/models/state.py index 003aaa212ac..3ce610bbee5 100644 --- a/amber/src/main/python/core/models/state.py +++ b/amber/src/main/python/core/models/state.py @@ -41,6 +41,10 @@ def from_json(cls, payload: str) -> "State": def from_tuple(cls, row: Tuple) -> "State": return cls.from_json(row[cls.CONTENT]) + @staticmethod + def uri_from_result_uri(result_uri: str) -> str: + return result_uri.replace("/result", "/state") + _TYPE_MARKER = "__texera_type__" _PAYLOAD_MARKER = "payload" diff --git a/amber/src/main/python/core/runnables/main_loop.py b/amber/src/main/python/core/runnables/main_loop.py index ab35cda81b9..1334af12bfe 100644 --- a/amber/src/main/python/core/runnables/main_loop.py +++ b/amber/src/main/python/core/runnables/main_loop.py @@ -202,6 +202,7 @@ def process_input_state(self) -> None: payload=batch, ) ) + self.context.output_manager.save_state_to_storage_if_needed(output_state) def process_tuple_with_udf(self) -> Iterator[Optional[Tuple]]: """ diff --git a/amber/src/main/python/core/runnables/test_main_loop.py b/amber/src/main/python/core/runnables/test_main_loop.py index c9daa633f55..534493f0c21 100644 --- a/amber/src/main/python/core/runnables/test_main_loop.py +++ b/amber/src/main/python/core/runnables/test_main_loop.py @@ -1388,6 +1388,82 @@ def fake_switch_context(): assert second_output.payload.frame["value"] == 42 assert second_output.payload.frame["port"] == 0 + @pytest.mark.timeout(2) + def test_process_input_state_persists_output_state_to_storage( + self, + main_loop, + mock_data_output_channel, + monkeypatch, + ): + # process_input_state must invoke save_state_to_storage_if_needed + # with the freshly emitted output state, so every state that flows + # downstream is also durable on the upstream output port. + class DummyExecutor: + @staticmethod + def process_state(state: State, port: int) -> State: + return State({"value": state["value"] + 1, "port": port}) + + saved_states: list[State] = [] + main_loop.context.executor_manager.executor = DummyExecutor() + monkeypatch.setattr(main_loop, "_check_and_process_control", lambda: None) + monkeypatch.setattr( + main_loop.context.output_manager, + "emit_state", + lambda state: [(mock_data_output_channel.to_worker_id, StateFrame(state))], + ) + monkeypatch.setattr( + main_loop.context.output_manager, + "save_state_to_storage_if_needed", + lambda state: saved_states.append(state), + ) + + def fake_switch_context(): + current_input_state = ( + main_loop.context.state_processing_manager.current_input_state + ) + if current_input_state is not None: + main_loop.context.state_processing_manager.current_output_state = ( + DummyExecutor.process_state(current_input_state, 0) + ) + + monkeypatch.setattr(main_loop, "_switch_context", fake_switch_context) + + main_loop._process_state(State({"value": 1})) + main_loop._process_state(State({"value": 41})) + + # Each input state produced one output state, so both must have + # been persisted in order. + assert [s["value"] for s in saved_states] == [2, 42] + assert all(s["port"] == 0 for s in saved_states) + + @pytest.mark.timeout(2) + def test_process_input_state_does_not_save_when_no_output( + self, + main_loop, + monkeypatch, + ): + # When the executor returns no output state (process_state returned + # None), save_state_to_storage_if_needed must not be called -- no + # state means nothing to materialize. + save_calls: list[State] = [] + monkeypatch.setattr(main_loop, "_check_and_process_control", lambda: None) + monkeypatch.setattr( + main_loop.context.output_manager, + "emit_state", + lambda state: [], + ) + monkeypatch.setattr( + main_loop.context.output_manager, + "save_state_to_storage_if_needed", + lambda state: save_calls.append(state), + ) + # Pretend DataProc consumed the input but produced no output. + monkeypatch.setattr(main_loop, "_switch_context", lambda: None) + + main_loop._process_state(State({"value": 1})) + + assert save_calls == [] + @pytest.mark.timeout(2) def test_main_loop_thread_can_process_state( self, diff --git a/amber/src/main/python/core/storage/document_factory.py b/amber/src/main/python/core/storage/document_factory.py index 9b686ab66b6..bd690ceb592 100644 --- a/amber/src/main/python/core/storage/document_factory.py +++ b/amber/src/main/python/core/storage/document_factory.py @@ -61,30 +61,35 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: if parsed_uri.scheme == VFSURIFactory.VFS_FILE_URI_SCHEME: _, _, _, resource_type = VFSURIFactory.decode_uri(uri) - if resource_type in {VFSResourceType.RESULT}: - storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) - - # Convert Amber Schema to Iceberg Schema with LARGE_BINARY - # field name encoding - iceberg_schema = amber_schema_to_iceberg_schema(schema) - - create_table( - IcebergCatalogInstance.get_instance(), - StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE, - storage_key, - iceberg_schema, - override_if_exists=True, - ) - - return IcebergDocument[Tuple]( - StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE, - storage_key, - iceberg_schema, - amber_tuples_to_arrow_table, - arrow_table_to_amber_tuples, - ) - else: - raise ValueError(f"Resource type {resource_type} is not supported") + match resource_type: + case VFSResourceType.RESULT: + namespace = StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE + case VFSResourceType.STATE: + namespace = StorageConfig.ICEBERG_TABLE_STATE_NAMESPACE + case _: + raise ValueError(f"Resource type {resource_type} is not supported") + + storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) + # Convert Amber Schema to Iceberg Schema with LARGE_BINARY + # field name encoding + iceberg_schema = amber_schema_to_iceberg_schema(schema) + + create_table( + IcebergCatalogInstance.get_instance(), + namespace, + storage_key, + iceberg_schema, + override_if_exists=True, + ) + + return IcebergDocument[Tuple]( + namespace, + storage_key, + iceberg_schema, + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples, + ) + else: raise NotImplementedError( f"Unsupported URI scheme: {parsed_uri.scheme} for creating the document" @@ -96,30 +101,36 @@ def open_document(uri: str) -> typing.Tuple[VirtualDocument, Optional[Schema]]: if parsed_uri.scheme == "vfs": _, _, _, resource_type = VFSURIFactory.decode_uri(uri) - if resource_type in {VFSResourceType.RESULT}: - storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) - - table = load_table_metadata( - IcebergCatalogInstance.get_instance(), - StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE, - storage_key, - ) - - if table is None: - raise ValueError("No storage is found for the given URI") - - amber_schema = Schema(table.schema().as_arrow()) - - document = IcebergDocument( - StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE, - storage_key, - table.schema(), - amber_tuples_to_arrow_table, - arrow_table_to_amber_tuples, - ) - return document, amber_schema - else: - raise ValueError(f"Resource type {resource_type} is not supported") + match resource_type: + case VFSResourceType.RESULT: + namespace = StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE + case VFSResourceType.STATE: + namespace = StorageConfig.ICEBERG_TABLE_STATE_NAMESPACE + case _: + raise ValueError(f"Resource type {resource_type} is not supported") + + storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) + + table = load_table_metadata( + IcebergCatalogInstance.get_instance(), + namespace, + storage_key, + ) + + if table is None: + raise ValueError("No storage is found for the given URI") + + amber_schema = Schema(table.schema().as_arrow()) + + document = IcebergDocument( + namespace, + storage_key, + table.schema(), + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples, + ) + return document, amber_schema + else: raise NotImplementedError( f"Unsupported URI scheme: {parsed_uri.scheme} for opening the document" diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py index 9b374f7d5c7..032376ae314 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py @@ -23,6 +23,7 @@ from concurrent.futures.thread import ThreadPoolExecutor from core.models import Schema, Tuple +from core.models.state import State from core.storage.document_factory import DocumentFactory from core.storage.storage_config import StorageConfig from core.storage.vfs_uri_factory import VFSURIFactory @@ -44,6 +45,7 @@ rest_catalog_uri="http://localhost:8181/catalog/", rest_catalog_warehouse_name="texera", table_result_namespace="operator-port-result", + table_state_namespace="operator-port-state", directory_path="../../../../../../amber/user-resources/workflow-results", commit_batch_size=4096, s3_endpoint="http://localhost:9000", @@ -317,3 +319,85 @@ def test_get_counts(self, iceberg_document, sample_items): assert iceberg_document.get_count() == len(sample_items), ( "get_count should return the same number as the length of sample_items" ) + + def test_state_materialization_round_trip(self): + operator_uuid = str(uuid.uuid4()).replace("-", "") + result_uri = VFSURIFactory.create_result_uri( + WorkflowIdentity(id=0), + ExecutionIdentity(id=0), + GlobalPortIdentity( + op_id=PhysicalOpIdentity( + logical_op_id=OperatorIdentity(id=f"test_state_{operator_uuid}"), + layer_name="main", + ), + port_id=PortIdentity(id=0), + input=False, + ), + ) + state_uri = State.uri_from_result_uri(result_uri) + DocumentFactory.create_document(state_uri, State.SCHEMA) + document, _ = DocumentFactory.open_document(state_uri) + + state = State( + { + "loop_counter": 3, + "name": "outer-loop", + "payload": b"\x00\x01state-bytes", + "nested": {"enabled": True, "values": [1, 2, 3]}, + } + ) + + writer = document.writer(str(uuid.uuid4())) + writer.open() + writer.put_one(state.to_tuple()) + writer.close() + + stored_rows = list(document.get()) + assert len(stored_rows) == 1 + assert State.from_tuple(stored_rows[0]) == state + + def test_multiple_states_materialize_as_rows_in_one_table(self): + operator_uuid = str(uuid.uuid4()).replace("-", "") + result_uri = VFSURIFactory.create_result_uri( + WorkflowIdentity(id=0), + ExecutionIdentity(id=0), + GlobalPortIdentity( + op_id=PhysicalOpIdentity( + logical_op_id=OperatorIdentity( + id=f"test_multiple_states_{operator_uuid}" + ), + layer_name="main", + ), + port_id=PortIdentity(id=0), + input=False, + ), + ) + state_uri = State.uri_from_result_uri(result_uri) + DocumentFactory.create_document(state_uri, State.SCHEMA) + document, _ = DocumentFactory.open_document(state_uri) + + states = [ + State({"loop_counter": 0, "i": 1, "payload": b"first"}), + State( + { + "loop_counter": 1, + "i": 2, + "payload": b"second", + "nested": {"values": [3, 4]}, + } + ), + ] + + writer = document.writer(str(uuid.uuid4())) + writer.open() + for state in states: + writer.put_one(state.to_tuple()) + writer.close() + + stored_rows = list(document.get()) + assert len(stored_rows) == len(states) + actual_states = sorted( + [State.from_tuple(row) for row in stored_rows], + key=lambda state: state["loop_counter"], + ) + assert actual_states == states diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index e49c0316cc7..bc2f069157e 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -17,8 +17,8 @@ import typing from loguru import logger -from pyarrow import Table from typing import Union +from pyarrow import Table from core.architecture.sendsemantics.broad_cast_partitioner import ( BroadcastPartitioner, @@ -34,7 +34,7 @@ from core.architecture.sendsemantics.round_robin_partitioner import ( RoundRobinPartitioner, ) -from core.models import Tuple, InternalQueue, DataFrame, DataPayload +from core.models import Tuple, InternalQueue, DataFrame, DataPayload, State, StateFrame from core.models.internal_queue import DataElement, ECMElement from core.storage.document_factory import DocumentFactory from core.util import Stoppable, get_one_of @@ -125,6 +125,15 @@ def tuple_to_batch_with_filter(self, tuple_: Tuple) -> typing.Iterator[DataFrame if receiver == self.worker_actor_id: yield self.tuples_to_data_frame(tuples) + def emit_state_with_filter(self, state: State) -> typing.Iterator[DataPayload]: + for receiver, payload in self.partitioner.flush_state(state): + if receiver == self.worker_actor_id: + yield ( + StateFrame(payload) + if isinstance(payload, State) + else self.tuples_to_data_frame(payload) + ) + def run(self) -> None: """ Main execution logic that reads tuples from the materialized storage and @@ -138,8 +147,21 @@ def run(self) -> None: self.uri ) self.emit_ecm("StartChannel", EmbeddedControlMessageType.NO_ALIGNMENT) - storage_iterator = self.materialization.get() + try: + state_document, _ = DocumentFactory.open_document( + State.uri_from_result_uri(self.uri) + ) + state_iterator = state_document.get() + for state in state_iterator: + for state_frame in self.emit_state_with_filter( + State.from_tuple(state) + ): + self.emit_payload(state_frame) + except ValueError: + pass + + storage_iterator = self.materialization.get() # Iterate and process tuples. for tup in storage_iterator: if self._stopped: diff --git a/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py new file mode 100644 index 00000000000..3662d023f59 --- /dev/null +++ b/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py @@ -0,0 +1,190 @@ +# 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. + +from unittest.mock import MagicMock, patch + +import pytest + +from core.models import State, StateFrame, Tuple +from core.models.internal_queue import DataElement +from core.models.schema import Schema +from core.storage.runnables.input_port_materialization_reader_runnable import ( + InputPortMaterializationReaderRunnable, +) +from proto.org.apache.texera.amber.core import ( + ActorVirtualIdentity, + ChannelIdentity, +) + + +class TestEmitStateWithFilter: + """Cover the partitioner-filter logic for state payloads in + InputPortMaterializationReaderRunnable. These tests bypass __init__ + so we don't need a real partitioner or storage URI. + """ + + @pytest.fixture + def me(self): + return ActorVirtualIdentity(name="me") + + @pytest.fixture + def someone_else(self): + return ActorVirtualIdentity(name="other") + + @pytest.fixture + def runnable(self, me): + # __new__ skips __init__ so we can wire only the fields we need. + instance = InputPortMaterializationReaderRunnable.__new__( + InputPortMaterializationReaderRunnable + ) + instance.worker_actor_id = me + instance.partitioner = MagicMock() + instance.tuple_schema = Schema(raw_schema={"x": "INTEGER"}) + return instance + + def test_yields_state_frame_for_matching_receiver(self, runnable, me): + state = State({"k": 1}) + runnable.partitioner.flush_state.return_value = [(me, state)] + + frames = list(runnable.emit_state_with_filter(state)) + + assert len(frames) == 1 + assert isinstance(frames[0], StateFrame) + assert frames[0].frame is state + + def test_filters_out_non_matching_receivers(self, runnable, me, someone_else): + state = State({"k": 1}) + runnable.partitioner.flush_state.return_value = [ + (someone_else, state), + (me, state), + (someone_else, state), + ] + + frames = list(runnable.emit_state_with_filter(state)) + + assert len(frames) == 1 + assert isinstance(frames[0], StateFrame) + + def test_yields_data_frame_for_non_state_payload(self, runnable, me): + # When the partitioner produces a tuple-batch payload (BroadcastPartitioner + # case), the runnable must convert it to a DataFrame instead of wrapping + # it as a StateFrame. + state = State({"k": 1}) + tuples = [Tuple({"x": 7}, schema=runnable.tuple_schema)] + runnable.partitioner.flush_state.return_value = [(me, tuples)] + + frames = list(runnable.emit_state_with_filter(state)) + + assert len(frames) == 1 + # Should not be wrapped as a StateFrame. + assert not isinstance(frames[0], StateFrame) + assert frames[0].frame.num_rows == 1 + + def test_empty_partitioner_output_yields_nothing(self, runnable): + state = State({}) + runnable.partitioner.flush_state.return_value = [] + + assert list(runnable.emit_state_with_filter(state)) == [] + + +class TestRunStateReadingBlock: + """Cover the inner try-block in run() that opens the state document and + emits its rows as StateFrames. + """ + + @pytest.fixture + def me(self): + return ActorVirtualIdentity(name="me") + + @pytest.fixture + def runnable(self, me): + instance = InputPortMaterializationReaderRunnable.__new__( + InputPortMaterializationReaderRunnable + ) + instance.uri = "vfs:///wf/0/exec/0/result/op-a" + instance.worker_actor_id = me + instance.tuple_schema = Schema(raw_schema={"x": "INTEGER"}) + instance._stopped = False + instance._finished = False + instance.channel_id = ChannelIdentity(me, me, is_control=False) + instance.queue = MagicMock() + instance.partitioner = MagicMock() + # No tuple-batches and no ECM-flush payloads in these tests. + instance.partitioner.flush.return_value = [] + return instance + + def test_state_rows_are_emitted_as_state_frames(self, runnable, me): + state_a = State({"loop_counter": 0}) + state_b = State({"loop_counter": 1}) + + # The state document yields opaque tuples; from_tuple deserializes + # them. Patch from_tuple so we don't have to wire a real serialization. + result_doc = MagicMock() + result_doc.get.return_value = iter([]) # No materialized tuples. + state_doc = MagicMock() + state_doc.get.return_value = iter(["row-a", "row-b"]) + + with ( + patch( + "core.storage.runnables.input_port_materialization_reader_runnable.DocumentFactory" + ) as mock_factory, + patch.object(State, "from_tuple") as mock_from_tuple, + ): + mock_factory.open_document.side_effect = [ + (result_doc, runnable.tuple_schema), + (state_doc, None), + ] + mock_from_tuple.side_effect = [state_a, state_b] + runnable.partitioner.flush_state.side_effect = [ + [(me, state_a)], + [(me, state_b)], + ] + + runnable.run() + + # Two StateFrames must have been put on the queue, in order. + state_frames = [ + call.args[0] + for call in runnable.queue.put.call_args_list + if isinstance(call.args[0], DataElement) + and isinstance(call.args[0].payload, StateFrame) + ] + assert [sf.payload.frame for sf in state_frames] == [state_a, state_b] + assert runnable._finished is True + + def test_missing_state_document_does_not_abort_run(self, runnable): + # The inner try is meant to swallow ValueError when no state document + # is provisioned; the outer run() should still finish cleanly. + result_doc = MagicMock() + result_doc.get.return_value = iter([]) + + with patch( + "core.storage.runnables.input_port_materialization_reader_runnable.DocumentFactory" + ) as mock_factory: + mock_factory.open_document.side_effect = [ + (result_doc, runnable.tuple_schema), + ValueError("no storage"), + ] + + runnable.run() + + assert runnable._finished is True + # No StateFrames should have been emitted. + for call in runnable.queue.put.call_args_list: + element = call.args[0] + if isinstance(element, DataElement): + assert not isinstance(element.payload, StateFrame) diff --git a/amber/src/main/python/core/storage/storage_config.py b/amber/src/main/python/core/storage/storage_config.py index 0e47bdb71ae..82335909874 100644 --- a/amber/src/main/python/core/storage/storage_config.py +++ b/amber/src/main/python/core/storage/storage_config.py @@ -32,6 +32,7 @@ class StorageConfig: ICEBERG_REST_CATALOG_URI = None ICEBERG_REST_CATALOG_WAREHOUSE_NAME = None ICEBERG_TABLE_RESULT_NAMESPACE = None + ICEBERG_TABLE_STATE_NAMESPACE = None ICEBERG_FILE_STORAGE_DIRECTORY_PATH = None ICEBERG_TABLE_COMMIT_BATCH_SIZE = None @@ -51,6 +52,7 @@ def initialize( rest_catalog_uri, rest_catalog_warehouse_name, table_result_namespace, + table_state_namespace, directory_path, commit_batch_size, s3_endpoint, @@ -71,6 +73,7 @@ def initialize( cls.ICEBERG_REST_CATALOG_WAREHOUSE_NAME = rest_catalog_warehouse_name cls.ICEBERG_TABLE_RESULT_NAMESPACE = table_result_namespace + cls.ICEBERG_TABLE_STATE_NAMESPACE = table_state_namespace cls.ICEBERG_FILE_STORAGE_DIRECTORY_PATH = directory_path cls.ICEBERG_TABLE_COMMIT_BATCH_SIZE = int(commit_batch_size) diff --git a/amber/src/main/python/core/storage/test_document_factory.py b/amber/src/main/python/core/storage/test_document_factory.py new file mode 100644 index 00000000000..859c0040246 --- /dev/null +++ b/amber/src/main/python/core/storage/test_document_factory.py @@ -0,0 +1,134 @@ +# 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. + +from unittest.mock import MagicMock, patch + +import pytest + +from core.models import Schema +from core.storage.document_factory import DocumentFactory +from core.storage.storage_config import StorageConfig +from core.storage.vfs_uri_factory import VFSResourceType + + +# Avoid initializing the real config (only initializable once per process). +StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE = "test-result-ns" +StorageConfig.ICEBERG_TABLE_STATE_NAMESPACE = "test-state-ns" + +VFS_URI = "vfs:///wid/0/eid/0/opid/test/main/0/0/result" + + +@pytest.fixture +def schema(): + return Schema(raw_schema={"x": "INTEGER"}) + + +def _decode_returning(resource_type): + """Helper: build a VFSURIFactory.decode_uri side_effect.""" + return lambda _uri: (None, None, None, resource_type) + + +@patch("core.storage.document_factory.IcebergDocument") +@patch("core.storage.document_factory.amber_schema_to_iceberg_schema") +@patch("core.storage.document_factory.create_table") +@patch("core.storage.document_factory.IcebergCatalogInstance") +@patch("core.storage.document_factory.VFSURIFactory") +class TestCreateDocumentNamespaceRouting: + def test_state_resource_type_uses_state_namespace( + self, mock_vfs, _icb, mock_create_table, _amber_schema, _doc, schema + ): + mock_vfs.VFS_FILE_URI_SCHEME = "vfs" + mock_vfs.decode_uri.side_effect = _decode_returning(VFSResourceType.STATE) + + DocumentFactory.create_document(VFS_URI, schema) + + args, _ = mock_create_table.call_args + assert args[1] == StorageConfig.ICEBERG_TABLE_STATE_NAMESPACE + + def test_result_resource_type_uses_result_namespace( + self, mock_vfs, _icb, mock_create_table, _amber_schema, _doc, schema + ): + mock_vfs.VFS_FILE_URI_SCHEME = "vfs" + mock_vfs.decode_uri.side_effect = _decode_returning(VFSResourceType.RESULT) + + DocumentFactory.create_document(VFS_URI, schema) + + args, _ = mock_create_table.call_args + assert args[1] == StorageConfig.ICEBERG_TABLE_RESULT_NAMESPACE + + def test_unsupported_resource_type_raises_value_error( + self, mock_vfs, _icb, _create_table, _amber_schema, _doc, schema + ): + mock_vfs.VFS_FILE_URI_SCHEME = "vfs" + # CONSOLE_MESSAGES has no namespace mapping in the Python factory. + mock_vfs.decode_uri.side_effect = _decode_returning( + VFSResourceType.CONSOLE_MESSAGES + ) + + with pytest.raises(ValueError, match="not supported"): + DocumentFactory.create_document(VFS_URI, schema) + + +def test_create_document_rejects_non_vfs_scheme(schema): + with pytest.raises(NotImplementedError, match="Unsupported URI scheme"): + DocumentFactory.create_document("file:///tmp/x", schema) + + +@patch("core.storage.document_factory.IcebergDocument") +@patch("core.storage.document_factory.Schema") +@patch("core.storage.document_factory.load_table_metadata") +@patch("core.storage.document_factory.IcebergCatalogInstance") +@patch("core.storage.document_factory.VFSURIFactory") +class TestOpenDocumentNamespaceRouting: + @staticmethod + def _stub_table(): + table = MagicMock() + table.schema.return_value.as_arrow.return_value = MagicMock() + return table + + def test_state_resource_type_uses_state_namespace( + self, mock_vfs, _icb, mock_load, _schema_cls, _doc + ): + mock_vfs.VFS_FILE_URI_SCHEME = "vfs" + mock_vfs.decode_uri.side_effect = _decode_returning(VFSResourceType.STATE) + mock_load.return_value = self._stub_table() + + DocumentFactory.open_document(VFS_URI) + + args, _ = mock_load.call_args + assert args[1] == StorageConfig.ICEBERG_TABLE_STATE_NAMESPACE + + def test_unsupported_resource_type_raises_value_error( + self, mock_vfs, _icb, _load, _schema_cls, _doc + ): + mock_vfs.VFS_FILE_URI_SCHEME = "vfs" + mock_vfs.decode_uri.side_effect = _decode_returning( + VFSResourceType.CONSOLE_MESSAGES + ) + + with pytest.raises(ValueError, match="not supported"): + DocumentFactory.open_document(VFS_URI) + + def test_missing_table_raises_value_error( + self, mock_vfs, _icb, mock_load, _schema_cls, _doc + ): + mock_vfs.VFS_FILE_URI_SCHEME = "vfs" + mock_vfs.decode_uri.side_effect = _decode_returning(VFSResourceType.STATE) + mock_load.return_value = None + + with pytest.raises(ValueError, match="No storage is found"): + DocumentFactory.open_document(VFS_URI) diff --git a/amber/src/main/python/core/storage/vfs_uri_factory.py b/amber/src/main/python/core/storage/vfs_uri_factory.py index de0c5db56ec..0e23e607055 100644 --- a/amber/src/main/python/core/storage/vfs_uri_factory.py +++ b/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -34,6 +34,7 @@ class VFSResourceType(str, Enum): RESULT = "result" RUNTIME_STATISTICS = "runtimeStatistics" CONSOLE_MESSAGES = "consoleMessages" + STATE = "state" class VFSURIFactory: diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_manager.py b/amber/src/main/python/pytexera/storage/test_large_binary_manager.py index 64c7080e520..1942e91f8bc 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_manager.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_manager.py @@ -34,6 +34,7 @@ def setup_storage_config(self): rest_catalog_uri="http://localhost:8181/catalog/", rest_catalog_warehouse_name="texera", table_result_namespace="test", + table_state_namespace="test-state", directory_path="/tmp/test", commit_batch_size=1000, s3_endpoint="http://localhost:9000", diff --git a/amber/src/main/python/texera_run_python_worker.py b/amber/src/main/python/texera_run_python_worker.py index 8687298f819..9b21fa53343 100644 --- a/amber/src/main/python/texera_run_python_worker.py +++ b/amber/src/main/python/texera_run_python_worker.py @@ -52,6 +52,7 @@ def init_loguru_logger(stream_log_level) -> None: iceberg_rest_catalog_uri, iceberg_rest_catalog_warehouse_name, iceberg_table_namespace, + iceberg_table_state_namespace, iceberg_file_storage_directory_path, iceberg_table_commit_batch_size, s3_endpoint, @@ -68,6 +69,7 @@ def init_loguru_logger(stream_log_level) -> None: iceberg_rest_catalog_uri, iceberg_rest_catalog_warehouse_name, iceberg_table_namespace, + iceberg_table_state_namespace, iceberg_file_storage_directory_path, iceberg_table_commit_batch_size, s3_endpoint, diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index 4ab3d18056f..80f22ace790 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -124,6 +124,9 @@ class OutputManager( : mutable.HashMap[PortIdentity, OutputPortResultWriterThread] = mutable.HashMap() + private val stateWriters: mutable.HashMap[PortIdentity, BufferedItemWriter[Tuple]] = + mutable.HashMap() + /** * Add down stream operator and its corresponding Partitioner. * @@ -232,6 +235,10 @@ class OutputManager( }) } + def saveStateToStorageIfNeeded(state: State): Unit = { + stateWriters.values.foreach(_.putOne(state.toTuple)) + } + /** * Singal the port storage writer to flush the remaining buffer and wait for commits to finish so that * the output port is properly completed. If the output port does not need storage, no action will be done. @@ -245,7 +252,7 @@ class OutputManager( writerThread.join() case None => } - + this.stateWriters.remove(outputPortId).foreach(_.close()) } def getPort(portId: PortIdentity): WorkerPort = ports(portId) @@ -288,6 +295,16 @@ class OutputManager( val writerThread = new OutputPortResultWriterThread(bufferedItemWriter) this.outputPortResultWriterThreads(portId) = writerThread writerThread.start() + + // The state document is provisioned alongside the result document + // by RegionExecutionCoordinator, so it is always present. + val stateWriter = DocumentFactory + .openDocument(State.uriFromResultUri(storageUri)) + ._1 + .writer(VirtualIdentityUtils.getWorkerIndex(actorId).toString) + .asInstanceOf[BufferedItemWriter[Tuple]] + stateWriter.open() + this.stateWriters(portId) = stateWriter } } diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala index 4ff5ff15ae3..3358e31e65f 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala @@ -187,6 +187,7 @@ class PythonWorkflowWorker( if (isRest) StorageConfig.icebergRESTCatalogUri else "", if (isRest) StorageConfig.icebergRESTCatalogWarehouseName else "", StorageConfig.icebergTableResultNamespace, + StorageConfig.icebergTableStateNamespace, StorageConfig.fileStorageDirectoryPath.toString, StorageConfig.icebergTableCommitBatchSize.toString, StorageConfig.s3Endpoint, diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala index 254c16bf34b..58fdf9f2428 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala @@ -21,6 +21,7 @@ package org.apache.texera.amber.engine.architecture.scheduling import org.apache.pekko.pattern.gracefulStop import com.twitter.util.{Duration => TwitterDuration, Future, JavaTimer, Return, Throw, Timer} +import org.apache.texera.amber.core.state.State import org.apache.texera.amber.core.storage.DocumentFactory import org.apache.texera.amber.core.storage.VFSURIFactory.decodeURI import org.apache.texera.amber.core.virtualidentity.ActorVirtualIdentity @@ -569,12 +570,14 @@ class RegionExecutionCoordinator( portConfigs.foreach { case (outputPortId, portConfig) => val storageUriToAdd = portConfig.storageURI + val stateUriToAdd = State.uriFromResultUri(storageUriToAdd) val (_, eid, _, _) = decodeURI(storageUriToAdd) val schemaOptional = region.getOperator(outputPortId.opId).outputPorts(outputPortId.portId)._3 val schema = schemaOptional.getOrElse(throw new IllegalStateException("Schema is missing")) DocumentFactory.createDocument(storageUriToAdd, schema) + DocumentFactory.createDocument(stateUriToAdd, State.schema) if (!isRestart) { WorkflowExecutionsResource.insertOperatorPortResultUri( eid = eid, diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala index 84f1e8ec659..b6c0c39aaff 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala @@ -126,6 +126,7 @@ class DataProcessor( val outputState = executor.processState(state, port) if (outputState.isDefined) { outputManager.emitState(outputState.get) + outputManager.saveStateToStorageIfNeeded(outputState.get) } } catch safely { case e => diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 10fbbc44a2c..90de86e1fda 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -21,6 +21,7 @@ package org.apache.texera.amber.engine.architecture.worker.managers import io.grpc.MethodDescriptor import org.apache.texera.amber.config.ApplicationConfig +import org.apache.texera.amber.core.state.State import org.apache.texera.amber.core.storage.DocumentFactory import org.apache.texera.amber.core.storage.model.VirtualDocument import org.apache.texera.amber.core.tuple.Tuple @@ -45,7 +46,11 @@ import org.apache.texera.amber.engine.architecture.worker.WorkflowWorker.{ DPInputQueueElement, FIFOMessageElement } -import org.apache.texera.amber.engine.common.ambermessage.{DataFrame, WorkflowFIFOMessage} +import org.apache.texera.amber.engine.common.ambermessage.{ + DataFrame, + StateFrame, + WorkflowFIFOMessage +} import org.apache.texera.amber.util.VirtualIdentityUtils.getFromActorIdForInputPortStorage import java.net.URI @@ -106,6 +111,25 @@ class InputPortMaterializationReaderThread( } // Flush any remaining tuples in the buffer. if (buffer.nonEmpty) flush() + + try { + val state_document = + DocumentFactory + .openDocument(State.uriFromResultUri(uri)) + ._1 + .asInstanceOf[VirtualDocument[Tuple]] + val stateReadIterator = state_document.get() + + while (stateReadIterator.hasNext) { + val state = State.fromTuple(stateReadIterator.next()) + inputMessageQueue.put( + FIFOMessageElement(WorkflowFIFOMessage(channelId, getSequenceNumber, StateFrame(state))) + ) + } + } catch { + case _: Exception => + } + emitECM(METHOD_END_CHANNEL, PORT_ALIGNMENT) isFinished.set(true) } catch { diff --git a/common/config/src/main/resources/storage.conf b/common/config/src/main/resources/storage.conf index 1f39359155c..da2f7ccc198 100644 --- a/common/config/src/main/resources/storage.conf +++ b/common/config/src/main/resources/storage.conf @@ -61,6 +61,9 @@ storage { runtime-statistics-namespace = "workflow-runtime-statistics" runtime-statistics-namespace = ${?STORAGE_ICEBERG_TABLE_RUNTIME_STATISTICS_NAMESPACE} + state-namespace = "operator-port-state" + state-namespace = ${?STORAGE_ICEBERG_TABLE_STATE_NAMESPACE} + commit { batch-size = 4096 # decide the buffer size of our IcebergTableWriter batch-size = ${?STORAGE_ICEBERG_TABLE_COMMIT_BATCH_SIZE} diff --git a/common/config/src/main/scala/org/apache/texera/amber/config/EnvironmentalVariable.scala b/common/config/src/main/scala/org/apache/texera/amber/config/EnvironmentalVariable.scala index 9ec52bba653..123c56505ee 100644 --- a/common/config/src/main/scala/org/apache/texera/amber/config/EnvironmentalVariable.scala +++ b/common/config/src/main/scala/org/apache/texera/amber/config/EnvironmentalVariable.scala @@ -67,6 +67,7 @@ object EnvironmentalVariable { "STORAGE_ICEBERG_TABLE_CONSOLE_MESSAGES_NAMESPACE" val ENV_ICEBERG_TABLE_RUNTIME_STATISTICS_NAMESPACE = "STORAGE_ICEBERG_TABLE_RUNTIME_STATISTICS_NAMESPACE" + val ENV_ICEBERG_TABLE_STATE_NAMESPACE = "STORAGE_ICEBERG_TABLE_STATE_NAMESPACE" val ENV_ICEBERG_TABLE_COMMIT_BATCH_SIZE = "STORAGE_ICEBERG_TABLE_COMMIT_BATCH_SIZE" val ENV_ICEBERG_TABLE_COMMIT_NUM_RETRIES = "STORAGE_ICEBERG_TABLE_COMMIT_NUM_RETRIES" val ENV_ICEBERG_TABLE_COMMIT_MIN_WAIT_MS = "STORAGE_ICEBERG_TABLE_COMMIT_MIN_WAIT_MS" diff --git a/common/config/src/main/scala/org/apache/texera/amber/config/StorageConfig.scala b/common/config/src/main/scala/org/apache/texera/amber/config/StorageConfig.scala index 728e3c0c2de..07447cfdbee 100644 --- a/common/config/src/main/scala/org/apache/texera/amber/config/StorageConfig.scala +++ b/common/config/src/main/scala/org/apache/texera/amber/config/StorageConfig.scala @@ -54,6 +54,8 @@ object StorageConfig { conf.getString("storage.iceberg.table.console-messages-namespace") val icebergTableRuntimeStatisticsNamespace: String = conf.getString("storage.iceberg.table.runtime-statistics-namespace") + val icebergTableStateNamespace: String = + conf.getString("storage.iceberg.table.state-namespace") val icebergTableCommitBatchSize: Int = conf.getInt("storage.iceberg.table.commit.batch-size") val icebergTableCommitNumRetries: Int = @@ -111,6 +113,7 @@ object StorageConfig { "STORAGE_ICEBERG_TABLE_CONSOLE_MESSAGES_NAMESPACE" val ENV_ICEBERG_TABLE_RUNTIME_STATISTICS_NAMESPACE = "STORAGE_ICEBERG_TABLE_RUNTIME_STATISTICS_NAMESPACE" + val ENV_ICEBERG_TABLE_STATE_NAMESPACE = "STORAGE_ICEBERG_TABLE_STATE_NAMESPACE" val ENV_ICEBERG_TABLE_COMMIT_BATCH_SIZE = "STORAGE_ICEBERG_TABLE_COMMIT_BATCH_SIZE" val ENV_ICEBERG_TABLE_COMMIT_NUM_RETRIES = "STORAGE_ICEBERG_TABLE_COMMIT_NUM_RETRIES" val ENV_ICEBERG_TABLE_COMMIT_MIN_WAIT_MS = "STORAGE_ICEBERG_TABLE_COMMIT_MIN_WAIT_MS" diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala index ba146f1d57c..532f355c17e 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.JsonNode import org.apache.texera.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} import org.apache.texera.amber.util.JSONUtils.objectMapper +import java.net.URI import java.util.Base64 import scala.jdk.CollectionConverters.IteratorHasAsScala @@ -57,6 +58,9 @@ object State { def fromTuple(row: Tuple): State = fromJson(row.getField[String](Content)) + def uriFromResultUri(resultUri: URI): URI = + new URI(resultUri.toString.replace("/result", "/state")) + private def toJsonValue(value: Any): Any = value match { case null => null diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/DocumentFactory.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/DocumentFactory.scala index 15949ef4717..00f6c70ba73 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/DocumentFactory.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/DocumentFactory.scala @@ -72,6 +72,7 @@ object DocumentFactory { case RESULT => StorageConfig.icebergTableResultNamespace case CONSOLE_MESSAGES => StorageConfig.icebergTableConsoleMessagesNamespace case RUNTIME_STATISTICS => StorageConfig.icebergTableRuntimeStatisticsNamespace + case STATE => StorageConfig.icebergTableStateNamespace case _ => throw new IllegalArgumentException(s"Resource type $resourceType is not supported") } @@ -119,6 +120,7 @@ object DocumentFactory { case RESULT => StorageConfig.icebergTableResultNamespace case CONSOLE_MESSAGES => StorageConfig.icebergTableConsoleMessagesNamespace case RUNTIME_STATISTICS => StorageConfig.icebergTableRuntimeStatisticsNamespace + case STATE => StorageConfig.icebergTableStateNamespace case _ => throw new IllegalArgumentException(s"Resource type $resourceType is not supported") } diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala index 0fbee64457d..e687b28a29b 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala @@ -34,6 +34,7 @@ object VFSResourceType extends Enumeration { val RESULT: Value = Value("result") val RUNTIME_STATISTICS: Value = Value("runtimeStatistics") val CONSOLE_MESSAGES: Value = Value("consoleMessages") + val STATE: Value = Value("state") } object VFSURIFactory { diff --git a/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 8fdf039f3ea..7f1d8573c2a 100644 --- a/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -20,6 +20,7 @@ package org.apache.texera.amber.storage.result.iceberg import org.apache.texera.amber.config.StorageConfig +import org.apache.texera.amber.core.state.State import org.apache.texera.amber.core.storage.model.{VirtualDocument, VirtualDocumentSpec} import org.apache.texera.amber.core.storage.{DocumentFactory, IcebergCatalogInstance, VFSURIFactory} import org.apache.texera.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} @@ -141,6 +142,84 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with BeforeAndAfter } } + it should "round trip materialized state documents" in { + val stateUri = State.uriFromResultUri(uri) + DocumentFactory.createDocument(stateUri, State.schema) + val stateDocument = + DocumentFactory.openDocument(stateUri)._1.asInstanceOf[VirtualDocument[Tuple]] + val state = State( + Map( + "loop_counter" -> 3, + "name" -> "outer-loop", + "payload" -> Array[Byte](0, 1, 2, 3), + "nested" -> Map("enabled" -> true, "values" -> List(1, 2, 3)) + ) + ) + + val writer = stateDocument.writer(UUID.randomUUID().toString) + writer.open() + writer.putOne(state.toTuple) + writer.close() + + val storedRows = stateDocument.get().toList + assert(storedRows.length == 1) + val deserialized = State.fromTuple(storedRows.head).values + assert(deserialized("loop_counter") == 3L) + assert(deserialized("name") == "outer-loop") + assert(deserialized("payload").asInstanceOf[Array[Byte]].sameElements(Array[Byte](0, 1, 2, 3))) + assert(deserialized("nested").asInstanceOf[Map[String, Any]]("enabled") == true) + assert(deserialized("nested").asInstanceOf[Map[String, Any]]("values") == List(1L, 2L, 3L)) + } + + it should "materialize multiple states as rows in one state table" in { + val stateUri = State.uriFromResultUri(uri) + DocumentFactory.createDocument(stateUri, State.schema) + val stateDocument = + DocumentFactory.openDocument(stateUri)._1.asInstanceOf[VirtualDocument[Tuple]] + val states: List[State] = List( + State(Map("loop_counter" -> 0, "i" -> 1, "payload" -> Array[Byte](1, 2, 3))), + State( + Map( + "loop_counter" -> 1, + "i" -> 2, + "payload" -> Array[Byte](4, 5, 6), + "nested" -> Map("values" -> List(3, 4)) + ) + ) + ) + + val writer = stateDocument.writer(UUID.randomUUID().toString) + writer.open() + states.foreach(state => writer.putOne(state.toTuple)) + writer.close() + + val deserializedStates = + stateDocument + .get() + .toList + .map(State.fromTuple) + .sortBy(_.values("loop_counter").asInstanceOf[Long]) + assert(deserializedStates.length == states.length) + deserializedStates.zip(states).foreach { + case (actual, expected) => + assert( + actual.values("loop_counter") == expected.values("loop_counter").asInstanceOf[Int].toLong + ) + assert(actual.values("i") == expected.values("i").asInstanceOf[Int].toLong) + assert( + actual + .values("payload") + .asInstanceOf[Array[Byte]] + .sameElements(expected.values("payload").asInstanceOf[Array[Byte]]) + ) + } + assert( + deserializedStates(1) + .values("nested") + .asInstanceOf[Map[String, Any]]("values") == List(3L, 4L) + ) + } + /** Returns a dynamic proxy for `realTable` that increments `counter` on every `refresh()` call. */ private def tableWithRefreshSpy(realTable: Table, counter: AtomicInteger): Table = Proxy From b90ffca8bb6708ce414bff3dc6954f9de0a4a1f8 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Mon, 4 May 2026 23:03:09 -0700 Subject: [PATCH 02/23] fix --- .../main/python/core/architecture/packaging/output_manager.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 08fa210eca5..f940d575205 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -182,9 +182,6 @@ def save_tuple_to_storage_if_needed(self, tuple_: Tuple, port_id=None) -> None: ) def save_state_to_storage_if_needed(self, state: State, port_id=None) -> None: - # Buffer the state on each long-lived writer; the writer flushes - # itself when its buffer fills, and the remaining buffer is - # flushed in close_port_storage_writers. if port_id is None: writers = self._state_writers.values() elif port_id in self._state_writers: From 885846fafd59e5f481c8a81f7e8612e25e187f83 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Mon, 4 May 2026 23:04:40 -0700 Subject: [PATCH 03/23] fix --- .../main/python/core/architecture/packaging/output_manager.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index f940d575205..5614a64a6b3 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -204,8 +204,6 @@ def close_port_storage_writers(self) -> None: for _, _, writer_thread in self._port_storage_writers.values(): # This blocking call will wait for all the writer to finish commit writer_thread.join() - # Close the long-lived state writers so the remaining buffered - # states are committed in a single Iceberg snapshot per port. for state_writer in self._state_writers.values(): state_writer.close() self._state_writers.clear() From 74fda4f77a10fc7d02c8d936630de73bc5545ac6 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 00:21:06 -0700 Subject: [PATCH 04/23] fix --- ...ut_port_materialization_reader_runnable.py | 20 ++++++-------- ...ut_port_materialization_reader_runnable.py | 22 --------------- ...InputPortMaterializationReaderThread.scala | 27 ++++++++----------- 3 files changed, 19 insertions(+), 50 deletions(-) diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index bc2f069157e..3498767da12 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -148,18 +148,14 @@ def run(self) -> None: ) self.emit_ecm("StartChannel", EmbeddedControlMessageType.NO_ALIGNMENT) - try: - state_document, _ = DocumentFactory.open_document( - State.uri_from_result_uri(self.uri) - ) - state_iterator = state_document.get() - for state in state_iterator: - for state_frame in self.emit_state_with_filter( - State.from_tuple(state) - ): - self.emit_payload(state_frame) - except ValueError: - pass + state_document, _ = DocumentFactory.open_document( + State.uri_from_result_uri(self.uri) + ) + for state in state_document.get(): + for state_frame in self.emit_state_with_filter( + State.from_tuple(state) + ): + self.emit_payload(state_frame) storage_iterator = self.materialization.get() # Iterate and process tuples. diff --git a/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py index 3662d023f59..838ba7b9104 100644 --- a/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py @@ -166,25 +166,3 @@ def test_state_rows_are_emitted_as_state_frames(self, runnable, me): assert [sf.payload.frame for sf in state_frames] == [state_a, state_b] assert runnable._finished is True - def test_missing_state_document_does_not_abort_run(self, runnable): - # The inner try is meant to swallow ValueError when no state document - # is provisioned; the outer run() should still finish cleanly. - result_doc = MagicMock() - result_doc.get.return_value = iter([]) - - with patch( - "core.storage.runnables.input_port_materialization_reader_runnable.DocumentFactory" - ) as mock_factory: - mock_factory.open_document.side_effect = [ - (result_doc, runnable.tuple_schema), - ValueError("no storage"), - ] - - runnable.run() - - assert runnable._finished is True - # No StateFrames should have been emitted. - for call in runnable.queue.put.call_args_list: - element = call.args[0] - if isinstance(element, DataElement): - assert not isinstance(element.payload, StateFrame) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 90de86e1fda..8d086327088 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -112,22 +112,17 @@ class InputPortMaterializationReaderThread( // Flush any remaining tuples in the buffer. if (buffer.nonEmpty) flush() - try { - val state_document = - DocumentFactory - .openDocument(State.uriFromResultUri(uri)) - ._1 - .asInstanceOf[VirtualDocument[Tuple]] - val stateReadIterator = state_document.get() - - while (stateReadIterator.hasNext) { - val state = State.fromTuple(stateReadIterator.next()) - inputMessageQueue.put( - FIFOMessageElement(WorkflowFIFOMessage(channelId, getSequenceNumber, StateFrame(state))) - ) - } - } catch { - case _: Exception => + val stateDocument = + DocumentFactory + .openDocument(State.uriFromResultUri(uri)) + ._1 + .asInstanceOf[VirtualDocument[Tuple]] + val stateReadIterator = stateDocument.get() + while (stateReadIterator.hasNext) { + val state = State.fromTuple(stateReadIterator.next()) + inputMessageQueue.put( + FIFOMessageElement(WorkflowFIFOMessage(channelId, getSequenceNumber, StateFrame(state))) + ) } emitECM(METHOD_END_CHANNEL, PORT_ALIGNMENT) From 04ccda6bde22c5ff1978969137b2f36f582a0de5 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 15:06:53 -0700 Subject: [PATCH 05/23] fix --- ...InputPortMaterializationReaderThread.scala | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 8d086327088..36998e729c4 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -89,6 +89,19 @@ class InputPortMaterializationReaderThread( // Notify the input port of start of input channel emitECM(METHOD_START_CHANNEL, NO_ALIGNMENT) try { + val stateDocument = + DocumentFactory + .openDocument(State.uriFromResultUri(uri)) + ._1 + .asInstanceOf[VirtualDocument[Tuple]] + val stateReadIterator = stateDocument.get() + while (stateReadIterator.hasNext) { + val state = State.fromTuple(stateReadIterator.next()) + inputMessageQueue.put( + FIFOMessageElement(WorkflowFIFOMessage(channelId, getSequenceNumber, StateFrame(state))) + ) + } + val materialization: VirtualDocument[Tuple] = DocumentFactory .openDocument(uri) ._1 @@ -112,19 +125,6 @@ class InputPortMaterializationReaderThread( // Flush any remaining tuples in the buffer. if (buffer.nonEmpty) flush() - val stateDocument = - DocumentFactory - .openDocument(State.uriFromResultUri(uri)) - ._1 - .asInstanceOf[VirtualDocument[Tuple]] - val stateReadIterator = stateDocument.get() - while (stateReadIterator.hasNext) { - val state = State.fromTuple(stateReadIterator.next()) - inputMessageQueue.put( - FIFOMessageElement(WorkflowFIFOMessage(channelId, getSequenceNumber, StateFrame(state))) - ) - } - emitECM(METHOD_END_CHANNEL, PORT_ALIGNMENT) isFinished.set(true) } catch { From 1fc823f19733c343447df668efc20d76392c431a Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 19:08:06 -0700 Subject: [PATCH 06/23] fix --- .../engine/architecture/messaginglayer/OutputManager.scala | 4 ++++ .../amber/engine/architecture/worker/DataProcessor.scala | 1 - 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index 80f22ace790..d13aeeccfd2 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -194,6 +194,10 @@ class OutputManager( def emitState(state: State): Unit = { networkOutputBuffers.foreach(kv => kv._2.sendState(state)) + // Persist alongside emission so any caller that pushes state + // downstream (DataProcessor.processInputState, StartChannelHandler, + // EndChannelHandler) cannot silently skip materialization. + saveStateToStorageIfNeeded(state) } def addPort(portId: PortIdentity, schema: Schema, storageURIOption: Option[URI]): Unit = { diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala index b6c0c39aaff..84f1e8ec659 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/DataProcessor.scala @@ -126,7 +126,6 @@ class DataProcessor( val outputState = executor.processState(state, port) if (outputState.isDefined) { outputManager.emitState(outputState.get) - outputManager.saveStateToStorageIfNeeded(outputState.get) } } catch safely { case e => From 1e77da78b3c3ca9e843d264f070ed72c2c640e19 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 19:28:41 -0700 Subject: [PATCH 07/23] fix fmt --- .../runnables/input_port_materialization_reader_runnable.py | 4 +--- .../test_input_port_materialization_reader_runnable.py | 1 - 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index 3498767da12..dd356a37953 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -152,9 +152,7 @@ def run(self) -> None: State.uri_from_result_uri(self.uri) ) for state in state_document.get(): - for state_frame in self.emit_state_with_filter( - State.from_tuple(state) - ): + for state_frame in self.emit_state_with_filter(State.from_tuple(state)): self.emit_payload(state_frame) storage_iterator = self.materialization.get() diff --git a/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py b/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py index 838ba7b9104..c90ef91f83c 100644 --- a/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py +++ b/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py @@ -165,4 +165,3 @@ def test_state_rows_are_emitted_as_state_frames(self, runnable, me): ] assert [sf.payload.frame for sf in state_frames] == [state_a, state_b] assert runnable._finished is True - From 06ef4f8b6ca545cdd76f5838a13f5a92e11b4fce Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 21:40:01 -0700 Subject: [PATCH 08/23] fix fmt --- .../runnables/input_port_materialization_reader_runnable.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index dd356a37953..af847621efa 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -17,8 +17,8 @@ import typing from loguru import logger -from typing import Union from pyarrow import Table +from typing import Union from core.architecture.sendsemantics.broad_cast_partitioner import ( BroadcastPartitioner, From e2f5ea1805b21b13cd13c5ff6bf2f11b167f66cf Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 21:40:57 -0700 Subject: [PATCH 09/23] fix fmt --- .../worker/managers/InputPortMaterializationReaderThread.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 36998e729c4..7439430ca2a 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -124,7 +124,6 @@ class InputPortMaterializationReaderThread( } // Flush any remaining tuples in the buffer. if (buffer.nonEmpty) flush() - emitECM(METHOD_END_CHANNEL, PORT_ALIGNMENT) isFinished.set(true) } catch { From 6d7d88e0599c05ed187139d5a76d9f25358d202f Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 21:41:37 -0700 Subject: [PATCH 10/23] fix fmt --- .../engine/architecture/messaginglayer/OutputManager.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index d13aeeccfd2..a39ac1dcb46 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -194,9 +194,6 @@ class OutputManager( def emitState(state: State): Unit = { networkOutputBuffers.foreach(kv => kv._2.sendState(state)) - // Persist alongside emission so any caller that pushes state - // downstream (DataProcessor.processInputState, StartChannelHandler, - // EndChannelHandler) cannot silently skip materialization. saveStateToStorageIfNeeded(state) } From 8e27025f6a93b72a99aa582d42ca906868349686 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 22:15:56 -0700 Subject: [PATCH 11/23] docs: explain state-before-tuple replay order in materialization readers Address PR #4490 review comment 3192875005: document why the input-port materialization reader replays states before tuples (downstream operators typically need their state in place before processing the incoming tuples). Co-Authored-By: Claude Opus 4.7 (1M context) --- .../runnables/input_port_materialization_reader_runnable.py | 5 +++++ .../managers/InputPortMaterializationReaderThread.scala | 4 ++++ 2 files changed, 9 insertions(+) diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index af847621efa..67c6370c006 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -148,6 +148,11 @@ def run(self) -> None: ) self.emit_ecm("StartChannel", EmbeddedControlMessageType.NO_ALIGNMENT) + # States and tuples are persisted to separate tables, so + # the original interleaving is lost and replay has to pick + # an order: we replay states first because downstream + # operators typically need their state set up before they + # process the incoming tuples. state_document, _ = DocumentFactory.open_document( State.uri_from_result_uri(self.uri) ) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 7439430ca2a..586d6ba207a 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -89,6 +89,10 @@ class InputPortMaterializationReaderThread( // Notify the input port of start of input channel emitECM(METHOD_START_CHANNEL, NO_ALIGNMENT) try { + // States and tuples are persisted to separate tables, so the + // original interleaving is lost and replay has to pick an order: + // we replay states first because downstream operators typically + // need their state set up before they process the incoming tuples. val stateDocument = DocumentFactory .openDocument(State.uriFromResultUri(uri)) From df9e360d6562d14baa516fa86f631ad1ffc358b6 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Tue, 5 May 2026 23:08:32 -0700 Subject: [PATCH 12/23] docs: note state replay broadcasts to every worker Address PR #4490 review comment 3192889029: explain why the state loop intentionally enqueues every row to every downstream worker while the tuple loop filters by partitioner -- state is shared context, not per-key data. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../managers/InputPortMaterializationReaderThread.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 586d6ba207a..98c68731aee 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -99,6 +99,10 @@ class InputPortMaterializationReaderThread( ._1 .asInstanceOf[VirtualDocument[Tuple]] val stateReadIterator = stateDocument.get() + // Every state is broadcast to every downstream worker -- no + // partitioner filtering here, unlike the tuple loop below. State + // is shared context (e.g. config / counters), not per-key data, + // so each worker needs the full set. while (stateReadIterator.hasNext) { val state = State.fromTuple(stateReadIterator.next()) inputMessageQueue.put( From 4428c9d5b0c9db35d6a018e9bb5571784c4ce735 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Wed, 6 May 2026 03:08:40 -0700 Subject: [PATCH 13/23] update --- .../architecture/packaging/output_manager.py | 46 +++++--- .../messaginglayer/OutputManager.scala | 16 +-- .../packaging/test_output_manager.py | 100 +++++++----------- 3 files changed, 82 insertions(+), 80 deletions(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 5614a64a6b3..1220a9f15f3 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -87,7 +87,9 @@ def __init__(self, worker_id: str): PortIdentity, typing.Tuple[Queue, PortStorageWriter, Thread] ] = dict() - self._state_writers: typing.Dict[PortIdentity, typing.Any] = dict() + self._port_state_writers: typing.Dict[ + PortIdentity, typing.Tuple[Queue, PortStorageWriter, Thread] + ] = dict() def is_missing_output_ports(self): """ @@ -150,9 +152,25 @@ def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): state_document, _ = DocumentFactory.open_document( State.uri_from_result_uri(storage_uri) ) - state_writer = state_document.writer(str(get_worker_index(self.worker_id))) - state_writer.open() - self._state_writers[port_id] = state_writer + state_buffered_item_writer = state_document.writer( + str(get_worker_index(self.worker_id)) + ) + state_writer_queue = Queue() + state_port_writer = PortStorageWriter( + buffered_item_writer=state_buffered_item_writer, + queue=state_writer_queue, + ) + state_writer_thread = threading.Thread( + target=state_port_writer.run, + daemon=True, + name=f"port_state_writer_thread_{port_id}", + ) + state_writer_thread.start() + self._port_state_writers[port_id] = ( + state_writer_queue, + state_port_writer, + state_writer_thread, + ) def get_port(self, port_id=None) -> WorkerPort: return list(self._ports.values())[0] @@ -182,14 +200,12 @@ def save_tuple_to_storage_if_needed(self, tuple_: Tuple, port_id=None) -> None: ) def save_state_to_storage_if_needed(self, state: State, port_id=None) -> None: + element = PortStorageWriterElement(data_tuple=state.to_tuple()) if port_id is None: - writers = self._state_writers.values() - elif port_id in self._state_writers: - writers = [self._state_writers[port_id]] - else: - return - for writer in writers: - writer.put_one(state.to_tuple()) + for writer_queue, _, _ in self._port_state_writers.values(): + writer_queue.put(element) + elif port_id in self._port_state_writers: + self._port_state_writers[port_id][0].put(element) def close_port_storage_writers(self) -> None: """ @@ -204,9 +220,11 @@ def close_port_storage_writers(self) -> None: for _, _, writer_thread in self._port_storage_writers.values(): # This blocking call will wait for all the writer to finish commit writer_thread.join() - for state_writer in self._state_writers.values(): - state_writer.close() - self._state_writers.clear() + for _, state_writer, _ in self._port_state_writers.values(): + state_writer.stop() + for _, _, state_writer_thread in self._port_state_writers.values(): + state_writer_thread.join() + self._port_state_writers.clear() def add_partitioning(self, tag: PhysicalLink, partitioning: Partitioning) -> None: """ diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index a39ac1dcb46..9455b925563 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -124,7 +124,7 @@ class OutputManager( : mutable.HashMap[PortIdentity, OutputPortResultWriterThread] = mutable.HashMap() - private val stateWriters: mutable.HashMap[PortIdentity, BufferedItemWriter[Tuple]] = + private val stateWriterThreads: mutable.HashMap[PortIdentity, OutputPortResultWriterThread] = mutable.HashMap() /** @@ -236,8 +236,8 @@ class OutputManager( }) } - def saveStateToStorageIfNeeded(state: State): Unit = { - stateWriters.values.foreach(_.putOne(state.toTuple)) + private def saveStateToStorageIfNeeded(state: State): Unit = { + stateWriterThreads.values.foreach(_.queue.put(Left(state.toTuple))) } /** @@ -253,7 +253,10 @@ class OutputManager( writerThread.join() case None => } - this.stateWriters.remove(outputPortId).foreach(_.close()) + this.stateWriterThreads.remove(outputPortId).foreach { writerThread => + writerThread.queue.put(Right(PortStorageWriterTerminateSignal)) + writerThread.join() + } } def getPort(portId: PortIdentity): WorkerPort = ports(portId) @@ -304,8 +307,9 @@ class OutputManager( ._1 .writer(VirtualIdentityUtils.getWorkerIndex(actorId).toString) .asInstanceOf[BufferedItemWriter[Tuple]] - stateWriter.open() - this.stateWriters(portId) = stateWriter + val stateWriterThread = new OutputPortResultWriterThread(stateWriter) + this.stateWriterThreads(portId) = stateWriterThread + stateWriterThread.start() } } diff --git a/amber/src/test/python/core/architecture/packaging/test_output_manager.py b/amber/src/test/python/core/architecture/packaging/test_output_manager.py index 8f1daf8052e..dcf7ccde673 100644 --- a/amber/src/test/python/core/architecture/packaging/test_output_manager.py +++ b/amber/src/test/python/core/architecture/packaging/test_output_manager.py @@ -15,15 +15,25 @@ # specific language governing permissions and limitations # under the License. -from unittest.mock import MagicMock, patch +from unittest.mock import MagicMock import pytest from core.architecture.packaging.output_manager import OutputManager from core.models.state import State +from core.storage.runnables.port_storage_writer import PortStorageWriterElement from proto.org.apache.texera.amber.core import PortIdentity +def _stub_state_writer(output_manager, port_id): + """Inject a (queue, writer, thread) triple as if a port were set up.""" + queue = MagicMock() + writer = MagicMock() + thread = MagicMock() + output_manager._port_state_writers[port_id] = (queue, writer, thread) + return queue, writer, thread + + class TestSaveStateToStorageIfNeeded: @pytest.fixture def output_manager(self): @@ -44,84 +54,54 @@ def state(self): def test_no_state_writers_is_a_noop(self, output_manager, state): # With no port set up, save_state_to_storage_if_needed must not # touch any writer. - output_manager.save_state_to_storage_if_needed(state) # no-op, no exception + output_manager.save_state_to_storage_if_needed(state) # no-op def test_unknown_port_id_is_a_noop(self, output_manager, state, port_a): output_manager.save_state_to_storage_if_needed(state, port_id=port_a) # No assertion needed -- the absence of any writer means nothing # was attempted. - def test_writes_to_every_port_when_port_id_omitted( + def test_enqueues_to_every_port_when_port_id_omitted( self, output_manager, state, port_a, port_b ): - writer_a = MagicMock() - writer_b = MagicMock() - output_manager._state_writers[port_a] = writer_a - output_manager._state_writers[port_b] = writer_b + queue_a, _, _ = _stub_state_writer(output_manager, port_a) + queue_b, _, _ = _stub_state_writer(output_manager, port_b) output_manager.save_state_to_storage_if_needed(state) - writer_a.put_one.assert_called_once() - writer_b.put_one.assert_called_once() - # Long-lived writers must NOT be closed per state -- otherwise - # we'd be back to one Iceberg snapshot per state. - writer_a.close.assert_not_called() - writer_b.close.assert_not_called() + # Each port's writer queue receives one PortStorageWriterElement. + # Critically, save is non-blocking -- the call must not invoke + # put_one / close on the buffered writer directly (those happen + # off-thread). + assert queue_a.put.call_count == 1 + assert queue_b.put.call_count == 1 + assert isinstance(queue_a.put.call_args.args[0], PortStorageWriterElement) + assert isinstance(queue_b.put.call_args.args[0], PortStorageWriterElement) - def test_writes_only_to_selected_port_when_port_id_specified( + def test_enqueues_only_to_selected_port_when_port_id_specified( self, output_manager, state, port_a, port_b ): - writer_a = MagicMock() - writer_b = MagicMock() - output_manager._state_writers[port_a] = writer_a - output_manager._state_writers[port_b] = writer_b + queue_a, _, _ = _stub_state_writer(output_manager, port_a) + queue_b, _, _ = _stub_state_writer(output_manager, port_b) output_manager.save_state_to_storage_if_needed(state, port_id=port_a) - writer_a.put_one.assert_called_once() - writer_b.put_one.assert_not_called() - - def test_state_writer_is_opened_at_port_setup(self, output_manager, port_a): - # set_up_port_storage_writer should open the result document AND - # the state document, then cache the state writer for reuse. - result_doc = MagicMock() - state_doc = MagicMock() - state_writer = MagicMock() - state_doc.writer.return_value = state_writer - - with patch( - "core.architecture.packaging.output_manager.DocumentFactory" - ) as mock_factory: - mock_factory.open_document.side_effect = [ - (result_doc, MagicMock()), - (state_doc, MagicMock()), - ] - - output_manager.set_up_port_storage_writer( - port_a, "vfs:///wf/0/exec/0/result/op-a" - ) - - opened = [c.args[0] for c in mock_factory.open_document.call_args_list] - assert opened == [ - "vfs:///wf/0/exec/0/result/op-a", - "vfs:///wf/0/exec/0/state/op-a", - ] - state_writer.open.assert_called_once() - assert output_manager._state_writers[port_a] is state_writer - - def test_close_port_storage_writers_flushes_state_writers( + assert queue_a.put.call_count == 1 + queue_b.put.assert_not_called() + + def test_close_port_storage_writers_stops_state_threads( self, output_manager, port_a, port_b ): - # After the port completes, the long-lived state writer's buffer - # must be flushed and the writer closed (one Iceberg commit per - # port instead of one per state). - writer_a = MagicMock() - writer_b = MagicMock() - output_manager._state_writers[port_a] = writer_a - output_manager._state_writers[port_b] = writer_b + # After the port completes, every state-writer thread must be + # stopped and joined so the buffered writer's close() (which + # flushes the final Iceberg commit) actually runs. + _, writer_a, thread_a = _stub_state_writer(output_manager, port_a) + _, writer_b, thread_b = _stub_state_writer(output_manager, port_b) output_manager.close_port_storage_writers() - writer_a.close.assert_called_once() - writer_b.close.assert_called_once() - assert output_manager._state_writers == {} + writer_a.stop.assert_called_once() + writer_b.stop.assert_called_once() + thread_a.join.assert_called_once() + thread_b.join.assert_called_once() + assert output_manager._port_state_writers == {} From 2722038e78adc5b42d0928f51c61e335b711aa23 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Wed, 6 May 2026 16:49:09 -0700 Subject: [PATCH 14/23] docs: note multi-output-port state fan-out is intentional Address PR #4490 review comment 3192916602: writing the same state row to every output port's state table mirrors the broadcast-to-all-workers behavior on the emit side -- state is shared context, not per-key data, so every downstream operator needs the full set. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../python/core/architecture/packaging/output_manager.py | 6 ++++++ .../engine/architecture/messaginglayer/OutputManager.scala | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 1220a9f15f3..cea76904feb 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -200,6 +200,12 @@ def save_tuple_to_storage_if_needed(self, tuple_: Tuple, port_id=None) -> None: ) def save_state_to_storage_if_needed(self, state: State, port_id=None) -> None: + # When port_id is omitted the same state row is fanned out to + # every output port's state table. This mirrors the + # broadcast-to-all-workers behavior on the emit side: state is + # shared context, not per-key data, so every downstream operator + # (and every worker reading the materialization) needs the full + # set. element = PortStorageWriterElement(data_tuple=state.to_tuple()) if port_id is None: for writer_queue, _, _ in self._port_state_writers.values(): diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index 9455b925563..d76a76be33f 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -237,6 +237,11 @@ class OutputManager( } private def saveStateToStorageIfNeeded(state: State): Unit = { + // The same state row is fanned out to every output port's state + // table. This mirrors the broadcast-to-all-workers behavior on the + // emit side: state is shared context, not per-key data, so every + // downstream operator (and every worker reading the materialization) + // needs the full set. stateWriterThreads.values.foreach(_.queue.put(Left(state.toTuple))) } From fd74c4c862b6a04d38af91baf35ec1abb87f3549 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Wed, 6 May 2026 16:52:49 -0700 Subject: [PATCH 15/23] refactor(pyamber): drop emit_state_with_filter from materialization reader Address PR #4490 review comment 3192901027: the partitioner detour in the Python reader was a no-op (every worker is supposed to see every state, so the broadcast-then-filter round-trip just reduced back to the input). Emit StateFrame(State.from_tuple(row)) directly in run(), matching the Scala reader. Test class TestEmitStateWithFilter is dropped; the run() block test asserts partitioner.flush_state is not called. Co-Authored-By: Claude Opus 4.7 (1M context) --- ...ut_port_materialization_reader_runnable.py | 19 ++-- ...ut_port_materialization_reader_runnable.py | 90 +++---------------- 2 files changed, 18 insertions(+), 91 deletions(-) diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index 67c6370c006..3e8bfd8c9f0 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -125,15 +125,6 @@ def tuple_to_batch_with_filter(self, tuple_: Tuple) -> typing.Iterator[DataFrame if receiver == self.worker_actor_id: yield self.tuples_to_data_frame(tuples) - def emit_state_with_filter(self, state: State) -> typing.Iterator[DataPayload]: - for receiver, payload in self.partitioner.flush_state(state): - if receiver == self.worker_actor_id: - yield ( - StateFrame(payload) - if isinstance(payload, State) - else self.tuples_to_data_frame(payload) - ) - def run(self) -> None: """ Main execution logic that reads tuples from the materialized storage and @@ -153,12 +144,16 @@ def run(self) -> None: # an order: we replay states first because downstream # operators typically need their state set up before they # process the incoming tuples. + # + # Every state is broadcast to every downstream worker -- no + # partitioner filtering here, unlike the tuple loop below. + # State is shared context (e.g. config / counters), not + # per-key data, so each worker needs the full set. state_document, _ = DocumentFactory.open_document( State.uri_from_result_uri(self.uri) ) - for state in state_document.get(): - for state_frame in self.emit_state_with_filter(State.from_tuple(state)): - self.emit_payload(state_frame) + for state_row in state_document.get(): + self.emit_payload(StateFrame(State.from_tuple(state_row))) storage_iterator = self.materialization.get() # Iterate and process tuples. diff --git a/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py b/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py index c90ef91f83c..5016c2df2f1 100644 --- a/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py +++ b/amber/src/test/python/core/storage/runnables/test_input_port_materialization_reader_runnable.py @@ -19,7 +19,7 @@ import pytest -from core.models import State, StateFrame, Tuple +from core.models import State, StateFrame from core.models.internal_queue import DataElement from core.models.schema import Schema from core.storage.runnables.input_port_materialization_reader_runnable import ( @@ -31,79 +31,11 @@ ) -class TestEmitStateWithFilter: - """Cover the partitioner-filter logic for state payloads in - InputPortMaterializationReaderRunnable. These tests bypass __init__ - so we don't need a real partitioner or storage URI. - """ - - @pytest.fixture - def me(self): - return ActorVirtualIdentity(name="me") - - @pytest.fixture - def someone_else(self): - return ActorVirtualIdentity(name="other") - - @pytest.fixture - def runnable(self, me): - # __new__ skips __init__ so we can wire only the fields we need. - instance = InputPortMaterializationReaderRunnable.__new__( - InputPortMaterializationReaderRunnable - ) - instance.worker_actor_id = me - instance.partitioner = MagicMock() - instance.tuple_schema = Schema(raw_schema={"x": "INTEGER"}) - return instance - - def test_yields_state_frame_for_matching_receiver(self, runnable, me): - state = State({"k": 1}) - runnable.partitioner.flush_state.return_value = [(me, state)] - - frames = list(runnable.emit_state_with_filter(state)) - - assert len(frames) == 1 - assert isinstance(frames[0], StateFrame) - assert frames[0].frame is state - - def test_filters_out_non_matching_receivers(self, runnable, me, someone_else): - state = State({"k": 1}) - runnable.partitioner.flush_state.return_value = [ - (someone_else, state), - (me, state), - (someone_else, state), - ] - - frames = list(runnable.emit_state_with_filter(state)) - - assert len(frames) == 1 - assert isinstance(frames[0], StateFrame) - - def test_yields_data_frame_for_non_state_payload(self, runnable, me): - # When the partitioner produces a tuple-batch payload (BroadcastPartitioner - # case), the runnable must convert it to a DataFrame instead of wrapping - # it as a StateFrame. - state = State({"k": 1}) - tuples = [Tuple({"x": 7}, schema=runnable.tuple_schema)] - runnable.partitioner.flush_state.return_value = [(me, tuples)] - - frames = list(runnable.emit_state_with_filter(state)) - - assert len(frames) == 1 - # Should not be wrapped as a StateFrame. - assert not isinstance(frames[0], StateFrame) - assert frames[0].frame.num_rows == 1 - - def test_empty_partitioner_output_yields_nothing(self, runnable): - state = State({}) - runnable.partitioner.flush_state.return_value = [] - - assert list(runnable.emit_state_with_filter(state)) == [] - - class TestRunStateReadingBlock: - """Cover the inner try-block in run() that opens the state document and - emits its rows as StateFrames. + """Cover the state-reading block in run() that opens the state + document and emits its rows as StateFrames directly to the input + queue (no partitioner filtering -- state is broadcast to every + worker). """ @pytest.fixture @@ -127,12 +59,13 @@ def runnable(self, me): instance.partitioner.flush.return_value = [] return instance - def test_state_rows_are_emitted_as_state_frames(self, runnable, me): + def test_state_rows_are_emitted_as_state_frames(self, runnable): state_a = State({"loop_counter": 0}) state_b = State({"loop_counter": 1}) # The state document yields opaque tuples; from_tuple deserializes - # them. Patch from_tuple so we don't have to wire a real serialization. + # them. Patch from_tuple so we don't have to wire a real + # serialization. result_doc = MagicMock() result_doc.get.return_value = iter([]) # No materialized tuples. state_doc = MagicMock() @@ -149,14 +82,13 @@ def test_state_rows_are_emitted_as_state_frames(self, runnable, me): (state_doc, None), ] mock_from_tuple.side_effect = [state_a, state_b] - runnable.partitioner.flush_state.side_effect = [ - [(me, state_a)], - [(me, state_b)], - ] runnable.run() # Two StateFrames must have been put on the queue, in order. + # The state replay must NOT route through the partitioner -- + # state is shared context, broadcast to every worker. + runnable.partitioner.flush_state.assert_not_called() state_frames = [ call.args[0] for call in runnable.queue.put.call_args_list From 1cef2ff0d4a2c3d34112276560586c6457caf337 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Wed, 6 May 2026 16:57:23 -0700 Subject: [PATCH 16/23] fix fmt --- ...put_port_materialization_reader_runnable.py | 18 ++++++++---------- .../InputPortMaterializationReaderThread.scala | 16 ++++++++-------- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index 3e8bfd8c9f0..bdb678aac27 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -132,6 +132,14 @@ def run(self) -> None: emits an EndChannel ECM. Use the same partitioner implementation as that in output manager, where a tuple is batched by the partitioner and only selected as the input of this worker according to the partitioner. + + States and tuples are persisted to separate tables, so the original + interleaving is lost and replay has to pick an order: we replay states + first because downstream operators typically need their state set up + before they process the incoming tuples. Every state is broadcast to + every downstream worker -- no partitioner filtering, unlike the tuple + loop. State is shared context (e.g. config / counters), not per-key + data, so each worker needs the full set. """ try: self.materialization, self.tuple_schema = DocumentFactory.open_document( @@ -139,16 +147,6 @@ def run(self) -> None: ) self.emit_ecm("StartChannel", EmbeddedControlMessageType.NO_ALIGNMENT) - # States and tuples are persisted to separate tables, so - # the original interleaving is lost and replay has to pick - # an order: we replay states first because downstream - # operators typically need their state set up before they - # process the incoming tuples. - # - # Every state is broadcast to every downstream worker -- no - # partitioner filtering here, unlike the tuple loop below. - # State is shared context (e.g. config / counters), not - # per-key data, so each worker needs the full set. state_document, _ = DocumentFactory.open_document( State.uri_from_result_uri(self.uri) ) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index 98c68731aee..e06453b1a3f 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -84,25 +84,25 @@ class InputPortMaterializationReaderThread( /** * Read from the materialization stoage, and mimcs the behavior of an upstream worker's output manager. + * + * States and tuples are persisted to separate tables, so the original + * interleaving is lost and replay has to pick an order: we replay states + * first because downstream operators typically need their state set up + * before they process the incoming tuples. Every state is broadcast to + * every downstream worker -- no partitioner filtering, unlike the tuple + * loop. State is shared context (e.g. config / counters), not per-key + * data, so each worker needs the full set. */ override def run(): Unit = { // Notify the input port of start of input channel emitECM(METHOD_START_CHANNEL, NO_ALIGNMENT) try { - // States and tuples are persisted to separate tables, so the - // original interleaving is lost and replay has to pick an order: - // we replay states first because downstream operators typically - // need their state set up before they process the incoming tuples. val stateDocument = DocumentFactory .openDocument(State.uriFromResultUri(uri)) ._1 .asInstanceOf[VirtualDocument[Tuple]] val stateReadIterator = stateDocument.get() - // Every state is broadcast to every downstream worker -- no - // partitioner filtering here, unlike the tuple loop below. State - // is shared context (e.g. config / counters), not per-key data, - // so each worker needs the full set. while (stateReadIterator.hasNext) { val state = State.fromTuple(stateReadIterator.next()) inputMessageQueue.put( From 4223cf56408e51283280fdc61bdb7008f5f6f2e7 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Wed, 6 May 2026 17:56:10 -0700 Subject: [PATCH 17/23] refactor(storage): make port URIs symmetric over a base URI Drop the result-as-primary asymmetry in VFSURIFactory: ports now build a single base URI, with result and state URIs as equal first-class derivatives via resultURI(base) / stateURI(base). Removes the substring-replace `siblingStateURI` helper and the asymmetric createResultURI / createStateURI pair. --- .../architecture/packaging/output_manager.py | 7 ++-- amber/src/main/python/core/models/state.py | 4 --- ...ut_port_materialization_reader_runnable.py | 5 +-- .../python/core/storage/vfs_uri_factory.py | 16 ++++++++-- .../messaginglayer/OutputManager.scala | 8 ++--- .../CostBasedScheduleGenerator.scala | 6 ++-- .../ExpansionGreedyScheduleGenerator.scala | 4 +-- .../RegionExecutionCoordinator.scala | 15 +++++---- ...InputPortMaterializationReaderThread.scala | 6 ++-- .../storage/iceberg/test_iceberg_document.py | 32 +++++++++++-------- .../texera/amber/core/state/State.scala | 4 --- .../amber/core/storage/VFSURIFactory.scala | 21 ++++++++---- .../core/storage/VFSURIFactorySpec.scala | 23 ++++++++----- .../result/iceberg/IcebergDocumentSpec.scala | 8 +++-- .../iceberg/IcebergTableStatsSpec.scala | 22 +++++++------ 15 files changed, 105 insertions(+), 76 deletions(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index cea76904feb..8521b98ef4a 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -45,6 +45,7 @@ from core.models.payload import DataPayload, DataFrame from core.models.state import State from core.storage.document_factory import DocumentFactory +from core.storage.vfs_uri_factory import VFSURIFactory from core.storage.runnables.port_storage_writer import ( PortStorageWriter, PortStorageWriterElement, @@ -131,7 +132,9 @@ def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): to storage in batch, and open a long-lived buffered writer for state materialization on the same port. """ - document, _ = DocumentFactory.open_document(storage_uri) + document, _ = DocumentFactory.open_document( + VFSURIFactory.result_uri(storage_uri) + ) buffered_item_writer = document.writer(str(get_worker_index(self.worker_id))) writer_queue = Queue() port_storage_writer = PortStorageWriter( @@ -150,7 +153,7 @@ def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): ) state_document, _ = DocumentFactory.open_document( - State.uri_from_result_uri(storage_uri) + VFSURIFactory.state_uri(storage_uri) ) state_buffered_item_writer = state_document.writer( str(get_worker_index(self.worker_id)) diff --git a/amber/src/main/python/core/models/state.py b/amber/src/main/python/core/models/state.py index 3ce610bbee5..003aaa212ac 100644 --- a/amber/src/main/python/core/models/state.py +++ b/amber/src/main/python/core/models/state.py @@ -41,10 +41,6 @@ def from_json(cls, payload: str) -> "State": def from_tuple(cls, row: Tuple) -> "State": return cls.from_json(row[cls.CONTENT]) - @staticmethod - def uri_from_result_uri(result_uri: str) -> str: - return result_uri.replace("/result", "/state") - _TYPE_MARKER = "__texera_type__" _PAYLOAD_MARKER = "payload" diff --git a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py index bdb678aac27..692bff65553 100644 --- a/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py +++ b/amber/src/main/python/core/storage/runnables/input_port_materialization_reader_runnable.py @@ -37,6 +37,7 @@ from core.models import Tuple, InternalQueue, DataFrame, DataPayload, State, StateFrame from core.models.internal_queue import DataElement, ECMElement from core.storage.document_factory import DocumentFactory +from core.storage.vfs_uri_factory import VFSURIFactory from core.util import Stoppable, get_one_of from core.util.runnable.runnable import Runnable from core.util.virtual_identity import get_from_actor_id_for_input_port_storage @@ -143,12 +144,12 @@ def run(self) -> None: """ try: self.materialization, self.tuple_schema = DocumentFactory.open_document( - self.uri + VFSURIFactory.result_uri(self.uri) ) self.emit_ecm("StartChannel", EmbeddedControlMessageType.NO_ALIGNMENT) state_document, _ = DocumentFactory.open_document( - State.uri_from_result_uri(self.uri) + VFSURIFactory.state_uri(self.uri) ) for state_row in state_document.get(): self.emit_payload(StateFrame(State.from_tuple(state_row))) diff --git a/amber/src/main/python/core/storage/vfs_uri_factory.py b/amber/src/main/python/core/storage/vfs_uri_factory.py index 0e23e607055..883450abf2b 100644 --- a/amber/src/main/python/core/storage/vfs_uri_factory.py +++ b/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -89,12 +89,22 @@ def extract_value(key: str) -> str: ) @staticmethod - def create_result_uri(workflow_id, execution_id, global_port_id) -> str: - """Creates a URI pointing to a result storage.""" - base_uri = ( + def create_port_base_uri(workflow_id, execution_id, global_port_id) -> str: + """Base URI for a port. Result and state URIs derive from it via + `result_uri` / `state_uri`. + """ + return ( f"{VFSURIFactory.VFS_FILE_URI_SCHEME}:///wid/{workflow_id.id}" f"/eid/{execution_id.id}/globalportid/" f"{serialize_global_port_identity(global_port_id)}" ) + @staticmethod + def result_uri(base_uri: str) -> str: + """The result-resource URI under a port base URI.""" return f"{base_uri}/{VFSResourceType.RESULT.value}" + + @staticmethod + def state_uri(base_uri: str) -> str: + """The state-resource URI under a port base URI.""" + return f"{base_uri}/{VFSResourceType.STATE.value}" diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index d76a76be33f..3b233d8cf72 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -20,7 +20,7 @@ package org.apache.texera.amber.engine.architecture.messaginglayer import org.apache.texera.amber.core.state.State -import org.apache.texera.amber.core.storage.DocumentFactory +import org.apache.texera.amber.core.storage.{DocumentFactory, VFSURIFactory} import org.apache.texera.amber.core.storage.model.BufferedItemWriter import org.apache.texera.amber.core.tuple._ import org.apache.texera.amber.core.virtualidentity.{ActorVirtualIdentity, ChannelIdentity} @@ -295,9 +295,9 @@ class OutputManager( ports.head._1 } - private def setupOutputStorageWriterThread(portId: PortIdentity, storageUri: URI): Unit = { + private def setupOutputStorageWriterThread(portId: PortIdentity, portBaseURI: URI): Unit = { val bufferedItemWriter = DocumentFactory - .openDocument(storageUri) + .openDocument(VFSURIFactory.resultURI(portBaseURI)) ._1 .writer(VirtualIdentityUtils.getWorkerIndex(actorId).toString) .asInstanceOf[BufferedItemWriter[Tuple]] @@ -308,7 +308,7 @@ class OutputManager( // The state document is provisioned alongside the result document // by RegionExecutionCoordinator, so it is always present. val stateWriter = DocumentFactory - .openDocument(State.uriFromResultUri(storageUri)) + .openDocument(VFSURIFactory.stateURI(portBaseURI)) ._1 .writer(VirtualIdentityUtils.getWorkerIndex(actorId).toString) .asInstanceOf[BufferedItemWriter[Tuple]] diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala index 401ccddc0a4..d2c31753682 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala @@ -20,7 +20,7 @@ package org.apache.texera.amber.engine.architecture.scheduling import org.apache.texera.amber.config.ApplicationConfig -import org.apache.texera.amber.core.storage.VFSURIFactory.createResultURI +import org.apache.texera.amber.core.storage.VFSURIFactory.createPortBaseURI import org.apache.texera.amber.core.virtualidentity.{ActorVirtualIdentity, PhysicalOpIdentity} import org.apache.texera.amber.core.workflow._ import org.apache.texera.amber.engine.architecture.scheduling.SchedulingUtils.replaceVertex @@ -174,12 +174,12 @@ class CostBasedScheduleGenerator( // Allocate an URI for each of these output ports val outputPortConfigs: Map[GlobalPortIdentity, OutputPortConfig] = outputPortIdsNeedingStorage.map { gpid => - val outputWriterURI = createResultURI( + val portBaseURI = createPortBaseURI( workflowId = workflowContext.workflowId, executionId = workflowContext.executionId, globalPortId = gpid ) - gpid -> OutputPortConfig(outputWriterURI) + gpid -> OutputPortConfig(portBaseURI) }.toMap val resourceConfig = ResourceConfig(portConfigs = outputPortConfigs) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/ExpansionGreedyScheduleGenerator.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/ExpansionGreedyScheduleGenerator.scala index 4bb89338967..304e1496f8a 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/ExpansionGreedyScheduleGenerator.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/ExpansionGreedyScheduleGenerator.scala @@ -21,7 +21,7 @@ package org.apache.texera.amber.engine.architecture.scheduling import com.typesafe.scalalogging.LazyLogging import org.apache.texera.amber.core.WorkflowRuntimeException -import org.apache.texera.amber.core.storage.VFSURIFactory.createResultURI +import org.apache.texera.amber.core.storage.VFSURIFactory.createPortBaseURI import org.apache.texera.amber.core.virtualidentity.PhysicalOpIdentity import org.apache.texera.amber.core.workflow.{ GlobalPortIdentity, @@ -331,7 +331,7 @@ class ExpansionGreedyScheduleGenerator( private def getStorageURIFromGlobalOutputPortId(outputPortId: GlobalPortIdentity) = { assert(!outputPortId.input) - createResultURI( + createPortBaseURI( workflowId = workflowContext.workflowId, executionId = workflowContext.executionId, globalPortId = outputPortId diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala index 58fdf9f2428..9262fcee18f 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala @@ -22,7 +22,7 @@ package org.apache.texera.amber.engine.architecture.scheduling import org.apache.pekko.pattern.gracefulStop import com.twitter.util.{Duration => TwitterDuration, Future, JavaTimer, Return, Throw, Timer} import org.apache.texera.amber.core.state.State -import org.apache.texera.amber.core.storage.DocumentFactory +import org.apache.texera.amber.core.storage.{DocumentFactory, VFSURIFactory} import org.apache.texera.amber.core.storage.VFSURIFactory.decodeURI import org.apache.texera.amber.core.virtualidentity.ActorVirtualIdentity import org.apache.texera.amber.core.workflow.{GlobalPortIdentity, PhysicalLink, PhysicalOp} @@ -569,20 +569,21 @@ class RegionExecutionCoordinator( ): Unit = { portConfigs.foreach { case (outputPortId, portConfig) => - val storageUriToAdd = portConfig.storageURI - val stateUriToAdd = State.uriFromResultUri(storageUriToAdd) - val (_, eid, _, _) = decodeURI(storageUriToAdd) + val portBaseURI = portConfig.storageURI + val resultURI = VFSURIFactory.resultURI(portBaseURI) + val stateURI = VFSURIFactory.stateURI(portBaseURI) val schemaOptional = region.getOperator(outputPortId.opId).outputPorts(outputPortId.portId)._3 val schema = schemaOptional.getOrElse(throw new IllegalStateException("Schema is missing")) - DocumentFactory.createDocument(storageUriToAdd, schema) - DocumentFactory.createDocument(stateUriToAdd, State.schema) + DocumentFactory.createDocument(resultURI, schema) + DocumentFactory.createDocument(stateURI, State.schema) if (!isRestart) { + val (_, eid, _, _) = decodeURI(resultURI) WorkflowExecutionsResource.insertOperatorPortResultUri( eid = eid, globalPortId = outputPortId, - uri = storageUriToAdd + uri = resultURI ) } } diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index e06453b1a3f..a7d259c37ed 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -22,7 +22,7 @@ package org.apache.texera.amber.engine.architecture.worker.managers import io.grpc.MethodDescriptor import org.apache.texera.amber.config.ApplicationConfig import org.apache.texera.amber.core.state.State -import org.apache.texera.amber.core.storage.DocumentFactory +import org.apache.texera.amber.core.storage.{DocumentFactory, VFSURIFactory} import org.apache.texera.amber.core.storage.model.VirtualDocument import org.apache.texera.amber.core.tuple.Tuple import org.apache.texera.amber.core.virtualidentity.{ @@ -99,7 +99,7 @@ class InputPortMaterializationReaderThread( try { val stateDocument = DocumentFactory - .openDocument(State.uriFromResultUri(uri)) + .openDocument(VFSURIFactory.stateURI(uri)) ._1 .asInstanceOf[VirtualDocument[Tuple]] val stateReadIterator = stateDocument.get() @@ -111,7 +111,7 @@ class InputPortMaterializationReaderThread( } val materialization: VirtualDocument[Tuple] = DocumentFactory - .openDocument(uri) + .openDocument(VFSURIFactory.resultURI(uri)) ._1 .asInstanceOf[VirtualDocument[Tuple]] val storageReadIterator = materialization.get() diff --git a/amber/src/test/python/core/storage/iceberg/test_iceberg_document.py b/amber/src/test/python/core/storage/iceberg/test_iceberg_document.py index 44e62bbb8a6..381f8e5ff64 100644 --- a/amber/src/test/python/core/storage/iceberg/test_iceberg_document.py +++ b/amber/src/test/python/core/storage/iceberg/test_iceberg_document.py @@ -83,17 +83,21 @@ def iceberg_document(self, amber_schema): with a random operator id """ operator_uuid = str(uuid.uuid4()).replace("-", "") - uri = VFSURIFactory.create_result_uri( - WorkflowIdentity(id=0), - ExecutionIdentity(id=0), - GlobalPortIdentity( - op_id=PhysicalOpIdentity( - logical_op_id=OperatorIdentity(id=f"test_table_{operator_uuid}"), - layer_name="main", + uri = VFSURIFactory.result_uri( + VFSURIFactory.create_port_base_uri( + WorkflowIdentity(id=0), + ExecutionIdentity(id=0), + GlobalPortIdentity( + op_id=PhysicalOpIdentity( + logical_op_id=OperatorIdentity( + id=f"test_table_{operator_uuid}" + ), + layer_name="main", + ), + port_id=PortIdentity(id=0), + input=False, ), - port_id=PortIdentity(id=0), - input=False, - ), + ) ) DocumentFactory.create_document(uri, amber_schema) document, _ = DocumentFactory.open_document(uri) @@ -327,7 +331,7 @@ def test_get_counts(self, iceberg_document, sample_items): def test_state_materialization_round_trip(self): operator_uuid = str(uuid.uuid4()).replace("-", "") - result_uri = VFSURIFactory.create_result_uri( + base_uri = VFSURIFactory.create_port_base_uri( WorkflowIdentity(id=0), ExecutionIdentity(id=0), GlobalPortIdentity( @@ -339,7 +343,7 @@ def test_state_materialization_round_trip(self): input=False, ), ) - state_uri = State.uri_from_result_uri(result_uri) + state_uri = VFSURIFactory.state_uri(base_uri) DocumentFactory.create_document(state_uri, State.SCHEMA) document, _ = DocumentFactory.open_document(state_uri) @@ -363,7 +367,7 @@ def test_state_materialization_round_trip(self): def test_multiple_states_materialize_as_rows_in_one_table(self): operator_uuid = str(uuid.uuid4()).replace("-", "") - result_uri = VFSURIFactory.create_result_uri( + base_uri = VFSURIFactory.create_port_base_uri( WorkflowIdentity(id=0), ExecutionIdentity(id=0), GlobalPortIdentity( @@ -377,7 +381,7 @@ def test_multiple_states_materialize_as_rows_in_one_table(self): input=False, ), ) - state_uri = State.uri_from_result_uri(result_uri) + state_uri = VFSURIFactory.state_uri(base_uri) DocumentFactory.create_document(state_uri, State.SCHEMA) document, _ = DocumentFactory.open_document(state_uri) diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala index 532f355c17e..ba146f1d57c 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/state/State.scala @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.JsonNode import org.apache.texera.amber.core.tuple.{Attribute, AttributeType, Schema, Tuple} import org.apache.texera.amber.util.JSONUtils.objectMapper -import java.net.URI import java.util.Base64 import scala.jdk.CollectionConverters.IteratorHasAsScala @@ -58,9 +57,6 @@ object State { def fromTuple(row: Tuple): State = fromJson(row.getField[String](Content)) - def uriFromResultUri(resultUri: URI): URI = - new URI(resultUri.toString.replace("/result", "/state")) - private def toJsonValue(value: Any): Any = value match { case null => null diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala index e687b28a29b..291c31896b0 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/core/storage/VFSURIFactory.scala @@ -84,18 +84,25 @@ object VFSURIFactory { } /** - * Create a URI pointing to a result storage + * Create the base URI for a port. Result and state URIs are derived + * from this base via `resultURI` / `stateURI`. */ - def createResultURI( + def createPortBaseURI( workflowId: WorkflowIdentity, executionId: ExecutionIdentity, globalPortId: GlobalPortIdentity - ): URI = { - val baseUri = - s"$VFS_FILE_URI_SCHEME:///wid/${workflowId.id}/eid/${executionId.id}/globalportid/${globalPortId.serializeAsString}" + ): URI = + new URI( + s"$VFS_FILE_URI_SCHEME:///wid/${workflowId.id}/eid/${executionId.id}" + + s"/globalportid/${globalPortId.serializeAsString}" + ) - new URI(s"$baseUri/${VFSResourceType.RESULT.toString.toLowerCase}") - } + def resultURI(baseURI: URI): URI = appendResource(baseURI, VFSResourceType.RESULT) + + def stateURI(baseURI: URI): URI = appendResource(baseURI, VFSResourceType.STATE) + + private def appendResource(baseURI: URI, resourceType: VFSResourceType.Value): URI = + new URI(s"$baseURI/${resourceType.toString.toLowerCase}") /** * Create a URI pointing to runtime statistics diff --git a/common/workflow-core/src/test/scala/org/apache/texera/amber/core/storage/VFSURIFactorySpec.scala b/common/workflow-core/src/test/scala/org/apache/texera/amber/core/storage/VFSURIFactorySpec.scala index 6fbe35873a7..0b8ae4a19c0 100644 --- a/common/workflow-core/src/test/scala/org/apache/texera/amber/core/storage/VFSURIFactorySpec.scala +++ b/common/workflow-core/src/test/scala/org/apache/texera/amber/core/storage/VFSURIFactorySpec.scala @@ -42,23 +42,30 @@ class VFSURIFactorySpec extends AnyFlatSpec { input = true ) - "VFSURIFactory.createResultURI" should "include workflow, execution, port, and the result resource type" in { - val uri = VFSURIFactory.createResultURI(workflowId, executionId, portId) - assert(uri.getScheme == VFSURIFactory.VFS_FILE_URI_SCHEME) - val path = uri.getPath + "VFSURIFactory.createPortBaseURI" should "include workflow, execution, and port segments without a resource type" in { + val baseURI = VFSURIFactory.createPortBaseURI(workflowId, executionId, portId) + assert(baseURI.getScheme == VFSURIFactory.VFS_FILE_URI_SCHEME) + val path = baseURI.getPath assert(path.contains("/wid/7")) assert(path.contains("/eid/11")) assert(path.contains("/globalportid/")) - assert(path.endsWith("/result")) + assert(!path.endsWith("/result")) + assert(!path.endsWith("/state")) } - it should "round-trip through decodeURI" in { - val uri = VFSURIFactory.createResultURI(workflowId, executionId, portId) - val (wid, eid, globalPortIdOpt, resourceType) = VFSURIFactory.decodeURI(uri) + "VFSURIFactory.resultURI / stateURI" should "append the resource segment and round-trip through decodeURI" in { + val baseURI = VFSURIFactory.createPortBaseURI(workflowId, executionId, portId) + val resultURI = VFSURIFactory.resultURI(baseURI) + val stateURI = VFSURIFactory.stateURI(baseURI) + assert(resultURI.getPath.endsWith("/result")) + assert(stateURI.getPath.endsWith("/state")) + + val (wid, eid, globalPortIdOpt, resourceType) = VFSURIFactory.decodeURI(resultURI) assert(wid == workflowId) assert(eid == executionId) assert(globalPortIdOpt.contains(portId)) assert(resourceType == VFSResourceType.RESULT) + assert(VFSURIFactory.decodeURI(stateURI)._4 == VFSResourceType.STATE) } "VFSURIFactory.createRuntimeStatisticsURI" should "produce a runtimeStatistics URI without an opid segment" in { diff --git a/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala b/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala index 7f1d8573c2a..b865fff94de 100644 --- a/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala +++ b/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergDocumentSpec.scala @@ -52,6 +52,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with BeforeAndAfter var deserde: (IcebergSchema, Record) => Tuple = _ var catalog: Catalog = _ val tableNamespace = "test_namespace" + var baseURI: URI = _ var uri: URI = _ override def beforeAll(): Unit = { @@ -80,7 +81,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with BeforeAndAfter override def beforeEach(): Unit = { // Generate a unique table name for each test - uri = VFSURIFactory.createResultURI( + baseURI = VFSURIFactory.createPortBaseURI( WorkflowIdentity(0), ExecutionIdentity(0), GlobalPortIdentity( @@ -92,6 +93,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with BeforeAndAfter PortIdentity() ) ) + uri = VFSURIFactory.resultURI(baseURI) DocumentFactory.createDocument(uri, amberSchema) super.beforeEach() } @@ -143,7 +145,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with BeforeAndAfter } it should "round trip materialized state documents" in { - val stateUri = State.uriFromResultUri(uri) + val stateUri = VFSURIFactory.stateURI(baseURI) DocumentFactory.createDocument(stateUri, State.schema) val stateDocument = DocumentFactory.openDocument(stateUri)._1.asInstanceOf[VirtualDocument[Tuple]] @@ -172,7 +174,7 @@ class IcebergDocumentSpec extends VirtualDocumentSpec[Tuple] with BeforeAndAfter } it should "materialize multiple states as rows in one state table" in { - val stateUri = State.uriFromResultUri(uri) + val stateUri = VFSURIFactory.stateURI(baseURI) DocumentFactory.createDocument(stateUri, State.schema) val stateDocument = DocumentFactory.openDocument(stateUri)._1.asInstanceOf[VirtualDocument[Tuple]] diff --git a/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergTableStatsSpec.scala b/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergTableStatsSpec.scala index 175ebc2c01b..b7611f6f772 100644 --- a/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergTableStatsSpec.scala +++ b/common/workflow-core/src/test/scala/org/apache/texera/amber/storage/result/iceberg/IcebergTableStatsSpec.scala @@ -50,16 +50,18 @@ class IcebergTableStatsSpec extends AnyFlatSpec with BeforeAndAfterAll with Suit var deserde: (IcebergSchema, Record) => Tuple = _ var catalog: Catalog = _ val tableNamespace = "test_namespace" - var uri: URI = VFSURIFactory.createResultURI( - WorkflowIdentity(0), - ExecutionIdentity(0), - GlobalPortIdentity( - PhysicalOpIdentity( - logicalOpId = - OperatorIdentity(s"test_table_${UUID.randomUUID().toString.replace("-", "")}"), - layerName = "main" - ), - PortIdentity() + var uri: URI = VFSURIFactory.resultURI( + VFSURIFactory.createPortBaseURI( + WorkflowIdentity(0), + ExecutionIdentity(0), + GlobalPortIdentity( + PhysicalOpIdentity( + logicalOpId = + OperatorIdentity(s"test_table_${UUID.randomUUID().toString.replace("-", "")}"), + layerName = "main" + ), + PortIdentity() + ) ) ) From 990acdf0ee9448da3f2dff7327b60f4250dae524 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Thu, 7 May 2026 16:55:54 -0700 Subject: [PATCH 18/23] fix(amber): rethrow state writer failures on close Mirror the result-writer cleanup added in #4683: if the state-writer thread captured a failure (e.g. iceberg commit retries exhausted), re-throw it from closeOutputStorageWriterIfNeeded so the DP thread surfaces a FatalError to the controller instead of silently announcing port completion. --- .../amber/engine/architecture/messaginglayer/OutputManager.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index 5fe053d4c15..c7c4195a937 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -267,6 +267,7 @@ class OutputManager( this.stateWriterThreads.remove(outputPortId).foreach { writerThread => writerThread.queue.put(Right(PortStorageWriterTerminateSignal)) writerThread.join() + writerThread.getFailure.foreach(throw _) } } From 7da78ed0f7a677a681f99b377a34bcc227315b1e Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Thu, 7 May 2026 17:04:22 -0700 Subject: [PATCH 19/23] =?UTF-8?q?refactor(amber):=20rename=20OutputPortRes?= =?UTF-8?q?ultWriterThread=20=E2=86=92=20OutputPortStorageWriterThread?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The thread is now used for both result and state writers, so the "Result" suffix is misleading. File and class renamed; spec file renamed alongside. --- .claude/launch.json | 11 +++++++++++ .claude/run-preview-worktree.js | 19 +++++++++++++++++++ .claude/settings.local.json | 18 ++++++++++++++++++ .claude/worktrees/dreamy-williamson-9fbfe5 | 1 + .../messaginglayer/OutputManager.scala | 10 +++++----- ...la => OutputPortStorageWriterThread.scala} | 2 +- ...> OutputPortStorageWriterThreadSpec.scala} | 18 +++++++++--------- 7 files changed, 64 insertions(+), 15 deletions(-) create mode 100644 .claude/launch.json create mode 100644 .claude/run-preview-worktree.js create mode 100644 .claude/settings.local.json create mode 160000 .claude/worktrees/dreamy-williamson-9fbfe5 rename amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/{OutputPortResultWriterThread.scala => OutputPortStorageWriterThread.scala} (98%) rename amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/{OutputPortResultWriterThreadSpec.scala => OutputPortStorageWriterThreadSpec.scala} (90%) diff --git a/.claude/launch.json b/.claude/launch.json new file mode 100644 index 00000000000..57fe0e60916 --- /dev/null +++ b/.claude/launch.json @@ -0,0 +1,11 @@ +{ + "version": "0.0.1", + "configurations": [ + { + "name": "frontend-mlc-v10", + "runtimeExecutable": "node", + "runtimeArgs": [".claude/run-preview-worktree.js"], + "port": 4321 + } + ] +} diff --git a/.claude/run-preview-worktree.js b/.claude/run-preview-worktree.js new file mode 100644 index 00000000000..fe30360f73f --- /dev/null +++ b/.claude/run-preview-worktree.js @@ -0,0 +1,19 @@ +#!/usr/bin/env node +// Spawn ng serve from the chore/monaco-lsp-v10 worktree. +// preview_start scopes cwd to the project root, so this trampoline lives in +// .claude/ here and execs the dev server in the sibling worktree. +const { spawn } = require("node:child_process"); +const path = require("node:path"); + +const cwd = path.resolve(__dirname, "..", "..", "texera-worktrees", "chore-monaco-lsp-v10", "frontend"); +const args = ["vite", "--port", "4321"]; + +const child = spawn("yarn", args, { + cwd, + stdio: "inherit", + shell: process.platform === "win32", +}); + +child.on("exit", code => process.exit(code ?? 0)); +process.on("SIGINT", () => child.kill("SIGINT")); +process.on("SIGTERM", () => child.kill("SIGTERM")); diff --git a/.claude/settings.local.json b/.claude/settings.local.json new file mode 100644 index 00000000000..f15bb635ced --- /dev/null +++ b/.claude/settings.local.json @@ -0,0 +1,18 @@ +{ + "permissions": { + "allow": [ + "PowerShell(Get-CimInstance Win32_Process | Where-Object { $_.CommandLine -match \"youthful-ride|zealous-knuth\" -or $_.ExecutablePath -match \"youthful-ride|zealous-knuth\" } | Select-Object ProcessId, Name, CommandLine | Format-List)", + "PowerShell(Get-Process node,claude -ErrorAction SilentlyContinue)", + "mcp__Claude_Preview__preview_start", + "Read(//c/Users/linxi/IdeaProjects/texera-worktrees/chore-monaco-lsp-v10/frontend/**)", + "Bash(yarn ng *)", + "Bash(gh api *)", + "Bash(rm -f .claude/tmp_issue_body.md)", + "Bash(curl -sf http://localhost:4321 -o /dev/null)", + "Bash(curl -sf http://localhost:4321/main.js -o /dev/null)", + "Bash(curl -sf http://localhost:4321/ -o /dev/null)", + "Bash(curl -sfo /dev/null http://localhost:4321/main.js)", + "Bash(curl -sfo /dev/null http://localhost:4321/)" + ] + } +} diff --git a/.claude/worktrees/dreamy-williamson-9fbfe5 b/.claude/worktrees/dreamy-williamson-9fbfe5 new file mode 160000 index 00000000000..9fc5b39d96c --- /dev/null +++ b/.claude/worktrees/dreamy-williamson-9fbfe5 @@ -0,0 +1 @@ +Subproject commit 9fc5b39d96ced08576732d78eab67c0d4645f4d0 diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index c7c4195a937..de922131052 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -33,7 +33,7 @@ import org.apache.texera.amber.engine.architecture.messaginglayer.OutputManager. import org.apache.texera.amber.engine.architecture.sendsemantics.partitioners._ import org.apache.texera.amber.engine.architecture.sendsemantics.partitionings._ import org.apache.texera.amber.engine.architecture.worker.managers.{ - OutputPortResultWriterThread, + OutputPortStorageWriterThread, PortStorageWriterTerminateSignal } import org.apache.texera.amber.engine.common.AmberLogging @@ -121,10 +121,10 @@ class OutputManager( mutable.HashMap[(PhysicalLink, ActorVirtualIdentity), NetworkOutputBuffer]() private val outputPortResultWriterThreads - : mutable.HashMap[PortIdentity, OutputPortResultWriterThread] = + : mutable.HashMap[PortIdentity, OutputPortStorageWriterThread] = mutable.HashMap() - private val stateWriterThreads: mutable.HashMap[PortIdentity, OutputPortResultWriterThread] = + private val stateWriterThreads: mutable.HashMap[PortIdentity, OutputPortStorageWriterThread] = mutable.HashMap() /** @@ -308,7 +308,7 @@ class OutputManager( ._1 .writer(VirtualIdentityUtils.getWorkerIndex(actorId).toString) .asInstanceOf[BufferedItemWriter[Tuple]] - val writerThread = new OutputPortResultWriterThread(bufferedItemWriter) + val writerThread = new OutputPortStorageWriterThread(bufferedItemWriter) this.outputPortResultWriterThreads(portId) = writerThread writerThread.start() @@ -319,7 +319,7 @@ class OutputManager( ._1 .writer(VirtualIdentityUtils.getWorkerIndex(actorId).toString) .asInstanceOf[BufferedItemWriter[Tuple]] - val stateWriterThread = new OutputPortResultWriterThread(stateWriter) + val stateWriterThread = new OutputPortStorageWriterThread(stateWriter) this.stateWriterThreads(portId) = stateWriterThread stateWriterThread.start() } diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortResultWriterThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortStorageWriterThread.scala similarity index 98% rename from amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortResultWriterThread.scala rename to amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortStorageWriterThread.scala index 4223d920da5..bcabde00894 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortResultWriterThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortStorageWriterThread.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal sealed trait TerminateSignal case object PortStorageWriterTerminateSignal extends TerminateSignal -class OutputPortResultWriterThread( +class OutputPortStorageWriterThread( bufferedItemWriter: BufferedItemWriter[Tuple] ) extends Thread { diff --git a/amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortResultWriterThreadSpec.scala b/amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortStorageWriterThreadSpec.scala similarity index 90% rename from amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortResultWriterThreadSpec.scala rename to amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortStorageWriterThreadSpec.scala index 31d8c41611d..d7ab0c18314 100644 --- a/amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortResultWriterThreadSpec.scala +++ b/amber/src/test/scala/org/apache/texera/amber/engine/architecture/worker/managers/OutputPortStorageWriterThreadSpec.scala @@ -32,7 +32,7 @@ import org.scalatest.flatspec.AnyFlatSpec import scala.collection.mutable -class OutputPortResultWriterThreadSpec extends AnyFlatSpec { +class OutputPortStorageWriterThreadSpec extends AnyFlatSpec { private class StubWriter( onPutOne: () => Unit = () => (), @@ -51,9 +51,9 @@ class OutputPortResultWriterThreadSpec extends AnyFlatSpec { private def throwing(msg: String): () => Unit = () => throw new RuntimeException(msg) - "OutputPortResultWriterThread" should "leave getFailure empty on a clean run" in { + "OutputPortStorageWriterThread" should "leave getFailure empty on a clean run" in { val writer = new StubWriter() - val thread = new OutputPortResultWriterThread(writer) + val thread = new OutputPortStorageWriterThread(writer) thread.start() thread.queue.put(Right(PortStorageWriterTerminateSignal)) thread.join() @@ -63,7 +63,7 @@ class OutputPortResultWriterThreadSpec extends AnyFlatSpec { it should "capture a close() exception in getFailure so the worker can re-throw" in { val writer = new StubWriter(onClose = throwing("test close failure")) - val thread = new OutputPortResultWriterThread(writer) + val thread = new OutputPortStorageWriterThread(writer) thread.start() thread.queue.put(Right(PortStorageWriterTerminateSignal)) thread.join() @@ -73,7 +73,7 @@ class OutputPortResultWriterThreadSpec extends AnyFlatSpec { it should "capture a putOne exception and still call close()" in { val writer = new StubWriter(onPutOne = throwing("test putOne failure")) - val thread = new OutputPortResultWriterThread(writer) + val thread = new OutputPortStorageWriterThread(writer) thread.start() thread.queue.put(Left(null.asInstanceOf[Tuple])) thread.queue.put(Right(PortStorageWriterTerminateSignal)) @@ -89,7 +89,7 @@ class OutputPortResultWriterThreadSpec extends AnyFlatSpec { onPutOne = throwing("test putOne failure"), onClose = throwing("test close failure") ) - val thread = new OutputPortResultWriterThread(writer) + val thread = new OutputPortStorageWriterThread(writer) thread.start() thread.queue.put(Left(null.asInstanceOf[Tuple])) thread.queue.put(Right(PortStorageWriterTerminateSignal)) @@ -110,14 +110,14 @@ class OutputPortResultWriterThreadSpec extends AnyFlatSpec { private def installWriterThread( manager: OutputManager, portId: PortIdentity, - thread: OutputPortResultWriterThread + thread: OutputPortStorageWriterThread ): Unit = { val field = classOf[OutputManager] .getDeclaredField("outputPortResultWriterThreads") field.setAccessible(true) field .get(manager) - .asInstanceOf[mutable.HashMap[PortIdentity, OutputPortResultWriterThread]] + .asInstanceOf[mutable.HashMap[PortIdentity, OutputPortStorageWriterThread]] .put(portId, thread) } @@ -130,7 +130,7 @@ class OutputPortResultWriterThreadSpec extends AnyFlatSpec { ) val portId = PortIdentity() val failingWriter = new StubWriter(onClose = throwing("test close failure")) - val failingThread = new OutputPortResultWriterThread(failingWriter) + val failingThread = new OutputPortStorageWriterThread(failingWriter) failingThread.start() installWriterThread(outputManager, portId, failingThread) val ex = intercept[RuntimeException] { From c1b6ae0543042578dfd9d227fa67b11f51dc4244 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Thu, 7 May 2026 17:04:44 -0700 Subject: [PATCH 20/23] chore: remove accidentally-committed .claude/ artifacts --- .claude/launch.json | 11 ----------- .claude/run-preview-worktree.js | 19 ------------------- .claude/settings.local.json | 18 ------------------ .claude/worktrees/dreamy-williamson-9fbfe5 | 1 - 4 files changed, 49 deletions(-) delete mode 100644 .claude/launch.json delete mode 100644 .claude/run-preview-worktree.js delete mode 100644 .claude/settings.local.json delete mode 160000 .claude/worktrees/dreamy-williamson-9fbfe5 diff --git a/.claude/launch.json b/.claude/launch.json deleted file mode 100644 index 57fe0e60916..00000000000 --- a/.claude/launch.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "version": "0.0.1", - "configurations": [ - { - "name": "frontend-mlc-v10", - "runtimeExecutable": "node", - "runtimeArgs": [".claude/run-preview-worktree.js"], - "port": 4321 - } - ] -} diff --git a/.claude/run-preview-worktree.js b/.claude/run-preview-worktree.js deleted file mode 100644 index fe30360f73f..00000000000 --- a/.claude/run-preview-worktree.js +++ /dev/null @@ -1,19 +0,0 @@ -#!/usr/bin/env node -// Spawn ng serve from the chore/monaco-lsp-v10 worktree. -// preview_start scopes cwd to the project root, so this trampoline lives in -// .claude/ here and execs the dev server in the sibling worktree. -const { spawn } = require("node:child_process"); -const path = require("node:path"); - -const cwd = path.resolve(__dirname, "..", "..", "texera-worktrees", "chore-monaco-lsp-v10", "frontend"); -const args = ["vite", "--port", "4321"]; - -const child = spawn("yarn", args, { - cwd, - stdio: "inherit", - shell: process.platform === "win32", -}); - -child.on("exit", code => process.exit(code ?? 0)); -process.on("SIGINT", () => child.kill("SIGINT")); -process.on("SIGTERM", () => child.kill("SIGTERM")); diff --git a/.claude/settings.local.json b/.claude/settings.local.json deleted file mode 100644 index f15bb635ced..00000000000 --- a/.claude/settings.local.json +++ /dev/null @@ -1,18 +0,0 @@ -{ - "permissions": { - "allow": [ - "PowerShell(Get-CimInstance Win32_Process | Where-Object { $_.CommandLine -match \"youthful-ride|zealous-knuth\" -or $_.ExecutablePath -match \"youthful-ride|zealous-knuth\" } | Select-Object ProcessId, Name, CommandLine | Format-List)", - "PowerShell(Get-Process node,claude -ErrorAction SilentlyContinue)", - "mcp__Claude_Preview__preview_start", - "Read(//c/Users/linxi/IdeaProjects/texera-worktrees/chore-monaco-lsp-v10/frontend/**)", - "Bash(yarn ng *)", - "Bash(gh api *)", - "Bash(rm -f .claude/tmp_issue_body.md)", - "Bash(curl -sf http://localhost:4321 -o /dev/null)", - "Bash(curl -sf http://localhost:4321/main.js -o /dev/null)", - "Bash(curl -sf http://localhost:4321/ -o /dev/null)", - "Bash(curl -sfo /dev/null http://localhost:4321/main.js)", - "Bash(curl -sfo /dev/null http://localhost:4321/)" - ] - } -} diff --git a/.claude/worktrees/dreamy-williamson-9fbfe5 b/.claude/worktrees/dreamy-williamson-9fbfe5 deleted file mode 160000 index 9fc5b39d96c..00000000000 --- a/.claude/worktrees/dreamy-williamson-9fbfe5 +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 9fc5b39d96ced08576732d78eab67c0d4645f4d0 From 7fd8e3e370cec22f53fcf659194ce5abe61ba7a0 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Thu, 7 May 2026 19:00:43 -0700 Subject: [PATCH 21/23] refactor: rename OutputPortConfig.storageURI to storageURIBase MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The field carries a port base URI, not a final URI you can pass to DocumentFactory.openDocument — the result and state URIs are derived from it via VFSURIFactory.resultURI / stateURI. Rename signals that intent at the field name. OutputManager.addPort's parameter and the Python add_output_port / set_up_port_storage_writer parameters are renamed to match. --- .../handlers/control/assign_port_handler.py | 6 +++--- .../core/architecture/packaging/output_manager.py | 15 ++++++++------- .../messaginglayer/OutputManager.scala | 10 +++++----- .../scheduling/CostBasedScheduleGenerator.scala | 2 +- .../scheduling/RegionExecutionCoordinator.scala | 4 ++-- .../scheduling/config/PortConfig.scala | 10 +++++++--- .../promisehandlers/AssignPortHandler.scala | 4 ++-- 7 files changed, 28 insertions(+), 23 deletions(-) diff --git a/amber/src/main/python/core/architecture/handlers/control/assign_port_handler.py b/amber/src/main/python/core/architecture/handlers/control/assign_port_handler.py index 73ebad26b31..71f1e7f96ba 100644 --- a/amber/src/main/python/core/architecture/handlers/control/assign_port_handler.py +++ b/amber/src/main/python/core/architecture/handlers/control/assign_port_handler.py @@ -44,10 +44,10 @@ async def assign_port(self, req: AssignPortRequest) -> EmptyReturn: channel_id=channel_id, port_id=req.port_id ) else: - storage_uri = None + storage_uri_base = None if len(req.storage_uris) > 0 and req.storage_uris[0]: - storage_uri = req.storage_uris[0] + storage_uri_base = req.storage_uris[0] self.context.output_manager.add_output_port( - req.port_id, Schema(raw_schema=req.schema), storage_uri + req.port_id, Schema(raw_schema=req.schema), storage_uri_base ) return EmptyReturn() diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 8521b98ef4a..b85e3e39bf1 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -112,28 +112,29 @@ def add_output_port( self, port_id: PortIdentity, schema: Schema, - storage_uri: typing.Optional[str] = None, + storage_uri_base: typing.Optional[str] = None, ) -> None: if port_id.id is None: port_id.id = 0 if port_id.internal is None: port_id.internal = False - if storage_uri is not None: - self.set_up_port_storage_writer(port_id, storage_uri) + if storage_uri_base is not None: + self.set_up_port_storage_writer(port_id, storage_uri_base) # each port can only be added and initialized once. if port_id not in self._ports: self._ports[port_id] = WorkerPort(schema) - def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): + def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri_base: str): """ Create a separate thread for saving output tuples of a port to storage in batch, and open a long-lived buffered writer for - state materialization on the same port. + state materialization on the same port. `storage_uri_base` is the + port's base URI; the result and state URIs are derived from it. """ document, _ = DocumentFactory.open_document( - VFSURIFactory.result_uri(storage_uri) + VFSURIFactory.result_uri(storage_uri_base) ) buffered_item_writer = document.writer(str(get_worker_index(self.worker_id))) writer_queue = Queue() @@ -153,7 +154,7 @@ def set_up_port_storage_writer(self, port_id: PortIdentity, storage_uri: str): ) state_document, _ = DocumentFactory.open_document( - VFSURIFactory.state_uri(storage_uri) + VFSURIFactory.state_uri(storage_uri_base) ) state_buffered_item_writer = state_document.writer( str(get_worker_index(self.worker_id)) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala index de922131052..2862714ffb7 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/messaginglayer/OutputManager.scala @@ -197,17 +197,17 @@ class OutputManager( saveStateToStorageIfNeeded(state) } - def addPort(portId: PortIdentity, schema: Schema, storageURIOption: Option[URI]): Unit = { + def addPort(portId: PortIdentity, schema: Schema, storageURIBaseOption: Option[URI]): Unit = { // each port can only be added and initialized once. if (this.ports.contains(portId)) { return } this.ports(portId) = WorkerPort(schema) - // if a storage URI is provided, set up a storage writer thread - storageURIOption match { - case Some(storageUri) => setupOutputStorageWriterThread(portId, storageUri) - case None => // No need to add a writer + // if a storage URI base is provided, set up storage writer threads + storageURIBaseOption match { + case Some(portBaseURI) => setupOutputStorageWriterThread(portId, portBaseURI) + case None => // No need to add a writer } } diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala index d2c31753682..43e8d281ce3 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/CostBasedScheduleGenerator.scala @@ -237,7 +237,7 @@ class CostBasedScheduleGenerator( s"the outout port $globalOutputPortId has not been assigned a URI yet." ) ) - .storageURI + .storageURIBase // Group all available URIs of this input port together acc.updated( diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala index 9262fcee18f..f72487f268b 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/RegionExecutionCoordinator.scala @@ -466,7 +466,7 @@ class RegionExecutionCoordinator( opId = physicalOp.id, portId = outputPortId ) => - cfg.storageURI.toString + cfg.storageURIBase.toString } .getOrElse("") Some( @@ -569,7 +569,7 @@ class RegionExecutionCoordinator( ): Unit = { portConfigs.foreach { case (outputPortId, portConfig) => - val portBaseURI = portConfig.storageURI + val portBaseURI = portConfig.storageURIBase val resultURI = VFSURIFactory.resultURI(portBaseURI) val stateURI = VFSURIFactory.stateURI(portBaseURI) val schemaOptional = diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/config/PortConfig.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/config/PortConfig.scala index b4a1e058b44..56743ae0956 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/config/PortConfig.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/scheduling/config/PortConfig.scala @@ -31,9 +31,13 @@ sealed trait PortConfig { def storageURIs: List[URI] } -/** An output port requires exactly one materialization URI. */ -final case class OutputPortConfig(storageURI: URI) extends PortConfig { - override val storageURIs: List[URI] = List(storageURI) +/** + * An output port requires exactly one materialization base URI. Result and + * state URIs hang off it via `VFSURIFactory.resultURI` / `stateURI`; this + * field is *not* a URI you can pass straight to `DocumentFactory`. + */ +final case class OutputPortConfig(storageURIBase: URI) extends PortConfig { + override val storageURIs: List[URI] = List(storageURIBase) } /** diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala index fe959733abb..9e2c7f7c2b6 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/promisehandlers/AssignPortHandler.scala @@ -58,11 +58,11 @@ trait AssignPortHandler { dp.stateManager.assertState(READY, RUNNING, PAUSED) } } else { - val storageURIOption: Option[URI] = msg.storageUris.head match { + val storageURIBaseOption: Option[URI] = msg.storageUris.head match { case "" => None case uriString => Some(URI.create(uriString)) } - dp.outputManager.addPort(msg.portId, schema, storageURIOption) + dp.outputManager.addPort(msg.portId, schema, storageURIBaseOption) } EmptyReturn() } From 8d8abfd2fc5e16d8f443eb1fcd897a2fe29e9d08 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Thu, 7 May 2026 23:01:22 -0700 Subject: [PATCH 22/23] Potential fix for pull request finding Co-authored-by: Copilot Autofix powered by AI <175728472+Copilot@users.noreply.github.com> Signed-off-by: Xinyuan Lin --- .../worker/managers/InputPortMaterializationReaderThread.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala index a7d259c37ed..428d9fb48cb 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/worker/managers/InputPortMaterializationReaderThread.scala @@ -83,7 +83,7 @@ class InputPortMaterializationReaderThread( def finished: Boolean = isFinished.get() /** - * Read from the materialization stoage, and mimcs the behavior of an upstream worker's output manager. + * Read from the materialization storage, and mimics the behavior of an upstream worker's output manager. * * States and tuples are persisted to separate tables, so the original * interleaving is lost and replay has to pick an order: we replay states From abef64f02258b2a873d4c7b25834b96b4a6c1402 Mon Sep 17 00:00:00 2001 From: Xinyuan Lin Date: Sat, 9 May 2026 01:58:52 -0700 Subject: [PATCH 23/23] add test --- .../test_state_materialization_e2e.py | 252 ++++++++++++++++++ .../python/core/runnables/test_main_loop.py | 81 ++++++ 2 files changed, 333 insertions(+) create mode 100644 amber/src/test/python/core/architecture/packaging/test_state_materialization_e2e.py diff --git a/amber/src/test/python/core/architecture/packaging/test_state_materialization_e2e.py b/amber/src/test/python/core/architecture/packaging/test_state_materialization_e2e.py new file mode 100644 index 00000000000..62e7e9832eb --- /dev/null +++ b/amber/src/test/python/core/architecture/packaging/test_state_materialization_e2e.py @@ -0,0 +1,252 @@ +# 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. + +""" +End-to-end integration test for cross-region state materialization. + +The unit tests for `OutputManager.save_state_to_storage_if_needed` and +`InputPortMaterializationReaderRunnable.run` mock out the iceberg layer, +so a regression in the writer/storage/reader join is invisible to them. +This test wires: + + OutputManager.set_up_port_storage_writer(port, base_uri) + → real PortStorageWriter thread + → real IcebergTableWriter (sqlite-backed SqlCatalog) + → state document at VFSURIFactory.state_uri(base_uri) + → InputPortMaterializationReaderRunnable.run() + → DataElement(StateFrame) on the consumer's input queue + +and asserts that a state put through `save_state_to_storage_if_needed` +on the producer side actually arrives at the consumer's queue, with the +same payload. +""" + +import tempfile +import threading +import uuid + +import pytest +from pyiceberg.catalog.sql import SqlCatalog + +from core.architecture.packaging.output_manager import OutputManager +from core.models import State, StateFrame +from core.models.internal_queue import DataElement, InternalQueue +from core.storage.document_factory import DocumentFactory +from core.storage.iceberg.iceberg_catalog_instance import IcebergCatalogInstance +from core.storage.runnables.input_port_materialization_reader_runnable import ( + InputPortMaterializationReaderRunnable, +) +from core.storage.storage_config import StorageConfig +from core.storage.vfs_uri_factory import VFSURIFactory +from proto.org.apache.texera.amber.core import ( + ActorVirtualIdentity, + ChannelIdentity, + ExecutionIdentity, + GlobalPortIdentity, + OperatorIdentity, + PhysicalOpIdentity, + PortIdentity, + WorkflowIdentity, +) +from proto.org.apache.texera.amber.engine.architecture.sendsemantics import ( + OneToOnePartitioning, + Partitioning, +) + + +# Module-level: configure storage once with a tempdir warehouse and a +# sqlite catalog that the test will inject below. This avoids requiring +# the dev postgres/minio stack that the existing iceberg tests depend on. +_WAREHOUSE_DIR = tempfile.mkdtemp(prefix="texera-state-e2e-warehouse-") +_RESULT_NAMESPACE = "operator-port-result-e2e" +_STATE_NAMESPACE = "operator-port-state-e2e" +StorageConfig.initialize( + catalog_type="postgres", # value is ignored once we replace_instance below + postgres_uri_without_scheme="unused", + postgres_username="unused", + postgres_password="unused", + rest_catalog_uri="unused", + rest_catalog_warehouse_name="unused", + table_result_namespace=_RESULT_NAMESPACE, + table_state_namespace=_STATE_NAMESPACE, + directory_path=_WAREHOUSE_DIR, + commit_batch_size=4096, + s3_endpoint="unused", + s3_region="unused", + s3_auth_username="unused", + s3_auth_password="unused", +) + + +@pytest.fixture(scope="module", autouse=True) +def sqlite_iceberg_catalog(): + """Inject a sqlite-backed SqlCatalog so the test runs without external + iceberg infra. Module-scoped so all tests in this file share one + warehouse, and so namespace creation only happens once. + """ + db_path = f"{_WAREHOUSE_DIR}/catalog.sqlite" + catalog = SqlCatalog( + "texera_iceberg_e2e", + **{ + "uri": f"sqlite:///{db_path}", + "warehouse": f"file://{_WAREHOUSE_DIR}", + }, + ) + catalog.create_namespace_if_not_exists(_RESULT_NAMESPACE) + catalog.create_namespace_if_not_exists(_STATE_NAMESPACE) + IcebergCatalogInstance.replace_instance(catalog) + yield catalog + + +def _fresh_base_uri() -> str: + """A unique port-base URI per test so tables don't collide.""" + return VFSURIFactory.create_port_base_uri( + WorkflowIdentity(id=0), + ExecutionIdentity(id=0), + GlobalPortIdentity( + op_id=PhysicalOpIdentity( + logical_op_id=OperatorIdentity( + id=f"e2e-{uuid.uuid4().hex}" + ), + layer_name="main", + ), + port_id=PortIdentity(id=0, internal=False), + input=False, + ), + ) + + +def test_state_written_by_output_manager_is_replayed_by_reader(): + """Producer side writes a state via OutputManager; consumer side reads + it via InputPortMaterializationReaderRunnable. The state must arrive + on the consumer's input queue intact. + """ + base_uri = _fresh_base_uri() + port_id = PortIdentity(id=0, internal=False) + worker_schema_for_result = State.SCHEMA # producer-side: only state matters + + # 1. RegionExecutionCoordinator's responsibility: provision result + + # state documents at the port base URI before any worker starts. + # We emulate that here. + DocumentFactory.create_document( + VFSURIFactory.result_uri(base_uri), worker_schema_for_result + ) + DocumentFactory.create_document( + VFSURIFactory.state_uri(base_uri), State.SCHEMA + ) + + # 2. Producer side: spin up an OutputManager, set up real state + + # result writer threads against the iceberg storage. + producer = OutputManager(worker_id="Worker:WF0-test-producer-main-0") + producer.add_output_port( + port_id, schema=worker_schema_for_result, storage_uri_base=base_uri + ) + + # 3. Drive a state through the producer-side path. + state = State({"flag": True, "loop_counter": 7, "name": "outer"}) + producer.save_state_to_storage_if_needed(state) + + # 4. Force the writer threads to flush + commit by closing them. + # Without this, the iceberg buffer holds the state in memory and + # nothing is durable yet. + producer.close_port_storage_writers() + + # 5. Consumer side: spin up the materialization reader against the + # same base URI. Each reader needs a partitioning even when no real + # downstream worker exists -- supply a OneToOnePartitioning whose + # only receiver is the consumer worker itself. + consumer_worker = ActorVirtualIdentity(name="consumer-worker-0") + consumer_queue = InternalQueue() + partitioning = Partitioning( + one_to_one_partitioning=OneToOnePartitioning( + batch_size=400, + channels=[ + ChannelIdentity( + from_worker_id=ActorVirtualIdentity(name="producer-worker-0"), + to_worker_id=consumer_worker, + is_control=False, + ) + ], + ) + ) + reader = InputPortMaterializationReaderRunnable( + uri=base_uri, + queue=consumer_queue, + worker_actor_id=consumer_worker, + partitioning=partitioning, + ) + + # Run the reader on a worker thread so we can time out cleanly if + # something goes wrong. + reader_thread = threading.Thread(target=reader.run, daemon=True) + reader_thread.start() + reader_thread.join(timeout=30) + assert not reader_thread.is_alive(), "reader did not finish within timeout" + assert reader.finished(), "reader exited but did not mark itself finished" + + # 6. Drain the consumer's queue and find the StateFrame(s). + state_frames: list[State] = [] + while not consumer_queue.is_empty(): + elem = consumer_queue.get() + if isinstance(elem, DataElement) and isinstance(elem.payload, StateFrame): + state_frames.append(elem.payload.frame) + + assert len(state_frames) == 1, ( + f"expected exactly one State to flow through writer→iceberg→reader; " + f"got {len(state_frames)}: {state_frames}" + ) + assert state_frames[0] == state, ( + f"replayed state did not match what was written; " + f"wrote={state}, read={state_frames[0]}" + ) + + +def test_state_table_persists_across_writer_close(): + """Independently verify the iceberg state table contains the row. + If this passes but the reader test above fails, the bug is in the + reader / consumer wiring; if this fails, the bug is in the writer / + storage layer. + """ + base_uri = _fresh_base_uri() + port_id = PortIdentity(id=0, internal=False) + + DocumentFactory.create_document( + VFSURIFactory.result_uri(base_uri), State.SCHEMA + ) + DocumentFactory.create_document( + VFSURIFactory.state_uri(base_uri), State.SCHEMA + ) + + producer = OutputManager(worker_id="Worker:WF0-test-producer2-main-0") + producer.add_output_port( + port_id, schema=State.SCHEMA, storage_uri_base=base_uri + ) + + state = State({"flag": False, "checkpoint": 42}) + producer.save_state_to_storage_if_needed(state) + producer.close_port_storage_writers() + + # Read directly from the iceberg state document, bypassing the reader. + state_document, _ = DocumentFactory.open_document( + VFSURIFactory.state_uri(base_uri) + ) + rows = list(state_document.get()) + assert len(rows) == 1, ( + f"expected exactly one row in the iceberg state table after the " + f"writer was closed; got {len(rows)} rows" + ) + assert State.from_tuple(rows[0]) == state diff --git a/amber/src/test/python/core/runnables/test_main_loop.py b/amber/src/test/python/core/runnables/test_main_loop.py index 78b8ab76aa2..c32f45b8886 100644 --- a/amber/src/test/python/core/runnables/test_main_loop.py +++ b/amber/src/test/python/core/runnables/test_main_loop.py @@ -1441,6 +1441,87 @@ def fake_switch_context(): assert [s["value"] for s in saved_states] == [2, 42] assert all(s["port"] == 0 for s in saved_states) + @pytest.mark.timeout(2) + def test_process_start_channel_persists_produce_state_on_start_output( + self, + main_loop, + mock_data_output_channel, + monkeypatch, + ): + # The state emitted by an executor's `produce_state_on_start` must + # also be persisted via `save_state_to_storage_if_needed`, so a + # downstream worker in a different region can replay it from the + # iceberg state table. + # + # This is the integration path exercised in real workflows when + # users override `produce_state_on_start`. `_process_start_channel` + # → `process_input_state` → DataProcessor.process_internal_marker + # (StartChannel) → executor.produce_state_on_start → _set_output_state + # → MainLoop reads output state → emit + save. + on_start_state = State({"flag": True, "loop_counter": 0}) + + class DummyExecutor: + @staticmethod + def produce_state_on_start(port: int) -> State: + # Tag with port so we can also assert the right port id + # was forwarded. + return State({**on_start_state, "port": port}) + + saved_states: list[State] = [] + main_loop.context.executor_manager.executor = DummyExecutor() + monkeypatch.setattr(main_loop, "_check_and_process_control", lambda: None) + monkeypatch.setattr( + main_loop.context.output_manager, + "emit_state", + lambda state: [(mock_data_output_channel.to_worker_id, StateFrame(state))], + ) + monkeypatch.setattr( + main_loop.context.output_manager, + "save_state_to_storage_if_needed", + lambda state: saved_states.append(state), + ) + # _send_ecm_to_data_channels touches output_manager state we don't + # set up here; for this test the ECM forwarding is irrelevant -- the + # SAVE path is what we're pinning. Stub it. + monkeypatch.setattr(main_loop, "_send_ecm_to_data_channels", lambda *_: None) + + # Simulate the DP-thread side: when MainLoop yields, the DataProcessor + # consumes the StartChannel marker and runs produce_state_on_start. + def fake_switch_context(): + from core.models.internal_marker import StartChannel as _StartChannel + + tpm = main_loop.context.tuple_processing_manager + if isinstance(tpm.current_internal_marker, _StartChannel): + # mimic DataProcessor.process_internal_marker(StartChannel) + produced = DummyExecutor.produce_state_on_start(port=0) + main_loop.context.state_processing_manager.current_output_state = ( + produced + ) + tpm.current_internal_marker = None # consumed + + monkeypatch.setattr(main_loop, "_switch_context", fake_switch_context) + + # Drive the path: this is exactly what `_process_ecm` calls when a + # StartChannel ECM arrives and the start_channel handler has set + # the marker. + from core.models.internal_marker import StartChannel + + main_loop.context.tuple_processing_manager.current_internal_marker = ( + StartChannel() + ) + main_loop._process_start_channel() + + # The state produced by produce_state_on_start must be persisted to + # iceberg via save_state_to_storage_if_needed. Without this, a + # downstream worker in a different region cannot observe the state. + assert len(saved_states) == 1, ( + f"produce_state_on_start emitted a state but it was not persisted " + f"to storage. saved_states={saved_states}" + ) + assert saved_states[0]["flag"] is True + assert saved_states[0]["loop_counter"] == 0 + assert saved_states[0]["port"] == 0 + @pytest.mark.timeout(2) def test_process_input_state_does_not_save_when_no_output( self,