risingwave_frontend/scheduler/
task_context.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::sync::Arc;

use prometheus::core::Atomic;
use risingwave_batch::error::Result;
use risingwave_batch::monitor::BatchMetrics;
use risingwave_batch::task::{BatchTaskContext, TaskOutput, TaskOutputId};
use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeManagerRef;
use risingwave_common::catalog::SysCatalogReaderRef;
use risingwave_common::config::BatchConfig;
use risingwave_common::memory::MemoryContext;
use risingwave_common::metrics::TrAdderAtomic;
use risingwave_common::util::addr::{is_local_address, HostAddr};
use risingwave_connector::source::monitor::SourceMetrics;
use risingwave_rpc_client::ComputeClientPoolRef;

use crate::catalog::system_catalog::SysCatalogReaderImpl;
use crate::session::SessionImpl;

/// Batch task execution context in frontend.
#[derive(Clone)]
pub struct FrontendBatchTaskContext {
    session: Arc<SessionImpl>,

    mem_context: MemoryContext,
}

impl FrontendBatchTaskContext {
    pub fn new(session: Arc<SessionImpl>) -> Self {
        let mem_context =
            MemoryContext::new(Some(session.env().mem_context()), TrAdderAtomic::new(0));
        Self {
            session,
            mem_context,
        }
    }
}

impl BatchTaskContext for FrontendBatchTaskContext {
    fn get_task_output(&self, _task_output_id: TaskOutputId) -> Result<TaskOutput> {
        unimplemented!("not supported in local mode")
    }

    fn catalog_reader(&self) -> SysCatalogReaderRef {
        Arc::new(SysCatalogReaderImpl::new(
            self.session.env().catalog_reader().clone(),
            self.session.env().user_info_reader().clone(),
            self.session.env().meta_client_ref(),
            self.session.auth_context(),
            self.session.shared_config(),
            self.session.env().system_params_manager().get_params(),
        ))
    }

    fn is_local_addr(&self, peer_addr: &HostAddr) -> bool {
        is_local_address(self.session.env().server_address(), peer_addr)
    }

    fn state_store(&self) -> risingwave_storage::store_impl::StateStoreImpl {
        unimplemented!("not supported in local mode")
    }

    fn batch_metrics(&self) -> Option<BatchMetrics> {
        None
    }

    fn client_pool(&self) -> ComputeClientPoolRef {
        self.session.env().client_pool()
    }

    fn get_config(&self) -> &BatchConfig {
        self.session.env().batch_config()
    }

    fn dml_manager(&self) -> risingwave_dml::dml_manager::DmlManagerRef {
        unimplemented!("not supported in local mode")
    }

    fn source_metrics(&self) -> Arc<SourceMetrics> {
        self.session.env().source_metrics()
    }

    fn spill_metrics(&self) -> Arc<risingwave_batch::monitor::BatchSpillMetrics> {
        self.session.env().spill_metrics()
    }

    fn create_executor_mem_context(&self, _executor_id: &str) -> MemoryContext {
        MemoryContext::new(Some(self.mem_context.clone()), TrAdderAtomic::new(0))
    }

    fn worker_node_manager(&self) -> Option<WorkerNodeManagerRef> {
        Some(self.session.env().worker_node_manager_ref())
    }
}