pgwire/
pg_protocol.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
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
// 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::any::Any;
use std::collections::HashMap;
use std::panic::AssertUnwindSafe;
use std::pin::Pin;
use std::str::Utf8Error;
use std::sync::{Arc, LazyLock, Weak};
use std::time::{Duration, Instant};
use std::{io, str};

use bytes::{Bytes, BytesMut};
use futures::stream::StreamExt;
use itertools::Itertools;
use openssl::ssl::{SslAcceptor, SslContext, SslContextRef, SslMethod};
use risingwave_common::types::DataType;
use risingwave_common::util::panic::FutureCatchUnwindExt;
use risingwave_common::util::query_log::*;
use risingwave_common::{PG_VERSION, SERVER_ENCODING, STANDARD_CONFORMING_STRINGS};
use risingwave_sqlparser::ast::{RedactSqlOptionKeywordsRef, Statement};
use risingwave_sqlparser::parser::Parser;
use thiserror_ext::AsReport;
use tokio::io::{AsyncRead, AsyncWrite, AsyncWriteExt};
use tokio_openssl::SslStream;
use tracing::Instrument;

use crate::error::{PsqlError, PsqlResult};
use crate::net::AddressRef;
use crate::pg_extended::ResultCache;
use crate::pg_message::{
    BeCommandCompleteMessage, BeMessage, BeParameterStatusMessage, FeBindMessage, FeCancelMessage,
    FeCloseMessage, FeDescribeMessage, FeExecuteMessage, FeMessage, FeParseMessage,
    FePasswordMessage, FeStartupMessage, TransactionStatus,
};
use crate::pg_server::{Session, SessionManager, UserAuthenticator};
use crate::types::Format;

/// Truncates query log if it's longer than `RW_QUERY_LOG_TRUNCATE_LEN`, to avoid log file being too
/// large.
static RW_QUERY_LOG_TRUNCATE_LEN: LazyLock<usize> =
    LazyLock::new(|| match std::env::var("RW_QUERY_LOG_TRUNCATE_LEN") {
        Ok(len) if len.parse::<usize>().is_ok() => len.parse::<usize>().unwrap(),
        _ => {
            if cfg!(debug_assertions) {
                65536
            } else {
                1024
            }
        }
    });

tokio::task_local! {
    /// The current session. Concrete type is erased for different session implementations.
    pub static CURRENT_SESSION: Weak<dyn Any + Send + Sync>
}

/// The state machine for each psql connection.
/// Read pg messages from tcp stream and write results back.
pub struct PgProtocol<S, SM>
where
    SM: SessionManager,
{
    /// Used for write/read pg messages.
    stream: Conn<S>,
    /// Current states of pg connection.
    state: PgProtocolState,
    /// Whether the connection is terminated.
    is_terminate: bool,

    session_mgr: Arc<SM>,
    session: Option<Arc<SM::Session>>,

    result_cache: HashMap<String, ResultCache<<SM::Session as Session>::ValuesStream>>,
    unnamed_prepare_statement: Option<<SM::Session as Session>::PreparedStatement>,
    prepare_statement_store: HashMap<String, <SM::Session as Session>::PreparedStatement>,
    unnamed_portal: Option<<SM::Session as Session>::Portal>,
    portal_store: HashMap<String, <SM::Session as Session>::Portal>,
    // Used to store the dependency of portal and prepare statement.
    // When we close a prepare statement, we need to close all the portals that depend on it.
    statement_portal_dependency: HashMap<String, Vec<String>>,

    // Used for ssl connection.
    // If None, not expected to build ssl connection (panic).
    tls_context: Option<SslContext>,

    // Used in extended query protocol. When encounter error in extended query, we need to ignore
    // the following message util sync message.
    ignore_util_sync: bool,

    // Client Address
    peer_addr: AddressRef,

    redact_sql_option_keywords: Option<RedactSqlOptionKeywordsRef>,
}

/// Configures TLS encryption for connections.
#[derive(Debug, Clone)]
pub struct TlsConfig {
    /// The path to the TLS certificate.
    pub cert: String,
    /// The path to the TLS key.
    pub key: String,
}

impl TlsConfig {
    pub fn new_default() -> Option<Self> {
        let cert = std::env::var("RW_SSL_CERT").ok()?;
        let key = std::env::var("RW_SSL_KEY").ok()?;
        tracing::info!("RW_SSL_CERT={}, RW_SSL_KEY={}", cert, key);
        Some(Self { cert, key })
    }
}

impl<S, SM> Drop for PgProtocol<S, SM>
where
    SM: SessionManager,
{
    fn drop(&mut self) {
        if let Some(session) = &self.session {
            // Clear the session in session manager.
            self.session_mgr.end_session(session);
        }
    }
}

/// States flow happened from top to down.
enum PgProtocolState {
    Startup,
    Regular,
}

/// Truncate 0 from C string in Bytes and stringify it (returns slice, no allocations).
///
/// PG protocol strings are always C strings.
pub fn cstr_to_str(b: &Bytes) -> Result<&str, Utf8Error> {
    let without_null = if b.last() == Some(&0) {
        &b[..b.len() - 1]
    } else {
        &b[..]
    };
    std::str::from_utf8(without_null)
}

