catalog/system_schema/information_schema/
cluster_info.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
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
// Copyright 2023 Greptime Team
//
// 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, Weak};
use std::time::Duration;

use arrow_schema::SchemaRef as ArrowSchemaRef;
use common_catalog::consts::INFORMATION_SCHEMA_CLUSTER_INFO_TABLE_ID;
use common_error::ext::BoxedError;
use common_meta::cluster::NodeInfo;
use common_recordbatch::adapter::RecordBatchStreamAdapter;
use common_recordbatch::{RecordBatch, SendableRecordBatchStream};
use common_time::timestamp::Timestamp;
use datafusion::execution::TaskContext;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef};
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::timestamp::TimestampMillisecond;
use datatypes::value::Value;
use datatypes::vectors::{
    Int64VectorBuilder, StringVectorBuilder, TimestampMillisecondVectorBuilder,
};
use snafu::ResultExt;
use store_api::storage::{ScanRequest, TableId};

use super::CLUSTER_INFO;
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result};
use crate::system_schema::information_schema::{InformationTable, Predicates};
use crate::system_schema::utils;
use crate::CatalogManager;

const PEER_ID: &str = "peer_id";
const PEER_TYPE: &str = "peer_type";
const PEER_ADDR: &str = "peer_addr";
const VERSION: &str = "version";
const GIT_COMMIT: &str = "git_commit";
const START_TIME: &str = "start_time";
const UPTIME: &str = "uptime";
const ACTIVE_TIME: &str = "active_time";

const INIT_CAPACITY: usize = 42;

/// The `CLUSTER_INFO` table provides information about the current topology information of the cluster.
///
/// - `peer_id`: the peer server id.
/// - `peer_type`: the peer type, such as `datanode`, `frontend`, `metasrv` etc.
/// - `peer_addr`: the peer gRPC address.
/// - `version`: the build package version of the peer.
/// - `git_commit`: the build git commit hash of the peer.
/// - `start_time`: the starting time of the peer.
/// - `uptime`: the uptime of the peer.
/// - `active_time`: the time since the last activity of the peer.
///
pub(super) struct InformationSchemaClusterInfo {
    schema: SchemaRef,
    catalog_manager: Weak<dyn CatalogManager>,
}

impl InformationSchemaClusterInfo {
    pub(super) fn new(catalog_manager: Weak<dyn CatalogManager>) -> Self {
        Self {
            schema: Self::schema(),
            catalog_manager,
        }
    }

    pub(crate) fn schema() -> SchemaRef {
        Arc::new(Schema::new(vec![
            ColumnSchema::new(PEER_ID, ConcreteDataType::int64_datatype(), false),
            ColumnSchema::new(PEER_TYPE, ConcreteDataType::string_datatype(), false),
            ColumnSchema::new(PEER_ADDR, ConcreteDataType::string_datatype(), true),
            ColumnSchema::new(VERSION, ConcreteDataType::string_datatype(), false),
            ColumnSchema::new(GIT_COMMIT, ConcreteDataType::string_datatype(), false),
            ColumnSchema::new(
                START_TIME,
                ConcreteDataType::timestamp_millisecond_datatype(),
                true,
            ),
            ColumnSchema::new(UPTIME, ConcreteDataType::string_datatype(), true),
            ColumnSchema::new(ACTIVE_TIME, ConcreteDataType::string_datatype(), true),
        ]))
    }

    fn builder(&self) -> InformationSchemaClusterInfoBuilder {
        InformationSchemaClusterInfoBuilder::new(self.schema.clone(), self.catalog_manager.clone())
    }
}

impl InformationTable for InformationSchemaClusterInfo {
    fn table_id(&self) -> TableId {
        INFORMATION_SCHEMA_CLUSTER_INFO_TABLE_ID
    }

    fn table_name(&self) -> &'static str {
        CLUSTER_INFO
    }

    fn schema(&self) -> SchemaRef {
        self.schema.clone()
    }

    fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
        let schema = self.schema.arrow_schema().clone();
        let mut builder = self.builder();
        let stream = Box::pin(DfRecordBatchStreamAdapter::new(
            schema,
            futures::stream::once(async move {
                builder
                    .make_cluster_info(Some(request))
                    .await
                    .map(|x| x.into_df_record_batch())
                    .map_err(Into::into)
            }),
        ));
        Ok(Box::pin(
            RecordBatchStreamAdapter::try_new(stream)
                .map_err(BoxedError::new)
                .context(InternalSnafu)?,
        ))
    }
}

struct InformationSchemaClusterInfoBuilder {
    schema: SchemaRef,
    catalog_manager: Weak<dyn CatalogManager>,

