risingwave_batch/task/
context.rs1use std::sync::Arc;
15
16use prometheus::core::Atomic;
17use risingwave_common::catalog::SysCatalogReaderRef;
18use risingwave_common::config::BatchConfig;
19use risingwave_common::memory::MemoryContext;
20use risingwave_common::metrics::TrAdderAtomic;
21use risingwave_common::util::addr::{HostAddr, is_local_address};
22use risingwave_connector::source::monitor::SourceMetrics;
23use risingwave_dml::dml_manager::DmlManagerRef;
24use risingwave_rpc_client::ComputeClientPoolRef;
25use risingwave_storage::StateStoreImpl;
26
27use crate::error::Result;
28use crate::monitor::{BatchMetrics, BatchMetricsInner, BatchSpillMetrics};
29use crate::task::{BatchEnvironment, TaskOutput, TaskOutputId};
30use crate::worker_manager::worker_node_manager::WorkerNodeManagerRef;
31
32pub trait BatchTaskContext: Send + Sync + 'static {
36 fn get_task_output(&self, task_output_id: TaskOutputId) -> Result<TaskOutput>;
40
41 fn catalog_reader(&self) -> SysCatalogReaderRef;
43
44 fn is_local_addr(&self, peer_addr: &HostAddr) -> bool;
46
47 fn dml_manager(&self) -> DmlManagerRef;
48
49 fn state_store(&self) -> StateStoreImpl;
50
51 fn batch_metrics(&self) -> Option<BatchMetrics>;
54
55 fn spill_metrics(&self) -> Arc<BatchSpillMetrics>;
56
57 fn client_pool(&self) -> ComputeClientPoolRef;
60
61 fn get_config(&self) -> &BatchConfig;
63
64 fn source_metrics(&self) -> Arc<SourceMetrics>;
65
66 fn create_executor_mem_context(&self, executor_id: &str) -> MemoryContext;
67
68 fn worker_node_manager(&self) -> Option<WorkerNodeManagerRef>;
69}
70
71#[derive(Clone)]
73pub struct ComputeNodeContext {
74 env: BatchEnvironment,
75
76 batch_metrics: BatchMetrics,
77
78 mem_context: MemoryContext,
79}
80
81impl BatchTaskContext for ComputeNodeContext {
82 fn get_task_output(&self, task_output_id: TaskOutputId) -> Result<TaskOutput> {
83 self.env
84 .task_manager()
85 .take_output(&task_output_id.to_prost())
86 }
87
88 fn catalog_reader(&self) -> SysCatalogReaderRef {
89 unimplemented!("not supported in distributed mode")
90 }
91
92 fn is_local_addr(&self, peer_addr: &HostAddr) -> bool {
93 is_local_address(self.env.server_address(), peer_addr)
94 }
95
96 fn dml_manager(&self) -> DmlManagerRef {
97 self.env.dml_manager_ref()
98 }
99
100 fn state_store(&self) -> StateStoreImpl {
101 self.env.state_store()
102 }
103
104 fn batch_metrics(&self) -> Option<BatchMetrics> {
105 Some(self.batch_metrics.clone())
106 }
107
108 fn spill_metrics(&self) -> Arc<BatchSpillMetrics> {
109 self.env.spill_metrics()
110 }
111
112 fn client_pool(&self) -> ComputeClientPoolRef {
113 self.env.client_pool()
114 }
115
116 fn get_config(&self) -> &BatchConfig {
117 self.env.config()
118 }
119
120 fn source_metrics(&self) -> Arc<SourceMetrics> {
121 self.env.source_metrics()
122 }
123
124 fn create_executor_mem_context(&self, _executor_id: &str) -> MemoryContext {
125 let counter = TrAdderAtomic::new(0);
126 MemoryContext::new(Some(self.mem_context.clone()), counter)
127 }
128
129 fn worker_node_manager(&self) -> Option<WorkerNodeManagerRef> {
130 None
131 }
132}
133
134impl ComputeNodeContext {
135 pub fn for_test() -> Arc<dyn BatchTaskContext> {
136 Arc::new(Self {
137 env: BatchEnvironment::for_test(),
138 batch_metrics: BatchMetricsInner::for_test(),
139 mem_context: MemoryContext::none(),
140 })
141 }
142
143 pub fn create(env: BatchEnvironment) -> Arc<dyn BatchTaskContext> {
144 let mem_context = env.task_manager().memory_context_ref();
145 let batch_metrics = Arc::new(BatchMetricsInner::new(
146 env.task_manager().metrics(),
147 env.executor_metrics(),
148 env.iceberg_scan_metrics(),
149 ));
150 Arc::new(Self {
151 env,
152 batch_metrics,
153 mem_context,
154 })
155 }
156}