/// Record `sql` in the current tracing span.
fn record_sql_in_span(sql: &str, redact_sql_option_keywords: Option<RedactSqlOptionKeywordsRef>) {
    let redacted_sql = if let Some(keywords) = redact_sql_option_keywords {
        redact_sql(sql, keywords)
    } else {
        sql.to_owned()
    };
    tracing::Span::current().record(
        "sql",
        tracing::field::display(truncated_fmt::TruncatedFmt(
            &redacted_sql,
            *RW_QUERY_LOG_TRUNCATE_LEN,
        )),
    );
}

fn redact_sql(sql: &str, keywords: RedactSqlOptionKeywordsRef) -> String {
    match Parser::parse_sql(sql) {
        Ok(sqls) => sqls
            .into_iter()
            .map(|sql| sql.to_redacted_string(keywords.clone()))
            .join(";"),
        Err(_) => sql.to_owned(),
    }
}

impl<S, SM> PgProtocol<S, SM>
where
    S: AsyncWrite + AsyncRead + Unpin,
    SM: SessionManager,
{
    pub fn new(
        stream: S,
        session_mgr: Arc<SM>,
        tls_config: Option<TlsConfig>,
        peer_addr: AddressRef,
        redact_sql_option_keywords: Option<RedactSqlOptionKeywordsRef>,
    ) -> Self {
        Self {
            stream: Conn::Unencrypted(PgStream {
                stream: Some(stream),
                write_buf: BytesMut::with_capacity(10 * 1024),
            }),
            is_terminate: false,
            state: PgProtocolState::Startup,
            session_mgr,
            session: None,
            tls_context: tls_config
                .as_ref()
                .and_then(|e| build_ssl_ctx_from_config(e).ok()),
            result_cache: Default::default(),
            unnamed_prepare_statement: Default::default(),
            prepare_statement_store: Default::default(),
            unnamed_portal: Default::default(),
            portal_store: Default::default(),
            statement_portal_dependency: Default::default(),
            ignore_util_sync: false,
            peer_addr,
            redact_sql_option_keywords,
        }
    }

    /// Processes one message. Returns true if the connection is terminated.
    pub async fn process(&mut self, msg: FeMessage) -> bool {
        self.do_process(msg).await.is_none() || self.is_terminate
    }

    /// The root tracing span for processing a message. The target of the span is
    /// [`PGWIRE_ROOT_SPAN_TARGET`].
    ///
    /// This is used to provide context for the (slow) query logs and traces.
    ///
    /// The span is only effective if there's a current session and the message is
    /// query-related. Otherwise, `Span::none()` is returned.
    fn root_span_for_msg(&self, msg: &FeMessage) -> tracing::Span {
        let Some(session_id) = self.session.as_ref().map(|s| s.id().0) else {
            return tracing::Span::none();
        };

        let mode = match msg {
            FeMessage::Query(_) => "simple query",
            FeMessage::Parse(_) => "extended query parse",
            FeMessage::Execute(_) => "extended query execute",
            _ => return tracing::Span::none(),
        };

        tracing::info_span!(
            target: PGWIRE_ROOT_SPAN_TARGET,
            "handle_query",
            mode,
            session_id,
            sql = tracing::field::Empty, // record SQL later in each `process` call
        )
    }

    /// Return type `Option<()>` is essentially a bool, but allows `?` for early return.
    /// - `None` means to terminate the current connection
    /// - `Some(())` means to continue processing the next message
    async fn do_process(&mut self, msg: FeMessage) -> Option<()> {
        let span = self.root_span_for_msg(&msg);
        let weak_session = self
            .session
            .as_ref()
            .map(|s| Arc::downgrade(s) as Weak<dyn Any + Send + Sync>);

        // Processing the message itself.
        //
        // Note: pin the future to avoid stack overflow as we'll wrap it multiple times
        // in the following code.
        let fut = Box::pin(self.do_process_inner(msg));

        // Set the current session as the context when processing the message, if exists.
        let fut = async move {
            if let Some(session) = weak_session {
                CURRENT_SESSION.scope(session, fut).await
            } else {
                fut.await
            }
        };

        // Catch unwind.
        let fut = async move {
            AssertUnwindSafe(fut)
                .rw_catch_unwind()
                .await
                .unwrap_or_else(|payload| {
                    Err(PsqlError::Panic(
                        panic_message::panic_message(&payload).to_owned(),
                    ))
                })
        };

        // Slow query log.
        let fut = async move {
            let period = *SLOW_QUERY_LOG_PERIOD;
            let mut fut = std::pin::pin!(fut);
            let mut elapsed = Duration::ZERO;

            // Report the SQL in the log periodically if the query is slow.
            loop {
                match tokio::time::timeout(period, &mut fut).await {
                    Ok(result) => break result,
                    Err(_) => {
                        elapsed += period;
                        tracing::info!(
                            target: PGWIRE_SLOW_QUERY_LOG,
                            elapsed = %format_args!("{}ms", elapsed.as_millis()),
                            "slow query"
                        );
                    }
                }
            }
        };

        // Query log.
        let fut = async move {
            let start = Instant::now();
            let result = fut.await;
            let elapsed = start.elapsed();

            // Always log if an error occurs.
            // Note: all messages will be processed through this code path, making it the
            //       only necessary place to log errors.
            if let Err(error) = &result {
                tracing::error!(error = %error.as_report(), "error when process message");
            }

            // Log to optionally-enabled target `PGWIRE_QUERY_LOG`.
            // Only log if we're currently in a tracing span set in `span_for_msg`.
            if !tracing::Span::current().is_none() {
                tracing::info!(
                    target: PGWIRE_QUERY_LOG,
                    status = if result.is_ok() { "ok" } else { "err" },
                    time = %format_args!("{}ms", elapsed.as_millis()),
                );
            }

            result
        };

        // Tracing span.
        let fut = fut.instrument(span);

        // Execute the future and handle the error.
        match fut.await {
            Ok(()) => Some(()),
            Err(e) => {
                match e {
                    PsqlError::IoError(io_err) => {
                        if io_err.kind() == std::io::ErrorKind::UnexpectedEof {
                            return None;
                        }
                    }

                    PsqlError::SslError(_) => {
                        // For ssl error, because the stream has already been consumed, so there is
                        // no way to write more message.
                        return None;
                    }

                    PsqlError::StartupError(_) | PsqlError::PasswordError => {
                        self.stream
                            .write_no_flush(&BeMessage::ErrorResponse(Box::new(e)))
                            .ok()?;
                        let _ = self.stream.flush().await;
                        return None;
                    }

                    PsqlError::SimpleQueryError(_) => {
                        self.stream
                            .write_no_flush(&BeMessage::ErrorResponse(Box::new(e)))
                            .ok()?;
                        self.ready_for_query().ok()?;
                    }

                    PsqlError::IdleInTxnTimeout | PsqlError::Panic(_) => {
                        self.stream
                            .write_no_flush(&BeMessage::ErrorResponse(Box::new(e)))
                            .ok()?;
                        let _ = self.stream.flush().await;

                        // 1. Catching the panic during message processing may leave the session in an
                        // inconsistent state. We forcefully close the connection (then end the
                        // session) here for safety.
                        // 2. Idle in transaction timeout should also close the connection.
                        return None;
                    }

                    PsqlError::Uncategorized(_)
                    | PsqlError::ExtendedPrepareError(_)
                    | PsqlError::ExtendedExecuteError(_) => {
                        self.stream
                            .write_no_flush(&BeMessage::ErrorResponse(Box::new(e)))
                            .ok()?;
                    }
                }
                let _ = self.stream.flush().await;
                Some(())
            }
        }
    }

    async fn do_process_inner(&mut self, msg: FeMessage) -> PsqlResult<()> {
        // Ignore util sync message.
        if self.ignore_util_sync {
            if let FeMessage::Sync = msg {
            } else {
                tracing::trace!("ignore message {:?} until sync.", msg);
                return Ok(());
            }
        }

        match msg {
            FeMessage::Gss => self.process_gss_msg().await?,
            FeMessage::Ssl => self.process_ssl_msg().await?,
            FeMessage::Startup(msg) => self.process_startup_msg(msg)?,
            FeMessage::Password(msg) => self.process_password_msg(msg).await?,
            FeMessage::Query(query_msg) => self.process_query_msg(query_msg.get_sql()).await?,
            FeMessage::CancelQuery(m) => self.process_cancel_msg(m)?,
            FeMessage::Terminate => self.process_terminate(),
            FeMessage::Parse(m) => {
                if let Err(err) = self.process_parse_msg(m).await {
                    self.ignore_util_sync = true;
                    return Err(err);
                }
            }
            FeMessage::Bind(m) => {
                if let Err(err) = self.process_bind_msg(m) {
                    self.ignore_util_sync = true;
                    return Err(err);
                }
            }
            FeMessage::Execute(m) => {
                if let Err(err) = self.process_execute_msg(m).await {
                    self.ignore_util_sync = true;
                    return Err(err);
                }
            }
            FeMessage::Describe(m) => {
                if let Err(err) = self.process_describe_msg(m) {
                    self.ignore_util_sync = true;
                    return Err(err);
                }
            }
            FeMessage::Sync => {
                self.ignore_util_sync = false;
                self.ready_for_query()?
            }
            FeMessage::Close(m) => {
                if let Err(err) = self.process_close_msg(m) {
                    self.ignore_util_sync = true;
                    return Err(err);
                }
            }
            FeMessage::Flush => {
                if let Err(err) = self.stream.flush().await {
                    self.ignore_util_sync = true;
                    return Err(err.into());
                }
            }
            FeMessage::HealthCheck => self.process_health_check(),
        }
        self.stream.flush().await?;
        Ok(())
    }

    pub async fn read_message(&mut self) -> io::Result<FeMessage> {
        match self.state {
            PgProtocolState::Startup => self.stream.read_startup().await,
            PgProtocolState::Regular => self.stream.read().await,
        }
    }

    /// Writes a `ReadyForQuery` message to the client without flushing.
    fn ready_for_query(&mut self) -> io::Result<()> {
        self.stream.write_no_flush(&BeMessage::ReadyForQuery(
            self.session
                .as_ref()
                .map(|s| s.transaction_status())
                .unwrap_or(TransactionStatus::Idle),
        ))
    }

    async fn process_gss_msg(&mut self) -> PsqlResult<()> {
        // We don't support GSSAPI, so we just say no gracefully.
        self.stream.write(&BeMessage::EncryptionResponseNo).await?;
        Ok(())
    }

    async fn process_ssl_msg(&mut self) -> PsqlResult<()> {
        if let Some(context) = self.tls_context.as_ref() {
            // If got and ssl context, say yes for ssl connection.
            // Construct ssl stream and replace with current one.
            self.stream.write(&BeMessage::EncryptionResponseSsl).await?;
            let ssl_stream = self.stream.ssl(context).await?;
            self.stream = Conn::Ssl(ssl_stream);
        } else {
            // If no, say no for encryption.
            self.stream.write(&BeMessage::EncryptionResponseNo).await?;
        }

        Ok(())
    }

    fn process_startup_msg(&mut self, msg: FeStartupMessage) -> PsqlResult<()> {
        let db_name = msg
            .config
            .get("database")
            .cloned()
            .unwrap_or_else(|| "dev".to_string());
        let user_name = msg
            .config
            .get("user")
            .cloned()
            .unwrap_or_else(|| "root".to_string());

        let session = self
            .session_mgr
            .connect(&db_name, &user_name, self.peer_addr.clone())
            .map_err(PsqlError::StartupError)?;

        let application_name = msg.config.get("application_name");
        if let Some(application_name) = application_name {
            session
                .set_config("application_name", application_name.clone())
                .map_err(PsqlError::StartupError)?;
        }

        match session.user_authenticator() {
            UserAuthenticator::None => {
                self.stream.write_no_flush(&BeMessage::AuthenticationOk)?;

                // Cancel request need this for identify and verification. According to postgres
                // doc, it should be written to buffer after receive AuthenticationOk.
                self.stream
                    .write_no_flush(&BeMessage::BackendKeyData(session.id()))?;

                self.stream
                    .write_parameter_status_msg_no_flush(&ParameterStatus {
                        application_name: application_name.cloned(),
                    })?;
                self.ready_for_query()?;
            }
            UserAuthenticator::ClearText(_) | UserAuthenticator::OAuth(_) => {
                self.stream
                    .write_no_flush(&BeMessage::AuthenticationCleartextPassword)?;
            }
            UserAuthenticator::Md5WithSalt { salt, .. } => {
                self.stream
                    .write_no_flush(&BeMessage::AuthenticationMd5Password(salt))?;
            }
        }

        self.session = Some(session);
        self.state = PgProtocolState::Regular;
        Ok(())
    }

    async fn process_password_msg(&mut self, msg: FePasswordMessage) -> PsqlResult<()> {
        let authenticator = self.session.as_ref().unwrap().user_authenticator();
        authenticator.authenticate(&msg.password).await?;
        self.stream.write_no_flush(&BeMessage::AuthenticationOk)?;
        self.stream
            .write_parameter_status_msg_no_flush(&ParameterStatus::default())?;
        self.ready_for_query()?;
        self.state = PgProtocolState::Regular;
        Ok(())
    }

    fn process_cancel_msg(&mut self, m: FeCancelMessage) -> PsqlResult<()> {
        let session_id = (m.target_process_id, m.target_secret_key);
        tracing::trace!("cancel query in session: {:?}", session_id);
        self.session_mgr.cancel_queries_in_session(session_id);
        self.session_mgr.cancel_creating_jobs_in_session(session_id);
        self.stream.write_no_flush(&BeMessage::EmptyQueryResponse)?;
        Ok(())
    }

    async fn process_query_msg(&mut self, query_string: io::Result<&str>) -> PsqlResult<()> {
        let sql: Arc<str> =
            Arc::from(query_string.map_err(|err| PsqlError::SimpleQueryError(Box::new(err)))?);
        record_sql_in_span(&sql, self.redact_sql_option_keywords.clone());
        let session = self.session.clone().unwrap();

        session.check_idle_in_transaction_timeout()?;
        let _exec_context_guard = session.init_exec_context(sql.clone());
        self.inner_process_query_msg(sql.clone(), session.clone())
            .await
    }

    async fn inner_process_query_msg(
        &mut self,
        sql: Arc<str>,
        session: Arc<SM::Session>,
    ) -> PsqlResult<()> {
        // Parse sql.
        let stmts =
            Parser::parse_sql(&sql).map_err(|err| PsqlError::SimpleQueryError(err.into()))?;
        if stmts.is_empty() {
            self.stream.write_no_flush(&BeMessage::EmptyQueryResponse)?;
        }

        // Execute multiple statements in simple query. KISS later.
        for stmt in stmts {
            self.inner_process_query_msg_one_stmt(stmt, session.clone())
                .await?;
        }
        // Put this line inside the for loop above will lead to unfinished/stuck regress test...Not
        // sure the reason.
        self.ready_for_query()?;
        Ok(())
    }

    async fn inner_process_query_msg_one_stmt(
        &mut self,
        stmt: Statement,
        session: Arc<SM::Session>,
    ) -> PsqlResult<()> {
        let session = session.clone();

        // execute query
        let res = session
            .clone()
            .run_one_query(stmt.clone(), Format::Text)
            .await;
        for notice in session.take_notices() {
            self.stream
                .write_no_flush(&BeMessage::NoticeResponse(&notice))?;
        }
        let mut res = res.map_err(PsqlError::SimpleQueryError)?;

        for notice in res.notices() {
            self.stream
                .write_no_flush(&BeMessage::NoticeResponse(notice))?;
        }

        let status = res.status();
        if let Some(ref application_name) = status.application_name {
            self.stream.write_no_flush(&BeMessage::ParameterStatus(
                BeParameterStatusMessage::ApplicationName(application_name),
            ))?;
        }

        if res.is_query() {
            self.stream
                .write_no_flush(&BeMessage::RowDescription(&res.row_desc()))?;

            let mut rows_cnt = 0;

            while let Some(row_set) = res.values_stream().next().await {
                let row_set = row_set.map_err(PsqlError::SimpleQueryError)?;
                for row in row_set {
                    self.stream.write_no_flush(&BeMessage::DataRow(&row))?;
                    rows_cnt += 1;
                }
            }

            // Run the callback before sending the `CommandComplete` message.
            res.run_callback().await?;

            self.stream
                .write_no_flush(&BeMessage::CommandComplete(BeCommandCompleteMessage {
                    stmt_type: res.stmt_type(),
                    rows_cnt,
                }))?;
        } else if res.stmt_type().is_dml() && !res.stmt_type().is_returning() {
            let first_row_set = res.values_stream().next().await;
            let first_row_set = match first_row_set {
                None => {
                    return Err(PsqlError::Uncategorized(
                        anyhow::anyhow!("no affected rows in output").into(),
                    ));
                }
                Some(row) => row.map_err(PsqlError::SimpleQueryError)?,
            };
            let affected_rows_str = first_row_set[0].values()[0]
                .as_ref()
                .expect("compute node should return affected rows in output");

            assert!(matches!(res.row_cnt_format(), Some(Format::Text)));
            let affected_rows_cnt = String::from_utf8(affected_rows_str.to_vec())
                .unwrap()
                .parse()
                .unwrap_or_default();

            // Run the callback before sending the `CommandComplete` message.
            res.run_callback().await?;

            self.stream
                .write_no_flush(&BeMessage::CommandComplete(BeCommandCompleteMessage {
                    stmt_type: res.stmt_type(),
                    rows_cnt: affected_rows_cnt,
                }))?;
        } else {
            // Run the callback before sending the `CommandComplete` message.
            res.run_callback().await?;

            self.stream
                .write_no_flush(&BeMessage::CommandComplete(BeCommandCompleteMessage {
                    stmt_type: res.stmt_type(),
                    rows_cnt: 0,
                }))?;
        }

        Ok(())
    }

    fn process_terminate(&mut self) {
        self.is_terminate = true;
    }

    fn process_health_check(&mut self) {
        tracing::debug!("health check");
        self.is_terminate = true;
    }

    async fn process_parse_msg(&mut self, msg: FeParseMessage) -> PsqlResult<()> {
        let sql = cstr_to_str(&msg.sql_bytes).unwrap();
        record_sql_in_span(sql, self.redact_sql_option_keywords.clone());
        let session = self.session.clone().unwrap();
        let statement_name = cstr_to_str(&msg.statement_name).unwrap().to_string();

        self.inner_process_parse_msg(session, sql, statement_name, msg.type_ids)
            .await
    }

    async fn inner_process_parse_msg(
        &mut self,
        session: Arc<SM::Session>,
        sql: &str,
        statement_name: String,
        type_ids: Vec<i32>,
    ) -> PsqlResult<()> {
        if statement_name.is_empty() {
            // Remove the unnamed prepare statement first, in case the unsupported sql binds a wrong
            // prepare statement.
            self.unnamed_prepare_statement.take();
        } else if self.prepare_statement_store.contains_key(&statement_name) {
            return Err(PsqlError::ExtendedPrepareError(
                "Duplicated statement name".into(),
            ));
        }

        let stmt = {
            let stmts = Parser::parse_sql(sql)
                .map_err(|err| PsqlError::ExtendedPrepareError(err.into()))?;

            if stmts.len() > 1 {
                return Err(PsqlError::ExtendedPrepareError(
                    "Only one statement is allowed in extended query mode".into(),
                ));
            }

            stmts.into_iter().next()
        };

        let param_types: Vec<Option<DataType>> = type_ids
            .iter()
            .map(|&id| {
                // 0 means unspecified type
                // ref: https://www.postgresql.org/docs/15/protocol-message-formats.html#:~:text=Placing%20a%20zero%20here%20is%20equivalent%20to%20leaving%20the%20type%20unspecified.
                if id == 0 {
                    Ok(None)
                } else {
                    DataType::from_oid(id)
                        .map(Some)
                        .map_err(|e| PsqlError::ExtendedPrepareError(e.into()))
                }
            })
            .try_collect()?;

        let prepare_statement = session
            .parse(stmt, param_types)
            .await
            .map_err(PsqlError::ExtendedPrepareError)?;

        if statement_name.is_empty() {
            self.unnamed_prepare_statement.replace(prepare_statement);
        } else {
            self.prepare_statement_store
                .insert(statement_name.clone(), prepare_statement);
        }

        self.statement_portal_dependency
            .entry(statement_name)
            .or_default()
            .clear();

        self.stream.write_no_flush(&BeMessage::ParseComplete)?;
        Ok(())
    }

    fn process_bind_msg(&mut self, msg: FeBindMessage) -> PsqlResult<()> {
        let statement_name = cstr_to_str(&msg.statement_name).unwrap().to_string();
        let portal_name = cstr_to_str(&msg.portal_name).unwrap().to_string();
        let session = self.session.clone().unwrap();

        if self.portal_store.contains_key(&portal_name) {
            return Err(PsqlError::Uncategorized("Duplicated portal name".into()));
        }

        let prepare_statement = self.get_statement(&statement_name)?;

        let result_formats = msg
            .result_format_codes
            .iter()
            .map(|&format_code| Format::from_i16(format_code))
            .try_collect()?;
        let param_formats = msg
            .param_format_codes
            .iter()
            .map(|&format_code| Format::from_i16(format_code))
            .try_collect()?;

        let portal = session
            .bind(prepare_statement, msg.params, param_formats, result_formats)
            .map_err(PsqlError::Uncategorized)?;

        if portal_name.is_empty() {
            self.result_cache.remove(&portal_name);
            self.unnamed_portal.replace(portal);
        } else {
            assert!(
                !self.result_cache.contains_key(&portal_name),
                "Named portal never can be overridden."
            );
            self.portal_store.insert(portal_name.clone(), portal);
        }

        self.statement_portal_dependency
            .get_mut(&statement_name)
            .unwrap()
            .push(portal_name);

        self.stream.write_no_flush(&BeMessage::BindComplete)?;
        Ok(())
    }

    async fn process_execute_msg(&mut self, msg: FeExecuteMessage) -> PsqlResult<()> {
        let portal_name = cstr_to_str(&msg.portal_name).unwrap().to_string();
        let row_max = msg.max_rows as usize;
        let session = self.session.clone().unwrap();

        if let Some(mut result_cache) = self.result_cache.remove(&portal_name) {
            assert!(self.portal_store.contains_key(&portal_name));

            let is_cosume_completed = result_cache.consume::<S>(row_max, &mut self.stream).await?;

            if !is_cosume_completed {
                self.result_cache.insert(portal_name, result_cache);
            }
        } else {
            let portal = self.get_portal(&portal_name)?;
            let sql: Arc<str> = Arc::from(format!("{}", portal));
            record_sql_in_span(&sql, self.redact_sql_option_keywords.clone());

            session.check_idle_in_transaction_timeout()?;
            let _exec_context_guard = session.init_exec_context(sql.clone());
            let result = session.clone().execute(portal).await;

            let pg_response = result.map_err(PsqlError::ExtendedExecuteError)?;
            let mut result_cache = ResultCache::new(pg_response);
            let is_consume_completed = result_cache.consume::<S>(row_max, &mut self.stream).await?;
            if !is_consume_completed {
                self.result_cache.insert(portal_name, result_cache);
            }
        }

        Ok(())
    }

    fn process_describe_msg(&mut self, msg: FeDescribeMessage) -> PsqlResult<()> {
        let name = cstr_to_str(&msg.name).unwrap().to_string();
        let session = self.session.clone().unwrap();
        //  b'S' => Statement
        //  b'P' => Portal

        assert!(msg.kind == b'S' || msg.kind == b'P');
        if msg.kind == b'S' {
            let prepare_statement = self.get_statement(&name)?;

            let (param_types, row_descriptions) = self
                .session
                .clone()
                .unwrap()
                .describe_statement(prepare_statement)
                .map_err(PsqlError::Uncategorized)?;
            self.stream
                .write_no_flush(&BeMessage::ParameterDescription(
                    &param_types.iter().map(|t| t.to_oid()).collect_vec(),
                ))?;

            if row_descriptions.is_empty() {
                // According https://www.postgresql.org/docs/current/protocol-flow.html#:~:text=The%20response%20is%20a%20RowDescri[…]0a%20query%20that%20will%20return%20rows%3B,
                // return NoData message if the statement is not a query.
                self.stream.write_no_flush(&BeMessage::NoData)?;
            } else {
                self.stream
                    .write_no_flush(&BeMessage::RowDescription(&row_descriptions))?;
            }
        } else if msg.kind == b'P' {
            let portal = self.get_portal(&name)?;

            let row_descriptions = session
                .describe_portal(portal)
                .map_err(PsqlError::Uncategorized)?;

            if row_descriptions.is_empty() {
                // According https://www.postgresql.org/docs/current/protocol-flow.html#:~:text=The%20response%20is%20a%20RowDescri[…]0a%20query%20that%20will%20return%20rows%3B,
                // return NoData message if the statement is not a query.
                self.stream.write_no_flush(&BeMessage::NoData)?;
            } else {
                self.stream
                    .write_no_flush(&BeMessage::RowDescription(&row_descriptions))?;
            }
        }
        Ok(())
    }

    fn process_close_msg(&mut self, msg: FeCloseMessage) -> PsqlResult<()> {
        let name = cstr_to_str(&msg.name).unwrap().to_string();
        assert!(msg.kind == b'S' || msg.kind == b'P');
        if msg.kind == b'S' {
            if name.is_empty() {
                self.unnamed_prepare_statement = None;
            } else {
                self.prepare_statement_store.remove(&name);
            }
            for portal_name in self
                .statement_portal_dependency
                .remove(&name)
                .unwrap_or_default()
            {
                self.remove_portal(&portal_name);
            }
        } else if msg.kind == b'P' {
            self.remove_portal(&name);
        }
        self.stream.write_no_flush(&BeMessage::CloseComplete)?;
        Ok(())
    }

    fn remove_portal(&mut self, portal_name: &str) {
        if portal_name.is_empty() {
            self.unnamed_portal = None;
        } else {
            self.portal_store.remove(portal_name);
        }
        self.result_cache.remove(portal_name);
    }

    fn get_portal(&self, portal_name: &str) -> PsqlResult<<SM::Session as Session>::Portal> {
        if portal_name.is_empty() {
            Ok(self
                .unnamed_portal
                .as_ref()
                .ok_or_else(|| PsqlError::Uncategorized("unnamed portal not found".into()))?
                .clone())
        } else {
            Ok(self
                .portal_store
                .get(portal_name)
                .ok_or_else(|| {
                    PsqlError::Uncategorized(format!("Portal {} not found", portal_name).into())
                })?
                .clone())
        }
    }

    fn get_statement(
        &self,
        statement_name: &str,
    ) -> PsqlResult<<SM::Session as Session>::PreparedStatement> {
        if statement_name.is_empty() {
            Ok(self
                .unnamed_prepare_statement
                .as_ref()
                .ok_or_else(|| {
                    PsqlError::Uncategorized("unnamed prepare statement not found".into())
                })?
                .clone())
        } else {
            Ok(self
                .prepare_statement_store
                .get(statement_name)
                .ok_or_else(|| {
                    PsqlError::Uncategorized(
                        format!("Prepare statement {} not found", statement_name).into(),
                    )
                })?
                .clone())
        }
    }
}

