pgwire/
pg_server.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
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
// 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::collections::HashMap;
use std::future::Future;
use std::str::FromStr;
use std::sync::Arc;
use std::time::Instant;

use bytes::Bytes;
use jsonwebtoken::{decode, decode_header, Algorithm, DecodingKey, Validation};
use parking_lot::Mutex;
use risingwave_common::types::DataType;
use risingwave_common::util::runtime::BackgroundShutdownRuntime;
use risingwave_common::util::tokio_util::sync::CancellationToken;
use risingwave_sqlparser::ast::{RedactSqlOptionKeywordsRef, Statement};
use serde::Deserialize;
use thiserror_ext::AsReport;
use tokio::io::{AsyncRead, AsyncWrite};

use crate::error::{PsqlError, PsqlResult};
use crate::net::{AddressRef, Listener, TcpKeepalive};
use crate::pg_field_descriptor::PgFieldDescriptor;
use crate::pg_message::TransactionStatus;
use crate::pg_protocol::{PgProtocol, TlsConfig};
use crate::pg_response::{PgResponse, ValuesStream};
use crate::types::Format;

pub type BoxedError = Box<dyn std::error::Error + Send + Sync>;
type ProcessId = i32;
type SecretKey = i32;
pub type SessionId = (ProcessId, SecretKey);

/// The interface for a database system behind pgwire protocol.
/// We can mock it for testing purpose.
pub trait SessionManager: Send + Sync + 'static {
    type Session: Session;

    /// In the process of auto schema change, we need a dummy session to access
    /// catalog information in frontend and build a replace plan for the table.
    fn create_dummy_session(
        &self,
        database_id: u32,
        user_id: u32,
    ) -> Result<Arc<Self::Session>, BoxedError>;

    fn connect(
        &self,
        database: &str,
        user_name: &str,
        peer_addr: AddressRef,
    ) -> Result<Arc<Self::Session>, BoxedError>;

    fn cancel_queries_in_session(&self, session_id: SessionId);

    fn cancel_creating_jobs_in_session(&self, session_id: SessionId);

    fn end_session(&self, session: &Self::Session);

    /// Run some cleanup tasks before the server shutdown.
    fn shutdown(&self) -> impl Future<Output = ()> + Send {
        async {}
    }
}

/// A psql connection. Each connection binds with a database. Switching database will need to
/// recreate another connection.
pub trait Session: Send + Sync {
    type ValuesStream: ValuesStream;
    type PreparedStatement: Send + Clone + 'static;
    type Portal: Send + Clone + std::fmt::Display + 'static;

    /// The str sql can not use the unparse from AST: There is some problem when dealing with create
    /// view, see <https://github.com/risingwavelabs/risingwave/issues/6801>.
    fn run_one_query(
        self: Arc<Self>,
        stmt: Statement,
        format: Format,
    ) -> impl Future<Output = Result<PgResponse<Self::ValuesStream>, BoxedError>> + Send;

    fn parse(
        self: Arc<Self>,
        sql: Option<Statement>,
        params_types: Vec<Option<DataType>>,
    ) -> impl Future<Output = Result<Self::PreparedStatement, BoxedError>> + Send;

    // TODO: maybe this function should be async and return the notice more timely
    /// try to take the current notices from the session
    fn take_notices(self: Arc<Self>) -> Vec<String>;

    fn bind(
        self: Arc<Self>,
        prepare_statement: Self::PreparedStatement,
        params: Vec<Option<Bytes>>,
        param_formats: Vec<Format>,
        result_formats: Vec<Format>,
    ) -> Result<Self::Portal, BoxedError>;

    fn execute(
        self: Arc<Self>,
        portal: Self::Portal,
    ) -> impl Future<Output = Result<PgResponse<Self::ValuesStream>, BoxedError>> + Send;

    fn describe_statement(
        self: Arc<Self>,
        prepare_statement: Self::PreparedStatement,
    ) -> Result<(Vec<DataType>, Vec<PgFieldDescriptor>), BoxedError>;

    fn describe_portal(
        self: Arc<Self>,
        portal: Self::Portal,
    ) -> Result<Vec<PgFieldDescriptor>, BoxedError>;

    fn user_authenticator(&self) -> &UserAuthenticator;

    fn id(&self) -> SessionId;

    fn set_config(&self, key: &str, value: String) -> Result<String, BoxedError>;

    fn transaction_status(&self) -> TransactionStatus;

    fn init_exec_context(&self, sql: Arc<str>) -> ExecContextGuard;

    fn check_idle_in_transaction_timeout(&self) -> PsqlResult<()>;
}