    peer_ids: Int64VectorBuilder,
    peer_types: StringVectorBuilder,
    peer_addrs: StringVectorBuilder,
    versions: StringVectorBuilder,
    git_commits: StringVectorBuilder,
    start_times: TimestampMillisecondVectorBuilder,
    uptimes: StringVectorBuilder,
    active_times: StringVectorBuilder,
}

impl InformationSchemaClusterInfoBuilder {
    fn new(schema: SchemaRef, catalog_manager: Weak<dyn CatalogManager>) -> Self {
        Self {
            schema,
            catalog_manager,
            peer_ids: Int64VectorBuilder::with_capacity(INIT_CAPACITY),
            peer_types: StringVectorBuilder::with_capacity(INIT_CAPACITY),
            peer_addrs: StringVectorBuilder::with_capacity(INIT_CAPACITY),
            versions: StringVectorBuilder::with_capacity(INIT_CAPACITY),
            git_commits: StringVectorBuilder::with_capacity(INIT_CAPACITY),
            start_times: TimestampMillisecondVectorBuilder::with_capacity(INIT_CAPACITY),
            uptimes: StringVectorBuilder::with_capacity(INIT_CAPACITY),
            active_times: StringVectorBuilder::with_capacity(INIT_CAPACITY),
        }
    }

    /// Construct the `information_schema.cluster_info` virtual table
    async fn make_cluster_info(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
        let predicates = Predicates::from_scan_request(&request);
        let information_extension = utils::information_extension(&self.catalog_manager)?;
        let node_infos = information_extension.nodes().await?;
        for node_info in node_infos {
            self.add_node_info(&predicates, node_info);
        }
        self.finish()
    }

    fn add_node_info(&mut self, predicates: &Predicates, node_info: NodeInfo) {
        let peer_type = node_info.status.role_name();

        let row = [
            (PEER_ID, &Value::from(node_info.peer.id)),
            (PEER_TYPE, &Value::from(peer_type)),
            (PEER_ADDR, &Value::from(node_info.peer.addr.as_str())),
            (VERSION, &Value::from(node_info.version.as_str())),
            (GIT_COMMIT, &Value::from(node_info.git_commit.as_str())),
        ];

        if !predicates.eval(&row) {
            return;
        }

        if peer_type == "FRONTEND" || peer_type == "METASRV" {
            // Always set peer_id to be -1 for frontends and metasrvs
            self.peer_ids.push(Some(-1));
        } else {
            self.peer_ids.push(Some(node_info.peer.id as i64));
        }

        self.peer_types.push(Some(peer_type));
        self.peer_addrs.push(Some(&node_info.peer.addr));
        self.versions.push(Some(&node_info.version));
        self.git_commits.push(Some(&node_info.git_commit));
        if node_info.start_time_ms > 0 {
            self.start_times
                .push(Some(TimestampMillisecond(Timestamp::new_millisecond(
                    node_info.start_time_ms as i64,
                ))));
            self.uptimes.push(Some(
                Self::format_duration_since(node_info.start_time_ms).as_str(),
            ));
        } else {
            self.start_times.push(None);
            self.uptimes.push(None);
        }

        if node_info.last_activity_ts > 0 {
            self.active_times.push(Some(
                Self::format_duration_since(node_info.last_activity_ts as u64).as_str(),
            ));
        } else {
            self.active_times.push(None);
        }
    }

    fn format_duration_since(ts: u64) -> String {
        let now = common_time::util::current_time_millis() as u64;
        let duration_since = now - ts;
        humantime::format_duration(Duration::from_millis(duration_since)).to_string()
    }

    fn finish(&mut self) -> Result<RecordBatch> {
        let columns: Vec<VectorRef> = vec![
            Arc::new(self.peer_ids.finish()),
            Arc::new(self.peer_types.finish()),
            Arc::new(self.peer_addrs.finish()),
            Arc::new(self.versions.finish()),
            Arc::new(self.git_commits.finish()),
            Arc::new(self.start_times.finish()),
            Arc::new(self.uptimes.finish()),
            Arc::new(self.active_times.finish()),
        ];
        RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
    }
}

impl DfPartitionStream for InformationSchemaClusterInfo {
    fn schema(&self) -> &ArrowSchemaRef {
        self.schema.arrow_schema()
    }

    fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
        let schema = self.schema.arrow_schema().clone();
        let mut builder = self.builder();
        Box::pin(DfRecordBatchStreamAdapter::new(
            schema,
            futures::stream::once(async move {
                builder
                    .make_cluster_info(None)
                    .await
                    .map(|x| x.into_df_record_batch())
                    .map_err(Into::into)
            }),
        ))
    }
}