/// Wraps a byte stream and read/write pg messages.
pub struct PgStream<S> {
    /// The underlying stream.
    stream: Option<S>,
    /// Write into buffer before flush to stream.
    write_buf: BytesMut,
}

/// At present there is a hard-wired set of parameters for which
/// ParameterStatus will be generated: they are:
///
///  * `server_version`
///  * `server_encoding`
///  * `client_encoding`
///  * `application_name`
///  * `is_superuser`
///  * `session_authorization`
///  * `DateStyle`
///  * `IntervalStyle`
///  * `TimeZone`
///  * `integer_datetimes`
///  * `standard_conforming_string`
///
/// See: <https://www.postgresql.org/docs/9.2/static/protocol-flow.html#PROTOCOL-ASYNC>.
#[derive(Debug, Default, Clone)]
pub struct ParameterStatus {
    pub application_name: Option<String>,
}

impl<S> PgStream<S>
where
    S: AsyncWrite + AsyncRead + Unpin,
{
    async fn read_startup(&mut self) -> io::Result<FeMessage> {
        FeStartupMessage::read(self.stream()).await
    }

    async fn read(&mut self) -> io::Result<FeMessage> {
        FeMessage::read(self.stream()).await
    }

    fn write_parameter_status_msg_no_flush(&mut self, status: &ParameterStatus) -> io::Result<()> {
        self.write_no_flush(&BeMessage::ParameterStatus(
            BeParameterStatusMessage::ClientEncoding(SERVER_ENCODING),
        ))?;
        self.write_no_flush(&BeMessage::ParameterStatus(
            BeParameterStatusMessage::StandardConformingString(STANDARD_CONFORMING_STRINGS),
        ))?;
        self.write_no_flush(&BeMessage::ParameterStatus(
            BeParameterStatusMessage::ServerVersion(PG_VERSION),
        ))?;
        if let Some(application_name) = &status.application_name {
            self.write_no_flush(&BeMessage::ParameterStatus(
                BeParameterStatusMessage::ApplicationName(application_name),
            ))?;
        }
        Ok(())
    }

    pub fn write_no_flush(&mut self, message: &BeMessage<'_>) -> io::Result<()> {
        BeMessage::write(&mut self.write_buf, message)
    }

    async fn write(&mut self, message: &BeMessage<'_>) -> io::Result<()> {
        self.write_no_flush(message)?;
        self.flush().await?;
        Ok(())
    }

    async fn flush(&mut self) -> io::Result<()> {
        self.stream
            .as_mut()
            .unwrap()
            .write_all(&self.write_buf)
            .await?;
        self.write_buf.clear();
        self.stream.as_mut().unwrap().flush().await?;
        Ok(())
    }

    fn stream(&mut self) -> &mut (impl AsyncRead + Unpin + AsyncWrite) {
        self.stream.as_mut().unwrap()
    }
}

