-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
server.go
1375 lines (1245 loc) · 49.2 KB
/
server.go
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
// Copyright 2015 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package pgwire
import (
"bufio"
"context"
"io"
"net"
"sync/atomic"
"time"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/clusterunique"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/identmap"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirecancel"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/ctxlog"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/log/logpb"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
"go.opentelemetry.io/otel/attribute"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
// ATTENTION: After changing this value in a unit test, you probably want to
// open a new connection pool since the connections in the existing one are not
// affected.
//
// The "results_buffer_size" connection parameter can be used to override this
// default for an individual connection.
var connResultsBufferSize = settings.RegisterByteSizeSetting(
settings.TenantWritable,
"sql.defaults.results_buffer.size",
"default size of the buffer that accumulates results for a statement or a batch "+
"of statements before they are sent to the client. This can be overridden on "+
"an individual connection with the 'results_buffer_size' parameter. Note that auto-retries "+
"generally only happen while no results have been delivered to the client, so "+
"reducing this size can increase the number of retriable errors a client "+
"receives. On the other hand, increasing the buffer size can increase the "+
"delay until the client receives the first result row. "+
"Updating the setting only affects new connections. "+
"Setting to 0 disables any buffering.",
16<<10, // 16 KiB
).WithPublic()
var logConnAuth = settings.RegisterBoolSetting(
settings.TenantWritable,
sql.ConnAuditingClusterSettingName,
"if set, log SQL client connect and disconnect events (note: may hinder performance on loaded nodes)",
false).WithPublic()
var logSessionAuth = settings.RegisterBoolSetting(
settings.TenantWritable,
sql.AuthAuditingClusterSettingName,
"if set, log SQL session login/disconnection events (note: may hinder performance on loaded nodes)",
false).WithPublic()
const (
// ErrSSLRequired is returned when a client attempts to connect to a
// secure server in cleartext.
ErrSSLRequired = "node is running secure mode, SSL connection required"
// ErrDrainingNewConn is returned when a client attempts to connect to a server
// which is not accepting client connections.
ErrDrainingNewConn = "server is not accepting clients, try another node"
// ErrDrainingExistingConn is returned when a connection is shut down because
// the server is draining.
ErrDrainingExistingConn = "server is shutting down"
)
// Fully-qualified names for metrics.
var (
MetaConns = metric.Metadata{
Name: "sql.conns",
Help: "Number of open SQL connections",
Measurement: "Connections",
Unit: metric.Unit_COUNT,
}
MetaNewConns = metric.Metadata{
Name: "sql.new_conns",
Help: "Counter of the number of SQL connections created",
Measurement: "Connections",
Unit: metric.Unit_COUNT,
}
MetaConnsWaitingToHash = metric.Metadata{
Name: "sql.conns_waiting_to_hash",
Help: "Number of SQL connection attempts that are being throttled in order to limit password hashing concurrency",
Measurement: "Connections",
Unit: metric.Unit_COUNT,
}
MetaBytesIn = metric.Metadata{
Name: "sql.bytesin",
Help: "Number of SQL bytes received",
Measurement: "SQL Bytes",
Unit: metric.Unit_BYTES,
}
MetaBytesOut = metric.Metadata{
Name: "sql.bytesout",
Help: "Number of SQL bytes sent",
Measurement: "SQL Bytes",
Unit: metric.Unit_BYTES,
}
MetaConnLatency = metric.Metadata{
Name: "sql.conn.latency",
Help: "Latency to establish and authenticate a SQL connection",
Measurement: "Nanoseconds",
Unit: metric.Unit_NANOSECONDS,
}
MetaConnFailures = metric.Metadata{
Name: "sql.conn.failures",
Help: "Number of SQL connection failures",
Measurement: "Connections",
Unit: metric.Unit_COUNT,
}
MetaPGWireCancelTotal = metric.Metadata{
Name: "sql.pgwire_cancel.total",
Help: "Counter of the number of pgwire query cancel requests",
Measurement: "Requests",
Unit: metric.Unit_COUNT,
}
MetaPGWireCancelIgnored = metric.Metadata{
Name: "sql.pgwire_cancel.ignored",
Help: "Counter of the number of pgwire query cancel requests that were ignored due to rate limiting",
Measurement: "Requests",
Unit: metric.Unit_COUNT,
}
MetaPGWireCancelSuccessful = metric.Metadata{
Name: "sql.pgwire_cancel.successful",
Help: "Counter of the number of pgwire query cancel requests that were successful",
Measurement: "Requests",
Unit: metric.Unit_COUNT,
}
)
const (
// The below constants can occur during the first message a client
// sends to the server. There are two categories: protocol version and
// request code. The protocol version is (major version number << 16)
// + minor version number. Request codes are (1234 << 16) + 5678 + N,
// where N started at 0 and is increased by 1 for every new request
// code added, which happens rarely during major or minor Postgres
// releases.
//
// See: https://www.postgresql.org/docs/current/protocol-message-formats.html
version30 = 196608 // (3 << 16) + 0
versionCancel = 80877102 // (1234 << 16) + 5678
versionSSL = 80877103 // (1234 << 16) + 5679
versionGSSENC = 80877104 // (1234 << 16) + 5680
)
// cancelMaxWait is the amount of time a draining server gives to sessions to
// react to cancellation and return before a forceful shutdown.
const cancelMaxWait = 1 * time.Second
// baseSQLMemoryBudget is the amount of memory pre-allocated in each connection.
var baseSQLMemoryBudget = envutil.EnvOrDefaultInt64("COCKROACH_BASE_SQL_MEMORY_BUDGET",
int64(2.1*float64(mon.DefaultPoolAllocationSize)))
// connReservationBatchSize determines for how many connections memory
// is pre-reserved at once.
var connReservationBatchSize = 5
var (
sslSupported = []byte{'S'}
sslUnsupported = []byte{'N'}
)
// cancelChanMap keeps track of channels that are closed after the associated
// cancellation function has been called and the cancellation has taken place.
type cancelChanMap map[chan struct{}]context.CancelFunc
// Server implements the server side of the PostgreSQL wire protocol for one
// specific tenant (i.e. its configuration is specific to one tenant).
type Server struct {
AmbientCtx log.AmbientContext
cfg *base.Config
SQLServer *sql.Server
execCfg *sql.ExecutorConfig
tenantMetrics *tenantSpecificMetrics
mu struct {
syncutil.Mutex
// connCancelMap entries represent connections started when the server
// was not draining. Each value is a function that can be called to
// cancel the associated connection. The corresponding key is a channel
// that is closed when the connection is done.
connCancelMap cancelChanMap
// draining is set to true when the server starts draining the SQL layer.
// When set to true, remaining SQL connections will be closed.
// After the timeout set by server.shutdown.query_wait,
// all connections will be closed regardless any queries in flight.
draining bool
// rejectNewConnections is set true when the server does not accept new
// SQL connections, e.g. when the draining process enters the phase whose
// duration is specified by the server.shutdown.connection_wait.
rejectNewConnections bool
}
auth struct {
syncutil.RWMutex
conf *hba.Conf
identityMap *identmap.Conf
}
// sqlMemoryPool is the parent memory pool for all SQL memory allocations
// for this tenant, including SQL query execution, etc.
sqlMemoryPool *mon.BytesMonitor
// tenantSpecificConnMonitor is the pool where the memory usage for the
// initial connection overhead is accounted for.
tenantSpecificConnMonitor *mon.BytesMonitor
// testing{Conn,Auth}LogEnabled is used in unit tests in this
// package to force-enable conn/auth logging without dancing around
// the asynchronicity of cluster settings.
testingConnLogEnabled syncutil.AtomicBool
testingAuthLogEnabled syncutil.AtomicBool
}
// tenantSpecificMetrics is the set of metrics for a pgwire server
// bound to a specific tenant.
type tenantSpecificMetrics struct {
BytesInCount *metric.Counter
BytesOutCount *metric.Counter
Conns *metric.Gauge
NewConns *metric.Counter
ConnsWaitingToHash *metric.Gauge
ConnLatency metric.IHistogram
ConnFailures *metric.Counter
PGWireCancelTotalCount *metric.Counter
PGWireCancelIgnoredCount *metric.Counter
PGWireCancelSuccessfulCount *metric.Counter
ConnMemMetrics sql.BaseMemoryMetrics
SQLMemMetrics sql.MemoryMetrics
}
func newTenantSpecificMetrics(
sqlMemMetrics sql.MemoryMetrics, histogramWindow time.Duration,
) *tenantSpecificMetrics {
return &tenantSpecificMetrics{
BytesInCount: metric.NewCounter(MetaBytesIn),
BytesOutCount: metric.NewCounter(MetaBytesOut),
Conns: metric.NewGauge(MetaConns),
NewConns: metric.NewCounter(MetaNewConns),
ConnsWaitingToHash: metric.NewGauge(MetaConnsWaitingToHash),
ConnLatency: metric.NewHistogram(metric.HistogramOptions{
Mode: metric.HistogramModePreferHdrLatency,
Metadata: MetaConnLatency,
Duration: histogramWindow,
Buckets: metric.IOLatencyBuckets,
}),
ConnFailures: metric.NewCounter(MetaConnFailures),
PGWireCancelTotalCount: metric.NewCounter(MetaPGWireCancelTotal),
PGWireCancelIgnoredCount: metric.NewCounter(MetaPGWireCancelIgnored),
PGWireCancelSuccessfulCount: metric.NewCounter(MetaPGWireCancelSuccessful),
ConnMemMetrics: sql.MakeBaseMemMetrics("conns", histogramWindow),
SQLMemMetrics: sqlMemMetrics,
}
}
// noteworthySQLMemoryUsageBytes is the minimum size tracked by the
// client SQL pool before the pool start explicitly logging overall
// usage growth in the log.
var noteworthySQLMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NOTEWORTHY_SQL_MEMORY_USAGE", 100*1024*1024)
// noteworthyConnMemoryUsageBytes is the minimum size tracked by the
// connection monitor before the monitor start explicitly logging overall
// usage growth in the log.
var noteworthyConnMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NOTEWORTHY_CONN_MEMORY_USAGE", 2*1024*1024)
// MakeServer creates a Server.
//
// Start() needs to be called on the Server so it begins processing.
func MakeServer(
ambientCtx log.AmbientContext,
cfg *base.Config,
st *cluster.Settings,
sqlMemMetrics sql.MemoryMetrics,
parentMemoryMonitor *mon.BytesMonitor,
histogramWindow time.Duration,
executorConfig *sql.ExecutorConfig,
) *Server {
ctx := ambientCtx.AnnotateCtx(context.Background())
server := &Server{
AmbientCtx: ambientCtx,
cfg: cfg,
execCfg: executorConfig,
tenantMetrics: newTenantSpecificMetrics(sqlMemMetrics, histogramWindow),
}
server.sqlMemoryPool = mon.NewMonitor("sql",
mon.MemoryResource,
// Note that we don't report metrics on this monitor. The reason for this is
// that we report metrics on the sum of all the child monitors of this pool.
// This monitor is the "main sql" monitor. It's a child of the root memory
// monitor. Its children are the sql monitors for each new connection. The
// sum of those children, plus the extra memory in the "conn" monitor below,
// is more than enough metrics information about the monitors.
nil, /* curCount */
nil, /* maxHist */
0, noteworthySQLMemoryUsageBytes, st)
server.sqlMemoryPool.StartNoReserved(ctx, parentMemoryMonitor)
server.SQLServer = sql.NewServer(executorConfig, server.sqlMemoryPool)
server.tenantSpecificConnMonitor = mon.NewMonitor("conn",
mon.MemoryResource,
server.tenantMetrics.ConnMemMetrics.CurBytesCount,
server.tenantMetrics.ConnMemMetrics.MaxBytesHist,
int64(connReservationBatchSize)*baseSQLMemoryBudget, noteworthyConnMemoryUsageBytes, st)
server.tenantSpecificConnMonitor.StartNoReserved(ctx, server.sqlMemoryPool)
server.mu.Lock()
server.mu.connCancelMap = make(cancelChanMap)
server.mu.Unlock()
connAuthConf.SetOnChange(&st.SV, func(ctx context.Context) {
loadLocalHBAConfigUponRemoteSettingChange(ctx, server, st)
})
ConnIdentityMapConf.SetOnChange(&st.SV, func(ctx context.Context) {
loadLocalIdentityMapUponRemoteSettingChange(ctx, server, st)
})
return server
}
// BytesOut returns the total number of bytes transmitted from this server.
func (s *Server) BytesOut() uint64 {
return uint64(s.tenantMetrics.BytesOutCount.Count())
}
// Match returns true if rd appears to be a Postgres connection.
func Match(rd io.Reader) bool {
buf := pgwirebase.MakeReadBuffer()
_, err := buf.ReadUntypedMsg(rd)
if err != nil {
return false
}
version, err := buf.GetUint32()
if err != nil {
return false
}
return version == version30 || version == versionSSL || version == versionCancel || version == versionGSSENC
}
// Start makes the Server ready for serving connections.
func (s *Server) Start(ctx context.Context, stopper *stop.Stopper) {
s.SQLServer.Start(ctx, stopper)
}
// IsDraining returns true if the server is not currently accepting
// connections.
func (s *Server) IsDraining() bool {
s.mu.Lock()
defer s.mu.Unlock()
return s.mu.draining
}
// Metrics returns the set of metrics structs.
func (s *Server) Metrics() []interface{} {
return []interface{}{
s.tenantMetrics,
&s.SQLServer.Metrics.StartedStatementCounters,
&s.SQLServer.Metrics.ExecutedStatementCounters,
&s.SQLServer.Metrics.EngineMetrics,
&s.SQLServer.Metrics.GuardrailMetrics,
&s.SQLServer.InternalMetrics.StartedStatementCounters,
&s.SQLServer.InternalMetrics.ExecutedStatementCounters,
&s.SQLServer.InternalMetrics.EngineMetrics,
&s.SQLServer.InternalMetrics.GuardrailMetrics,
&s.SQLServer.ServerMetrics.StatsMetrics,
&s.SQLServer.ServerMetrics.ContentionSubsystemMetrics,
&s.SQLServer.ServerMetrics.InsightsMetrics,
}
}
// Drain prevents new connections from being served and waits the duration of
// queryWait for open connections to terminate before canceling them.
// An error will be returned when connections that have been canceled have not
// responded to this cancellation and closed themselves in time. The server
// will remain in draining state, though open connections may continue to
// exist.
// The RFC on drain modes has more information regarding the specifics of
// what will happen to connections in different states:
// https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20160425_drain_modes.md
//
// The reporter callback, if non-nil, is called on a best effort basis
// to report work that needed to be done and which may or may not have
// been done by the time this call returns. See the explanation in
// pkg/server/drain.go for details.
func (s *Server) Drain(
ctx context.Context,
queryWait time.Duration,
reporter func(int, redact.SafeString),
stopper *stop.Stopper,
) error {
return s.drainImpl(ctx, queryWait, cancelMaxWait, reporter, stopper)
}
// Undrain switches the server back to the normal mode of operation in which
// connections are accepted.
func (s *Server) Undrain() {
s.mu.Lock()
defer s.mu.Unlock()
s.setRejectNewConnectionsLocked(false)
s.setDrainingLocked(false)
}
// setDrainingLocked sets the server's draining state and returns whether the
// state changed (i.e. drain != s.mu.draining). s.mu must be locked when
// setDrainingLocked is called.
func (s *Server) setDrainingLocked(drain bool) bool {
if s.mu.draining == drain {
return false
}
s.mu.draining = drain
return true
}
// setRejectNewConnectionsLocked sets the server's rejectNewConnections state.
// s.mu must be locked when setRejectNewConnectionsLocked is called.
func (s *Server) setRejectNewConnectionsLocked(rej bool) {
s.mu.rejectNewConnections = rej
}
// GetConnCancelMapLen returns the length of connCancelMap of the server.
// This is a helper function when the server waits the SQL connections to be
// closed. During this period, the server listens to the status of all
// connections, and early exits this draining phase if there remains no active
// SQL connections.
func (s *Server) GetConnCancelMapLen() int {
s.mu.Lock()
defer s.mu.Unlock()
return len(s.mu.connCancelMap)
}
// WaitForSQLConnsToClose waits for the client to close all SQL connections for the
// duration of connectionWait.
// With this phase, the node starts rejecting SQL connections, and as
// soon as all existing SQL connections are closed, the server early exits this
// draining phase.
func (s *Server) WaitForSQLConnsToClose(
ctx context.Context, connectionWait time.Duration, stopper *stop.Stopper,
) error {
// If we're already draining the SQL connections, we don't need to wait again.
if s.IsDraining() {
return nil
}
s.mu.Lock()
s.setRejectNewConnectionsLocked(true)
s.mu.Unlock()
if connectionWait == 0 {
return nil
}
log.Ops.Info(ctx, "waiting for clients to close existing SQL connections")
timer := time.NewTimer(connectionWait)
defer timer.Stop()
_, allConnsDone, quitWaitingForConns := s.waitConnsDone()
defer close(quitWaitingForConns)
select {
// Connection wait times out.
case <-time.After(connectionWait):
log.Ops.Warningf(ctx,
"%d connections remain after waiting %s; proceeding to drain SQL connections",
s.GetConnCancelMapLen(),
connectionWait,
)
case <-allConnsDone:
case <-ctx.Done():
return ctx.Err()
case <-stopper.ShouldQuiesce():
return context.Canceled
}
return nil
}
// waitConnsDone returns a copy of s.mu.connCancelMap, and a channel that
// will be closed once all sql connections are closed, or the server quits
// waiting for connections, whichever earlier.
func (s *Server) waitConnsDone() (cancelChanMap, chan struct{}, chan struct{}) {
connCancelMap := func() cancelChanMap {
s.mu.Lock()
defer s.mu.Unlock()
connCancelMap := make(cancelChanMap, len(s.mu.connCancelMap))
for done, cancel := range s.mu.connCancelMap {
connCancelMap[done] = cancel
}
return connCancelMap
}()
allConnsDone := make(chan struct{}, 1)
quitWaitingForConns := make(chan struct{}, 1)
go func() {
defer close(allConnsDone)
for done := range connCancelMap {
select {
case <-done:
case <-quitWaitingForConns:
return
}
}
}()
return connCancelMap, allConnsDone, quitWaitingForConns
}
// drainImpl drains the SQL clients.
//
// The queryWait duration is used to wait on clients to
// self-disconnect after their session has been canceled. The
// cancelWait is used to wait after the queryWait timer has expired
// and there are still clients connected, and their context.Context is
// canceled.
//
// The reporter callback, if non-nil, is called on a best effort basis
// to report work that needed to be done and which may or may not have
// been done by the time this call returns. See the explanation in
// pkg/server/drain.go for details.
func (s *Server) drainImpl(
ctx context.Context,
queryWait time.Duration,
cancelWait time.Duration,
reporter func(int, redact.SafeString),
stopper *stop.Stopper,
) error {
s.mu.Lock()
if !s.setDrainingLocked(true) {
// We are already draining.
s.mu.Unlock()
return nil
}
s.mu.Unlock()
// If there is no open SQL connections to drain, just return.
if s.GetConnCancelMapLen() == 0 {
return nil
}
log.Ops.Info(ctx, "starting draining SQL connections")
// Spin off a goroutine that waits for all connections to signal that they
// are done and reports it on allConnsDone. The main goroutine signals this
// goroutine to stop work through quitWaitingForConns.
// This s.waitConnsDone function returns a copy of s.mu.connCancelMap if there
// are any active connections to cancel. We will only attempt to cancel
// connections that were active at the moment the draining switch happened.
// It is enough to do this because:
// 1) If no new connections are added to the original map all connections
// will be canceled.
// 2) If new connections are added to the original map, it follows that they
// were added when s.mu.draining = false, thus not requiring cancellation.
// These connections are not our responsibility and will be handled when the
// server starts draining again.
connCancelMap, allConnsDone, quitWaitingForConns := s.waitConnsDone()
defer close(quitWaitingForConns)
if reporter != nil {
// Report progress to the Drain RPC.
reporter(len(connCancelMap), "SQL clients")
}
// Wait for connections to finish up their queries for the duration of queryWait.
select {
case <-time.After(queryWait):
log.Ops.Warningf(ctx, "canceling all sessions after waiting %s", queryWait)
case <-allConnsDone:
case <-ctx.Done():
return ctx.Err()
case <-stopper.ShouldQuiesce():
return context.Canceled
}
// Cancel the contexts of all sessions if the server is still in draining
// mode.
if stop := func() bool {
s.mu.Lock()
defer s.mu.Unlock()
if !s.mu.draining {
return true
}
for _, cancel := range connCancelMap {
// There is a possibility that different calls to SetDraining have
// overlapping connCancelMaps, but context.CancelFunc calls are
// idempotent.
cancel()
}
return false
}(); stop {
return nil
}
select {
case <-time.After(cancelWait):
return errors.Errorf("some sessions did not respond to cancellation within %s", cancelWait)
case <-allConnsDone:
}
return nil
}
// SocketType indicates the connection type. This is an optimization to
// prevent a comparison against conn.LocalAddr().Network().
type SocketType int
const (
SocketUnknown SocketType = iota
// SocketTCP is used for TCP sockets. The standard.
SocketTCP
// SocketUnix is used for unix datagram sockets.
SocketUnix
// SocketInternalLoopback is used for internal connections running over our
// loopback listener.
SocketInternalLoopback
)
func (s SocketType) asConnType() (hba.ConnType, error) {
switch s {
case SocketTCP:
return hba.ConnHostNoSSL, nil
case SocketUnix:
return hba.ConnLocal, nil
case SocketInternalLoopback:
return hba.ConnInternalLoopback, nil
default:
return 0, errors.AssertionFailedf("unimplemented socket type: %v", errors.Safe(s))
}
}
func (s *Server) connLogEnabled() bool {
return s.testingConnLogEnabled.Get() || logConnAuth.Get(&s.execCfg.Settings.SV)
}
// TestingEnableConnLogging is exported for use in tests.
func (s *Server) TestingEnableConnLogging() {
s.testingConnLogEnabled.Set(true)
}
// TestingEnableAuthLogging is exported for use in tests.
func (s *Server) TestingEnableAuthLogging() {
s.testingAuthLogEnabled.Set(true)
}
// ServeConn serves a single connection, driving the handshake process and
// delegating to the appropriate connection type.
//
// The socketType argument is an optimization to avoid a string
// compare on conn.LocalAddr().Network(). When the socket type is
// unix datagram (local filesystem), SSL negotiation is disabled
// even when the server is running securely with certificates.
// This has the effect of forcing password auth, also in a way
// compatible with postgres.
//
// An error is returned if the initial handshake of the connection fails.
func (s *Server) ServeConn(
ctx context.Context, conn net.Conn, preServeStatus PreServeStatus,
) (err error) {
if preServeStatus.State != PreServeReady {
return errors.AssertionFailedf("programming error: cannot call ServeConn with state %v (did you mean HandleCancel?)", preServeStatus.State)
}
defer func() {
if err != nil {
s.tenantMetrics.ConnFailures.Inc(1)
}
}()
ctx, rejectNewConnections, onCloseFn := s.registerConn(ctx)
defer onCloseFn()
sessionID := s.execCfg.GenerateID()
connDetails := eventpb.CommonConnectionDetails{
InstanceID: int32(s.execCfg.NodeInfo.NodeID.SQLInstanceID()),
Network: conn.RemoteAddr().Network(),
RemoteAddress: conn.RemoteAddr().String(),
SessionID: sessionID.String(),
}
// Some bookkeeping, for security-minded administrators.
// This registers the connection to the authentication log.
connStart := timeutil.Now()
if s.connLogEnabled() {
ev := &eventpb.ClientConnectionStart{
CommonEventDetails: logpb.CommonEventDetails{Timestamp: connStart.UnixNano()},
CommonConnectionDetails: connDetails,
}
log.StructuredEvent(ctx, ev)
}
defer func() {
// The duration of the session is logged at the end so that the
// reader of the log file can know how much to look back in time
// to find when the connection was opened. This is important
// because the log files may have been rotated since.
if s.connLogEnabled() {
endTime := timeutil.Now()
ev := &eventpb.ClientConnectionEnd{
CommonEventDetails: logpb.CommonEventDetails{Timestamp: endTime.UnixNano()},
CommonConnectionDetails: connDetails,
Duration: endTime.Sub(connStart).Nanoseconds(),
}
log.StructuredEvent(ctx, ev)
}
}()
st := s.execCfg.Settings
// If the server is shutting down, terminate the connection early.
if rejectNewConnections {
log.Ops.Info(ctx, "rejecting new connection while server is draining")
return s.sendErr(ctx, st, conn, newAdminShutdownErr(ErrDrainingNewConn))
}
sArgs, err := finalizeClientParameters(ctx, preServeStatus.clientParameters, &st.SV)
if err != nil {
preServeStatus.Reserved.Close(ctx)
return s.sendErr(ctx, st, conn, err)
}
// Transfer the memory account into this tenant.
tenantReserved, err := s.tenantSpecificConnMonitor.TransferAccount(ctx, &preServeStatus.Reserved)
if err != nil {
preServeStatus.Reserved.Close(ctx)
return s.sendErr(ctx, st, conn, err)
}
// Populate the client address field in the context tags and the
// shared struct for structured logging.
// Only now do we know the remote client address for sure (it may have
// been overridden by a status parameter).
connDetails.RemoteAddress = sArgs.RemoteAddr.String()
sp := tracing.SpanFromContext(ctx)
ctx = logtags.AddTag(ctx, "client", log.SafeOperational(connDetails.RemoteAddress))
ctx = logtags.AddTag(ctx, preServeStatus.ConnType.String(), nil)
sp.SetTag("conn_type", attribute.StringValue(preServeStatus.ConnType.String()))
sp.SetTag("client", attribute.StringValue(connDetails.RemoteAddress))
// If a test is hooking in some authentication option, load it.
var testingAuthHook func(context.Context) error
if k := s.execCfg.PGWireTestingKnobs; k != nil {
testingAuthHook = k.AuthHook
}
hbaConf, identMap := s.GetAuthenticationConfiguration()
// Defer the rest of the processing to the connection handler.
// This includes authentication.
if log.V(2) {
log.Infof(ctx, "new connection with options: %+v", sArgs)
}
ctx, cancelConn := context.WithCancel(ctx)
defer cancelConn()
c := s.newConn(
ctx,
cancelConn,
conn,
sArgs,
connStart,
)
// Do the reading of commands from the network.
s.serveImpl(
ctx,
c,
&tenantReserved,
authOptions{
connType: preServeStatus.ConnType,
connDetails: connDetails,
insecure: s.cfg.Insecure,
auth: hbaConf,
identMap: identMap,
testingAuthHook: testingAuthHook,
},
sessionID,
)
return nil
}
func (s *Server) newConn(
ctx context.Context,
cancelConn context.CancelFunc,
netConn net.Conn,
sArgs sql.SessionArgs,
connStart time.Time,
) *conn {
// The net.Conn is switched to a conn that exits if the ctx is canceled.
rtc := &readTimeoutConn{
Conn: netConn,
}
sv := &s.execCfg.Settings.SV
c := &conn{
conn: rtc,
cancelConn: cancelConn,
sessionArgs: sArgs,
metrics: s.tenantMetrics,
startTime: connStart,
rd: *bufio.NewReader(rtc),
sv: sv,
readBuf: pgwirebase.MakeReadBuffer(pgwirebase.ReadBufferOptionWithClusterSettings(sv)),
alwaysLogAuthActivity: s.testingAuthLogEnabled.Get(),
}
c.stmtBuf.Init()
c.res.released = true
c.writerState.fi.buf = &c.writerState.buf
c.writerState.fi.lastFlushed = -1
c.msgBuilder.init(s.tenantMetrics.BytesOutCount)
c.errWriter.sv = sv
c.errWriter.msgBuilder = &c.msgBuilder
var sentDrainSignal bool
rtc.checkExitConds = func() error {
// If the context was canceled, it's time to stop reading. Either a
// higher-level server or the command processor have canceled us.
if err := ctx.Err(); err != nil {
return err
}
// If the server is draining, we'll let the processor know by pushing a
// DrainRequest. This will make the processor quit whenever it finds a good
// time.
if !sentDrainSignal && s.IsDraining() {
_ /* err */ = c.stmtBuf.Push(ctx, sql.DrainRequest{})
sentDrainSignal = true
}
return nil
}
return c
}
// maxRepeatedErrorCount is the number of times an error can be received
// while reading from the network connection before the server decides to give
// up and abort the connection.
const maxRepeatedErrorCount = 1 << 15
// serveImpl continuously reads from the network connection and pushes execution
// instructions into a sql.StmtBuf, from where they'll be processed by a command
// "processor" goroutine (a connExecutor).
// The method returns when the pgwire termination message is received, when
// network communication fails, when the server is draining or when ctx is
// canceled (which also happens when draining (but not from the get-go), and
// when the processor encounters a fatal error).
//
// serveImpl always closes the network connection before returning.
//
// sqlServer is used to create the command processor. As a special facility for
// tests, sqlServer can be nil, in which case the command processor and the
// write-side of the connection will not be created.
//
// Internally, a connExecutor will be created to execute commands. Commands read
// from the network are buffered in a stmtBuf which is consumed by the
// connExecutor. The connExecutor produces results which are buffered and
// sometimes synchronously flushed to the network.
//
// The reader goroutine (this one) outlives the connExecutor's goroutine (the
// "processor goroutine").
// However, they can both signal each other to stop. Here's how the different
// cases work:
// 1) The reader receives a ClientMsgTerminate protocol packet: the reader
// closes the stmtBuf and also cancels the command processing context. These
// actions will prompt the command processor to finish.
// 2) The reader gets a read error from the network connection: like above, the
// reader closes the command processor.
// 3) The reader's context is canceled (happens when the server is draining but
// the connection was busy and hasn't quit yet): the reader notices the canceled
// context and, like above, closes the processor.
// 4) The processor encounters an error. This error can come from various fatal
// conditions encountered internally by the processor, or from a network
// communication error encountered while flushing results to the network.
// The processor will cancel the reader's context and terminate.
// Note that query processing errors are different; they don't cause the
// termination of the connection.
//
// Draining notes:
//
// The reader notices that the server is draining by polling the IsDraining
// closure passed to serveImpl. At that point, the reader delegates the
// responsibility of closing the connection to the statement processor: it will
// push a DrainRequest to the stmtBuf which signals the processor to quit ASAP.
// The processor will quit immediately upon seeing that command if it's not
// currently in a transaction. If it is in a transaction, it will wait until the
// first time a Sync command is processed outside of a transaction - the logic
// being that we want to stop when we're both outside transactions and outside
// batches.
func (s *Server) serveImpl(
ctx context.Context,
c *conn,
reserved *mon.BoundAccount,
authOpt authOptions,
sessionID clusterunique.ID,
) {
if c.sessionArgs.User.IsRootUser() || c.sessionArgs.User.IsNodeUser() {
ctx = logtags.AddTag(ctx, "user", redact.Safe(c.sessionArgs.User))
} else {
ctx = logtags.AddTag(ctx, "user", c.sessionArgs.User)
}
tracing.SpanFromContext(ctx).SetTag("user", attribute.StringValue(c.sessionArgs.User.Normalized()))
sqlServer := s.SQLServer
inTestWithoutSQL := sqlServer == nil
if !inTestWithoutSQL {
sessionStart := timeutil.Now()
defer func() {
if c.authLogEnabled() {
endTime := timeutil.Now()
ev := &eventpb.ClientSessionEnd{
CommonEventDetails: logpb.CommonEventDetails{Timestamp: endTime.UnixNano()},
CommonConnectionDetails: authOpt.connDetails,
Duration: endTime.Sub(sessionStart).Nanoseconds(),
}
log.StructuredEvent(ctx, ev)
}
}()
}
// NOTE: We're going to write a few messages to the connection in this method,
// for the handshake. After that, all writes are done async, in the
// startWriter() goroutine.
// the authPipe below logs authentication messages iff its auth
// logger is non-nil. We define this here.
logAuthn := !inTestWithoutSQL && c.authLogEnabled()
// We'll build an authPipe to communicate with the authentication process.
systemIdentity := c.sessionArgs.SystemIdentity
if systemIdentity.Undefined() {
systemIdentity = c.sessionArgs.User
}
authPipe := newAuthPipe(c, logAuthn, authOpt, systemIdentity)
// procCh is the channel on which we'll receive the termination signal from
// the command processor.
procCh := make(chan struct{})
// We need a value for the unqualified int size here, but it is controlled
// by a session variable, and this layer doesn't have access to the session
// data. The callback below is called whenever default_int_size changes.
// It happens in a different goroutine, so it has to be changed atomically.
var atomicUnqualifiedIntSize = new(int32)
onDefaultIntSizeChange := func(newSize int32) {
atomic.StoreInt32(atomicUnqualifiedIntSize, newSize)
}
if !inTestWithoutSQL {
// Spawn the command processing goroutine, which also handles connection
// authentication). It will notify us when it's done through procCh, and
// we'll also interact with the authentication process through authPipe.
go func() {
// Inform the connection goroutine.
defer close(procCh)
c.processCommands(
ctx,
authOpt,
authPipe,
sqlServer,
reserved,
onDefaultIntSizeChange,
sessionID,
)
}()
} else {