/// Each session could run different SQLs multiple times.
/// `ExecContext` represents the lifetime of a running SQL in the current session.
pub struct ExecContext {
    pub running_sql: Arc<str>,
    /// The instant of the running sql
    pub last_instant: Instant,
    /// A reference used to update when `ExecContext` is dropped
    pub last_idle_instant: Arc<Mutex<Option<Instant>>>,
}

/// `ExecContextGuard` holds a `Arc` pointer. Once `ExecContextGuard` is dropped,
/// the inner `Arc<ExecContext>` should not be referred anymore, so that its `Weak` reference (used in `SessionImpl`) will be the same lifecycle of the running sql execution context.
pub struct ExecContextGuard(#[allow(dead_code)] Arc<ExecContext>);

impl ExecContextGuard {
    pub fn new(exec_context: Arc<ExecContext>) -> Self {
        Self(exec_context)
    }
}

impl Drop for ExecContext {
    fn drop(&mut self) {
        *self.last_idle_instant.lock() = Some(Instant::now());
    }
}

#[derive(Debug, Clone)]
pub enum UserAuthenticator {
    // No need to authenticate.
    None,
    // raw password in clear-text form.
    ClearText(Vec<u8>),
    // password encrypted with random salt.
    Md5WithSalt {
        encrypted_password: Vec<u8>,
        salt: [u8; 4],
    },
    OAuth(HashMap<String, String>),
}

/// A JWK Set is a JSON object that represents a set of JWKs.
/// The JSON object MUST have a "keys" member, with its value being an array of JWKs.
/// See <https://www.rfc-editor.org/rfc/rfc7517.html#section-5> for more details.
#[derive(Debug, Deserialize)]
struct Jwks {
    keys: Vec<Jwk>,
}

/// A JSON Web Key (JWK) is a JSON object that represents a cryptographic key.
/// See <https://www.rfc-editor.org/rfc/rfc7517.html#section-4> for more details.
#[derive(Debug, Deserialize)]
struct Jwk {
    kid: String, // Key ID
    alg: String, // Algorithm
    n: String,   // Modulus
    e: String,   // Exponent
}

async fn validate_jwt(
    jwt: &str,
    jwks_url: &str,
    issuer: &str,
    metadata: &HashMap<String, String>,
) -> Result<bool, BoxedError> {
    let header = decode_header(jwt)?;
    let jwks: Jwks = reqwest::get(jwks_url).await?.json().await?;

    // 1. Retrieve the kid from the header to find the right JWK in the JWK Set.
    let kid = header.kid.ok_or("kid not found in jwt header")?;
    let jwk = jwks
        .keys
        .into_iter()
        .find(|k| k.kid == kid)
        .ok_or("kid not found in jwks")?;

    // 2. Check if the algorithms are matched.
    if Algorithm::from_str(&jwk.alg)? != header.alg {
        return Err("alg in jwt header does not match with alg in jwk".into());
    }

    // 3. Decode the JWT and validate the claims.
    let decoding_key = DecodingKey::from_rsa_components(&jwk.n, &jwk.e)?;
    let mut validation = Validation::new(header.alg);
    validation.set_issuer(&[issuer]);
    validation.set_required_spec_claims(&["exp", "iss"]);
    let token_data = decode::<HashMap<String, serde_json::Value>>(jwt, &decoding_key, &validation)?;

    // 4. Check if the metadata in the token matches.
    if !metadata.iter().all(
        |(k, v)| matches!(token_data.claims.get(k), Some(serde_json::Value::String(s)) if s == v),
    ) {
        return Err("metadata in jwt does not match with metadata declared with user".into());
    }
    Ok(true)
}

impl UserAuthenticator {
    pub async fn authenticate(&self, password: &[u8]) -> PsqlResult<()> {
        let success = match self {
            UserAuthenticator::None => true,
            UserAuthenticator::ClearText(text) => password == text,
            UserAuthenticator::Md5WithSalt {
                encrypted_password, ..
            } => encrypted_password == password,
            UserAuthenticator::OAuth(metadata) => {
                let mut metadata = metadata.clone();
                let jwks_url = metadata.remove("jwks_url").unwrap();
                let issuer = metadata.remove("issuer").unwrap();
                validate_jwt(
                    &String::from_utf8_lossy(password),
                    &jwks_url,
                    &issuer,
                    &metadata,
                )
                .await
                .map_err(PsqlError::StartupError)?
            }
        };
        if !success {
            return Err(PsqlError::PasswordError);
        }
        Ok(())
    }
}

/// Binds a Tcp or Unix listener at `addr`. Spawn a coroutine to serve every new connection.
///
/// Returns when the `shutdown` token is triggered.
pub async fn pg_serve(
    addr: &str,
    tcp_keepalive: TcpKeepalive,
    session_mgr: Arc<impl SessionManager>,
    tls_config: Option<TlsConfig>,
    redact_sql_option_keywords: Option<RedactSqlOptionKeywordsRef>,
    shutdown: CancellationToken,
) -> Result<(), BoxedError> {
    let listener = Listener::bind(addr).await?;
    tracing::info!(addr, "server started");

    let acceptor_runtime = BackgroundShutdownRuntime::from({
        let mut builder = tokio::runtime::Builder::new_multi_thread();
        builder.worker_threads(1);
        builder
            .thread_name("rw-acceptor")
            .enable_all()
            .build()
            .unwrap()
    });

    #[cfg(not(madsim))]
    let worker_runtime = tokio::runtime::Handle::current();
    #[cfg(madsim)]
    let worker_runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap();

    let session_mgr_clone = session_mgr.clone();
    let f = async move {
        loop {
            let conn_ret = listener.accept(&tcp_keepalive).await;
            match conn_ret {
                Ok((stream, peer_addr)) => {
                    tracing::info!(%peer_addr, "accept connection");
                    worker_runtime.spawn(handle_connection(
                        stream,
                        session_mgr_clone.clone(),
                        tls_config.clone(),
                        Arc::new(peer_addr),
                        redact_sql_option_keywords.clone(),
                    ));
                }

                Err(e) => {
                    tracing::error!(error = %e.as_report(), "failed to accept connection",);
                }
            }
        }
    };
    acceptor_runtime.spawn(f);

    // Wait for the shutdown signal.
    shutdown.cancelled().await;

    // Stop accepting new connections.
    drop(acceptor_runtime);
    // Shutdown session manager, typically close all existing sessions.
    session_mgr.shutdown().await;

    Ok(())
}

pub async fn handle_connection<S, SM>(
    stream: S,
    session_mgr: Arc<SM>,
    tls_config: Option<TlsConfig>,
    peer_addr: AddressRef,
    redact_sql_option_keywords: Option<RedactSqlOptionKeywordsRef>,
) where
    S: AsyncWrite + AsyncRead + Unpin,
    SM: SessionManager,
{
    let mut pg_proto = PgProtocol::new(
        stream,
        session_mgr,
        tls_config,
        peer_addr,
        redact_sql_option_keywords,
    );
    loop {
        let msg = match pg_proto.read_message().await {
            Ok(msg) => msg,
            Err(e) => {
                tracing::error!(error = %e.as_report(), "error when reading message");
                break;
            }
        };
        tracing::trace!("Received message: {:?}", msg);
        let ret = pg_proto.process(msg).await;
        if ret {
            break;
        }
    }
}

#[cfg(test)]
mod tests {
    use std::error::Error;
    use std::sync::Arc;
    use std::time::Instant;

    use bytes::Bytes;
    use futures::stream::BoxStream;
    use futures::StreamExt;
    use risingwave_common::types::DataType;
    use risingwave_common::util::tokio_util::sync::CancellationToken;
    use risingwave_sqlparser::ast::Statement;
    use tokio_postgres::NoTls;

    use crate::error::PsqlResult;
    use crate::pg_field_descriptor::PgFieldDescriptor;
    use crate::pg_message::TransactionStatus;
    use crate::pg_response::{PgResponse, RowSetResult, StatementType};
    use crate::pg_server::{
        pg_serve, BoxedError, ExecContext, ExecContextGuard, Session, SessionId, SessionManager,
        UserAuthenticator,
    };
    use crate::types;
    use crate::types::Row;

    struct MockSessionManager {}
    struct MockSession {}

    impl SessionManager for MockSessionManager {
        type Session = MockSession;

        fn create_dummy_session(
            &self,
            _database_id: u32,
            _user_name: u32,
        ) -> Result<Arc<Self::Session>, BoxedError> {
            unimplemented!()
        }

        fn connect(
            &self,
            _database: &str,
            _user_name: &str,
            _peer_addr: crate::net::AddressRef,
        ) -> Result<Arc<Self::Session>, Box<dyn Error + Send + Sync>> {
            Ok(Arc::new(MockSession {}))
        }

        fn cancel_queries_in_session(&self, _session_id: SessionId) {
            todo!()
        }

        fn cancel_creating_jobs_in_session(&self, _session_id: SessionId) {
            todo!()
        }

        fn end_session(&self, _session: &Self::Session) {}
    }

    impl Session for MockSession {
        type Portal = String;
        type PreparedStatement = String;
        type ValuesStream = BoxStream<'static, RowSetResult>;

        async fn run_one_query(
            self: Arc<Self>,
            _stmt: Statement,
            _format: types::Format,
        ) -> Result<PgResponse<BoxStream<'static, RowSetResult>>, BoxedError> {
            Ok(PgResponse::builder(StatementType::SELECT)
                .values(
                    futures::stream::iter(vec![Ok(vec![Row::new(vec![Some(Bytes::new())])])])
                        .boxed(),
                    vec![
                        // 1043 is the oid of varchar type.
                        // -1 is the type len of varchar type.
                        PgFieldDescriptor::new("".to_string(), 1043, -1);
                        1
                    ],
                )
                .into())
        }

        async fn parse(
            self: Arc<Self>,
            _sql: Option<Statement>,
            _params_types: Vec<Option<DataType>>,
        ) -> Result<String, BoxedError> {
            Ok(String::new())
        }

        fn bind(
            self: Arc<Self>,
            _prepare_statement: String,
            _params: Vec<Option<Bytes>>,
            _param_formats: Vec<types::Format>,
            _result_formats: Vec<types::Format>,
        ) -> Result<String, BoxedError> {
            Ok(String::new())
        }

        async fn execute(
            self: Arc<Self>,
            _portal: String,
        ) -> Result<PgResponse<BoxStream<'static, RowSetResult>>, BoxedError> {
            Ok(PgResponse::builder(StatementType::SELECT)
                .values(
                    futures::stream::iter(vec![Ok(vec![Row::new(vec![Some(Bytes::new())])])])
                        .boxed(),
                    vec![
                    // 1043 is the oid of varchar type.
                    // -1 is the type len of varchar type.
                    PgFieldDescriptor::new("".to_string(), 1043, -1);
                    1
                ],
                )
                .into())
        }

        fn describe_statement(
            self: Arc<Self>,
            _statement: String,
        ) -> Result<(Vec<DataType>, Vec<PgFieldDescriptor>), BoxedError> {
            Ok((
                vec![],
                vec![PgFieldDescriptor::new("".to_string(), 1043, -1)],
            ))
        }

        fn describe_portal(
            self: Arc<Self>,
            _portal: String,
        ) -> Result<Vec<PgFieldDescriptor>, BoxedError> {
            Ok(vec![PgFieldDescriptor::new("".to_string(), 1043, -1)])
        }

        fn user_authenticator(&self) -> &UserAuthenticator {
            &UserAuthenticator::None
        }

        fn id(&self) -> SessionId {
            (0, 0)
        }

        fn set_config(&self, _key: &str, _value: String) -> Result<String, BoxedError> {
            Ok("".to_string())
        }

        fn take_notices(self: Arc<Self>) -> Vec<String> {
            vec![]
        }

        fn transaction_status(&self) -> TransactionStatus {
            TransactionStatus::Idle
        }

        fn init_exec_context(&self, sql: Arc<str>) -> ExecContextGuard {
            let exec_context = Arc::new(ExecContext {
                running_sql: sql,
                last_instant: Instant::now(),
                last_idle_instant: Default::default(),
            });
            ExecContextGuard::new(exec_context)
        }

        fn check_idle_in_transaction_timeout(&self) -> PsqlResult<()> {
            Ok(())
        }
    }

    async fn do_test_query(bind_addr: impl Into<String>, pg_config: impl Into<String>) {
        let bind_addr = bind_addr.into();
        let pg_config = pg_config.into();

        let session_mgr = MockSessionManager {};
        tokio::spawn(async move {
            pg_serve(
                &bind_addr,
                socket2::TcpKeepalive::new(),
                Arc::new(session_mgr),
                None,
                None,
                CancellationToken::new(), // dummy
            )
            .await
        });
        // wait for server to start
        tokio::time::sleep(std::time::Duration::from_millis(100)).await;

        // Connect to the database.
        let (client, connection) = tokio_postgres::connect(&pg_config, NoTls).await.unwrap();

        // The connection object performs the actual communication with the database,
        // so spawn it off to run on its own.
        tokio::spawn(async move {
            if let Err(e) = connection.await {
                eprintln!("connection error: {}", e);
            }
        });

        let rows = client
            .simple_query("SELECT ''")
            .await
            .expect("Error executing query");
        // Row + CommandComplete
        assert_eq!(rows.len(), 2);

        let rows = client
            .query("SELECT ''", &[])
            .await
            .expect("Error executing query");
        assert_eq!(rows.len(), 1);
    }

    #[tokio::test]
    async fn test_query_tcp() {
        do_test_query("127.0.0.1:10000", "host=localhost port=10000").await;
    }

    #[cfg(not(madsim))]
    #[tokio::test]
    async fn test_query_unix() {
        let port: i16 = 10000;
        let dir = tempfile::TempDir::new().unwrap();
        let sock = dir.path().join(format!(".s.PGSQL.{port}"));

        do_test_query(
            format!("unix:{}", sock.to_str().unwrap()),
            format!("host={} port={}", dir.path().to_str().unwrap(), port),
        )
        .await;
    }
}