/// The logic of Conn is very simple, just a static dispatcher for TcpStream: Unencrypted or Ssl:
/// Encrypted.
pub enum Conn<S> {
    Unencrypted(PgStream<S>),
    Ssl(PgStream<SslStream<S>>),
}

impl<S> PgStream<S>
where
    S: AsyncWrite + AsyncRead + Unpin,
{
    async fn ssl(&mut self, ssl_ctx: &SslContextRef) -> PsqlResult<PgStream<SslStream<S>>> {
        // Note: Currently we take the ownership of previous Tcp Stream and then turn into a
        // SslStream. Later we can avoid storing stream inside PgProtocol to do this more
        // fluently.
        let stream = self.stream.take().unwrap();
        let ssl = openssl::ssl::Ssl::new(ssl_ctx).unwrap();
        let mut stream = tokio_openssl::SslStream::new(ssl, stream).unwrap();
        if let Err(e) = Pin::new(&mut stream).accept().await {
            tracing::warn!(error = %e.as_report(), "Unable to set up an ssl connection");
            let _ = stream.shutdown().await;
            return Err(e.into());
        }

        Ok(PgStream {
            stream: Some(stream),
            write_buf: BytesMut::with_capacity(10 * 1024),
        })
    }
}

impl<S> Conn<S>
where
    S: AsyncWrite + AsyncRead + Unpin,
{
    async fn read_startup(&mut self) -> io::Result<FeMessage> {
        match self {
            Conn::Unencrypted(s) => s.read_startup().await,
            Conn::Ssl(s) => s.read_startup().await,
        }
    }

    async fn read(&mut self) -> io::Result<FeMessage> {
        match self {
            Conn::Unencrypted(s) => s.read().await,
            Conn::Ssl(s) => s.read().await,
        }
    }

    fn write_parameter_status_msg_no_flush(&mut self, status: &ParameterStatus) -> io::Result<()> {
        match self {
            Conn::Unencrypted(s) => s.write_parameter_status_msg_no_flush(status),
            Conn::Ssl(s) => s.write_parameter_status_msg_no_flush(status),
        }
    }

    pub fn write_no_flush(&mut self, message: &BeMessage<'_>) -> io::Result<()> {
        match self {
            Conn::Unencrypted(s) => s.write_no_flush(message),
            Conn::Ssl(s) => s.write_no_flush(message),
        }
        .inspect_err(|error| tracing::error!(error = %error.as_report(), "flush error"))
    }

    async fn write(&mut self, message: &BeMessage<'_>) -> io::Result<()> {
        match self {
            Conn::Unencrypted(s) => s.write(message).await,
            Conn::Ssl(s) => s.write(message).await,
        }
    }

    async fn flush(&mut self) -> io::Result<()> {
        match self {
            Conn::Unencrypted(s) => s.flush().await,
            Conn::Ssl(s) => s.flush().await,
        }
        .inspect_err(|error| tracing::error!(error = %error.as_report(), "flush error"))
    }

    async fn ssl(&mut self, ssl_ctx: &SslContextRef) -> PsqlResult<PgStream<SslStream<S>>> {
        match self {
            Conn::Unencrypted(s) => s.ssl(ssl_ctx).await,
            Conn::Ssl(_s) => panic!("can not turn a ssl stream into a ssl stream"),
        }
    }
}

fn build_ssl_ctx_from_config(tls_config: &TlsConfig) -> PsqlResult<SslContext> {
    let mut acceptor = SslAcceptor::mozilla_intermediate_v5(SslMethod::tls()).unwrap();

    let key_path = &tls_config.key;
    let cert_path = &tls_config.cert;

    // Build ssl acceptor according to the config.
    // Now we set every verify to true.
    acceptor
        .set_private_key_file(key_path, openssl::ssl::SslFiletype::PEM)
        .map_err(|e| PsqlError::Uncategorized(e.into()))?;
    acceptor
        .set_ca_file(cert_path)
        .map_err(|e| PsqlError::Uncategorized(e.into()))?;
    acceptor
        .set_certificate_chain_file(cert_path)
        .map_err(|e| PsqlError::Uncategorized(e.into()))?;
    let acceptor = acceptor.build();

    Ok(acceptor.into_context())
}

pub mod truncated_fmt {
    use std::fmt::*;

    struct TruncatedFormatter<'a, 'b> {
        remaining: usize,
        finished: bool,
        f: &'a mut Formatter<'b>,
    }
    impl Write for TruncatedFormatter<'_, '_> {
        fn write_str(&mut self, s: &str) -> Result {
            if self.finished {
                return Ok(());
            }

            if self.remaining < s.len() {
                let actual = s.floor_char_boundary(self.remaining);
                self.f.write_str(&s[0..actual])?;
                self.remaining -= actual;
                self.f.write_str("...(truncated)")?;
                self.finished = true; // so that ...(truncated) is printed exactly once
            } else {
                self.f.write_str(s)?;
                self.remaining -= s.len();
            }
            Ok(())
        }
    }

    pub struct TruncatedFmt<'a, T>(pub &'a T, pub usize);

    impl<T> Debug for TruncatedFmt<'_, T>
    where
        T: Debug,
    {
        fn fmt(&self, f: &mut Formatter<'_>) -> Result {
            TruncatedFormatter {
                remaining: self.1,
                finished: false,
                f,
            }
            .write_fmt(format_args!("{:?}", self.0))
        }
    }

    impl<T> Display for TruncatedFmt<'_, T>
    where
        T: Display,
    {
        fn fmt(&self, f: &mut Formatter<'_>) -> Result {
            TruncatedFormatter {
                remaining: self.1,
                finished: false,
                f,
            }
            .write_fmt(format_args!("{}", self.0))
        }
    }

    #[cfg(test)]
    mod tests {
        use super::*;

        #[test]
        fn test_trunc_utf8() {
            assert_eq!(
                format!("{}", TruncatedFmt(&"select '🌊';", 10)),
                "select '...(truncated)",
            );
        }
    }
}

#[cfg(test)]
mod tests {
    use std::collections::HashSet;

    use super::*;

    #[test]
    fn test_redact_parsable_sql() {
        let keywords = Arc::new(HashSet::from(["v2".into(), "v4".into(), "b".into()]));
        let sql = r"
        create source temp (k bigint, v varchar) with (
            connector = 'datagen',
            v1 = 123,
            v2 = 'with',
            v3 = false,
            v4 = '',
        ) FORMAT plain ENCODE json (a='1',b='2')
        ";
        assert_eq!(redact_sql(sql, keywords), "CREATE SOURCE temp (k BIGINT, v CHARACTER VARYING) WITH (connector = 'datagen', v1 = 123, v2 = [REDACTED], v3 = false, v4 = [REDACTED]) FORMAT PLAIN ENCODE JSON (a = '1', b = [REDACTED])");
    }
}