-
Notifications
You must be signed in to change notification settings - Fork 285
/
Copy pathserver.go
5769 lines (5294 loc) · 171 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 2016-2020 The NATS Authors
// 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.
package server
import (
"crypto/tls"
"errors"
"fmt"
"net"
"net/url"
"os"
"path/filepath"
"regexp"
"runtime"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/hashicorp/raft"
natsdLogger "github.com/nats-io/nats-server/v2/logger"
"github.com/nats-io/nats-server/v2/server"
"github.com/nats-io/nats-streaming-server/logger"
"github.com/nats-io/nats-streaming-server/spb"
"github.com/nats-io/nats-streaming-server/stores"
"github.com/nats-io/nats-streaming-server/util"
"github.com/nats-io/nats.go"
"github.com/nats-io/nuid"
"github.com/nats-io/stan.go/pb"
)
// A single NATS Streaming Server
// Server defaults.
const (
// VERSION is the current version for the NATS Streaming server.
VERSION = "0.20.0"
DefaultClusterID = "test-cluster"
DefaultDiscoverPrefix = "_STAN.discover"
DefaultPubPrefix = "_STAN.pub"
DefaultSubPrefix = "_STAN.sub"
DefaultSubClosePrefix = "_STAN.subclose"
DefaultUnSubPrefix = "_STAN.unsub"
DefaultClosePrefix = "_STAN.close"
defaultAcksPrefix = "_STAN.ack"
defaultSnapshotPrefix = "_STAN.snap"
defaultRaftPrefix = "_STAN.raft"
DefaultStoreType = stores.TypeMemory
// Prefix of subject active server is sending HBs to
ftHBPrefix = "_STAN.ft"
// DefaultHeartBeatInterval is the interval at which server sends heartbeat to a client
DefaultHeartBeatInterval = 30 * time.Second
// DefaultClientHBTimeout is how long server waits for a heartbeat response
DefaultClientHBTimeout = 10 * time.Second
// DefaultMaxFailedHeartBeats is the number of failed heartbeats before server closes
// the client connection (total= (heartbeat interval + heartbeat timeout) * (fail count + 1)
DefaultMaxFailedHeartBeats = int((5 * time.Minute) / DefaultHeartBeatInterval)
// Timeout used to ping the known client when processing a connection
// request for a duplicate client ID.
defaultCheckDupCIDTimeout = 500 * time.Millisecond
// DefaultIOBatchSize is the maximum number of messages to accumulate before flushing a store.
DefaultIOBatchSize = 1024
// DefaultIOSleepTime is the duration (in micro-seconds) the server waits for more messages
// before starting processing. Set to 0 (or negative) to disable the wait.
DefaultIOSleepTime = int64(0)
// DefaultLogCacheSize is the number of Raft log entries to cache in memory
// to reduce disk IO.
DefaultLogCacheSize = 512
// DefaultLogSnapshots is the number of Raft log snapshots to retain.
DefaultLogSnapshots = 2
// DefaultTrailingLogs is the number of log entries to leave after a
// snapshot and compaction.
DefaultTrailingLogs = 10240
// Length of the channel used to schedule subscriptions start requests.
// Subscriptions requests are processed from the same NATS subscription.
// When a subscriber starts and it has pending messages, the server
// processes the new subscription request and sends avail messages out
// (up to MaxInflight). When done in place, this can cause other
// new subscriptions requests to timeout. Server uses a channel to schedule
// start (that is sending avail messages) of new subscriptions. This is
// the default length of that channel.
defaultSubStartChanLen = 2048
// Name of the file to store Raft log.
raftLogFile = "raft.log"
// In partitioning mode, when a client connects, the connect request
// may reach several servers, but the first response the client gets
// allows it to proceed with either publish or subscribe.
// So it is possible for a server running in partitioning mode to
// receives a connection request followed by a message or subscription.
// Although the conn request would be first in the tcp connection, it
// is then possible that the PubMsg or SubscriptionRequest be processed
// first due to the use of different nats subscriptions.
// To prevent that, when checking if a client exists, in this particular
// mode we will possibly wait to be notified when the client has been
// registered. This is the default duration for this wait.
defaultClientCheckTimeout = time.Second
// Interval at which server goes through list of subscriptions with
// pending sent/ack operations that needs to be replicated.
defaultLazyReplicationInterval = time.Second
// Log statement printed when server is considered ready
streamingReadyLog = "Streaming Server is ready"
)
// Constant to indicate that sendMsgToSub() should check number of acks pending
// against MaxInFlight to know if message should be sent out.
const (
forceDelivery = true
honorMaxInFlight = false
)
// Constants to indicate if we are replicating a Sent or an Ack
const (
replicateSent = true
replicateAck = false
)
const (
// Client send connID in ConnectRequest and PubMsg, and server
// listens and responds to client PINGs. The validity of the
// connection (based on connID) is checked on incoming PINGs.
protocolOne = int32(1)
)
// Errors.
var (
ErrInvalidSubject = errors.New("stan: invalid subject")
ErrInvalidStart = errors.New("stan: invalid start position")
ErrInvalidSub = errors.New("stan: invalid subscription")
ErrInvalidClient = errors.New("stan: clientID already registered")
ErrMissingClient = errors.New("stan: clientID missing")
ErrInvalidClientID = errors.New("stan: invalid clientID: only alphanumeric and `-` or `_` characters allowed")
ErrInvalidAckWait = errors.New("stan: invalid ack wait time, should be >= 1s")
ErrInvalidMaxInflight = errors.New("stan: invalid MaxInflight, should be >= 1")
ErrInvalidConnReq = errors.New("stan: invalid connection request")
ErrInvalidPubReq = errors.New("stan: invalid publish request")
ErrInvalidSubReq = errors.New("stan: invalid subscription request")
ErrInvalidUnsubReq = errors.New("stan: invalid unsubscribe request")
ErrInvalidCloseReq = errors.New("stan: invalid close request")
ErrDupDurable = errors.New("stan: duplicate durable registration")
ErrInvalidDurName = errors.New("stan: durable name of a durable queue subscriber can't contain the character ':'")
ErrUnknownClient = errors.New("stan: unknown clientID")
ErrNoChannel = errors.New("stan: no configured channel")
ErrClusteredRestart = errors.New("stan: cannot restart server in clustered mode if it was not previously clustered")
ErrChanDelInProgress = errors.New("stan: channel is being deleted")
)
// Shared regular expression to check clientID validity.
// No lock required since from doc: https://golang.org/pkg/regexp/
// A Regexp is safe for concurrent use by multiple goroutines.
var clientIDRegEx *regexp.Regexp
var (
testAckWaitIsInMillisecond bool
clientCheckTimeout = defaultClientCheckTimeout
lazyReplicationInterval = defaultLazyReplicationInterval
testDeleteChannel bool
testSubSentAndAckSlowApply bool
)
var (
// gitCommit injected at build
gitCommit string
)
func computeAckWait(wait int32) time.Duration {
unit := time.Second
if testAckWaitIsInMillisecond && wait < 0 {
wait = wait * -1
unit = time.Millisecond
}
return time.Duration(wait) * unit
}
func init() {
if re, err := regexp.Compile("^[a-zA-Z0-9_-]+$"); err != nil {
panic("Unable to compile regular expression")
} else {
clientIDRegEx = re
}
}
// ioPendingMsg is a record that embeds the pointer to the incoming
// NATS Message, the PubMsg and PubAck structures so we reduce the
// number of memory allocations to 1 when processing a message from
// producer.
type ioPendingMsg struct {
m *nats.Msg
pm pb.PubMsg
pa pb.PubAck
c *channel
dc bool // if true, this is a request to delete this channel.
// Use for synchronization between ioLoop and other routines
sc chan struct{}
sdc chan struct{}
}
// Constant that defines the size of the channel that feeds the IO thread.
const ioChannelSize = 64 * 1024
// subStartInfo contains information used when a subscription request
// is successful and the start (sending avail messages) is scheduled.
type subStartInfo struct {
c *channel
sub *subState
qs *queueState
isDurable bool
}
// State represents the possible server states
type State int8
// Possible server states
const (
Standalone State = iota
FTActive
FTStandby
Failed
Shutdown
Clustered
)
func (state State) String() string {
switch state {
case Standalone:
return "STANDALONE"
case FTActive:
return "FT_ACTIVE"
case FTStandby:
return "FT_STANDBY"
case Failed:
return "FAILED"
case Shutdown:
return "SHUTDOWN"
case Clustered:
return "CLUSTERED"
default:
return "UNKNOW STATE"
}
}
type channelStore struct {
sync.RWMutex
channels map[string]*channel
store stores.Store
stan *StanServer
}
func newChannelStore(srv *StanServer, s stores.Store) *channelStore {
cs := &channelStore{
channels: make(map[string]*channel),
store: s,
stan: srv,
}
return cs
}
func (cs *channelStore) get(name string) *channel {
cs.RLock()
c := cs.channels[name]
cs.RUnlock()
return c
}
func (cs *channelStore) getIfNotAboutToBeDeleted(name string) *channel {
cs.RLock()
c := cs.channels[name]
if c != nil && c.activity != nil && c.activity.deleteInProgress {
cs.RUnlock()
return nil
}
cs.RUnlock()
return c
}
func (cs *channelStore) createChannel(s *StanServer, name string) (*channel, error) {
cs.Lock()
c, err := cs.createChannelLocked(s, name, 0)
cs.Unlock()
return c, err
}
func (cs *channelStore) createChannelLocked(s *StanServer, name string, id uint64) (*channel, error) {
// It is possible that there were 2 concurrent calls to lookupOrCreateChannel
// which first uses `channelStore.get()` and if not found, calls this function.
// So we need to check now that we have the write lock that the channel has
// not already been created.
c := cs.channels[name]
if c != nil {
if c.activity != nil && c.activity.deleteInProgress {
return nil, ErrChanDelInProgress
}
return c, nil
}
if s.isClustered {
if s.isLeader() && id == 0 {
var err error
if id, err = s.raft.store.LastIndex(); err != nil {
return nil, err
}
}
if err := s.raft.store.SetChannelID(name, id); err != nil {
return nil, err
}
}
sc, err := cs.store.CreateChannel(name)
if err != nil {
return nil, err
}
c, err = cs.create(s, name, sc)
if err != nil {
if id != 0 {
cs.stan.raft.store.DeleteChannelID(name)
}
return nil, err
}
c.id = id
if s.isStandaloneOrLeader() && c.activity != nil {
c.startDeleteTimer()
}
cs.stan.log.Noticef("Channel %q has been created", name)
return c, nil
}
// low-level creation and storage in memory of a *channel
// Lock is held on entry or not needed.
func (cs *channelStore) create(s *StanServer, name string, sc *stores.Channel) (*channel, error) {
c := &channel{name: name, store: sc, ss: s.createSubStore(), stan: s, nextSubID: 1}
lastSequence, err := c.store.Msgs.LastSequence()
if err != nil {
return nil, err
}
c.nextSequence = lastSequence + 1
cs.channels[name] = c
cl := cs.store.GetChannelLimits(name)
if cl.MaxInactivity > 0 {
c.activity = &channelActivity{maxInactivity: cl.MaxInactivity}
}
return c, nil
}
func (cs *channelStore) getAll() map[string]*channel {
cs.RLock()
m := make(map[string]*channel, len(cs.channels))
for k, v := range cs.channels {
m[k] = v
}
cs.RUnlock()
return m
}
func (cs *channelStore) msgsState(channelName string) (int, uint64, error) {
cs.RLock()
defer cs.RUnlock()
if channelName != "" {
c := cs.channels[channelName]
if c == nil {
return 0, 0, fmt.Errorf("channel %q not found", channelName)
}
return c.store.Msgs.State()
}
var (
count int
bytes uint64
)
for _, c := range cs.channels {
m, b, err := c.store.Msgs.State()
if err != nil {
return 0, 0, err
}
count += m
bytes += b
}
return count, bytes, nil
}
func (cs *channelStore) count() int {
cs.RLock()
count := len(cs.channels)
cs.RUnlock()
return count
}
func (cs *channelStore) maybeStartChannelDeleteTimer(name string, c *channel) {
cs.Lock()
if c == nil {
c = cs.channels[name]
}
if c != nil && c.activity != nil && !c.activity.deleteInProgress && !c.ss.hasActiveSubs() {
c.startDeleteTimer()
}
cs.Unlock()
}
func (cs *channelStore) stopDeleteTimer(c *channel) {
cs.Lock()
c.stopDeleteTimer()
if c.activity != nil {
c.activity.deleteInProgress = false
}
cs.Unlock()
}
func (cs *channelStore) turnOffPreventDelete(c *channel) {
cs.Lock()
if c != nil && c.activity != nil {
c.activity.preventDelete = false
}
cs.Unlock()
}
type channel struct {
// This is used in clustering mode in a specific situation where
// all messages may have been expired so the store reports 0,0
// but we know that the firstSeq should be something else.
// Used with atomic operation.
firstSeq uint64
nextSequence uint64
name string
id uint64
store *stores.Channel
ss *subStore
lTimestamp int64
stan *StanServer
activity *channelActivity
nextSubID uint64
// Used in cluster mode. This is to know if the message store
// last sequence should be checked before storing a message in
// Apply(). Protected by the raft's FSM lock.
lSeqChecked bool
}
type channelActivity struct {
last time.Time
maxInactivity time.Duration
timer *time.Timer
deleteInProgress bool
preventDelete bool
timerSet bool
}
// Starts the delete timer that when firing will post
// a channel delete request to the ioLoop.
// The channelStore's mutex must be held on entry.
func (c *channel) startDeleteTimer() {
c.activity.last = time.Now()
c.resetDeleteTimer(c.activity.maxInactivity)
}
// Stops the delete timer.
// The channelStore's mutex must be held on entry.
func (c *channel) stopDeleteTimer() {
if c.activity.timer != nil {
c.activity.timer.Stop()
c.activity.timerSet = false
if c.stan.debug {
c.stan.log.Debugf("Channel %q delete timer stopped", c.name)
}
}
}
// Resets the delete timer to the given duration.
// If the timer was not created, this call will create it.
// The channelStore's mutex must be held on entry.
func (c *channel) resetDeleteTimer(newDuration time.Duration) {
a := c.activity
if a.timer == nil {
a.timer = time.AfterFunc(newDuration, func() {
c.stan.sendDeleteChannelRequest(c)
})
} else {
a.timer.Reset(newDuration)
}
if c.stan.debug {
c.stan.log.Debugf("Channel %q delete timer set to fire in %v", c.name, newDuration)
}
a.timerSet = true
}
// pubMsgToMsgProto converts a PubMsg to a MsgProto and assigns a timestamp
// which is monotonic with respect to the channel.
func (c *channel) pubMsgToMsgProto(pm *pb.PubMsg, seq uint64) *pb.MsgProto {
m := &pb.MsgProto{
Sequence: seq,
Subject: pm.Subject,
Reply: pm.Reply,
Data: pm.Data,
Timestamp: time.Now().UnixNano(),
}
if c.lTimestamp > 0 && m.Timestamp < c.lTimestamp {
m.Timestamp = c.lTimestamp
}
c.lTimestamp = m.Timestamp
return m
}
// Sets a subscription that will handle snapshot restore requests from followers.
func (s *StanServer) subToSnapshotRestoreRequests() error {
var (
msgBuf []byte
buf []byte
snapshotRestorePrefix = fmt.Sprintf("%s.%s.", defaultSnapshotPrefix, s.info.ClusterID)
prefixLen = len(snapshotRestorePrefix)
)
sub, err := s.ncsr.Subscribe(snapshotRestorePrefix+">", func(m *nats.Msg) {
if len(m.Data) != 16 {
s.log.Errorf("Invalid snapshot request, data len=%v", len(m.Data))
return
}
// Servers up to 0.14.1 (included) had a defect both on the leader and
// follower side. That is, they would both break out of their loop when
// getting the first "not found" message (expired or removed due to
// limits).
// Starting at 0.14.2, server needs to complete their loop to make sure
// they have restored all messages in the start/end range.
// The code below tries to handle pre and post 0.14.2 follower requests.
// A follower at version 0.14.2+ will include a known suffix to its
// reply subject. The leader here can then send the first available
// message when getting a request for a message of a given sequence
// that is not found.
v2 := strings.HasSuffix(m.Reply, "."+restoreMsgsV2)
// For the newer servers, include a "reply" subject to the response
// to let the follower know that this is coming from a 0.14.2+ server.
var reply string
var replyFirstAvail string
if v2 {
reply = restoreMsgsV2
replyFirstAvail = restoreMsgsV2 + restoreMsgsFirstAvailSuffix
}
cname := m.Subject[prefixLen:]
c := s.channels.getIfNotAboutToBeDeleted(cname)
if c == nil {
s.ncsr.PublishRequest(m.Reply, reply, nil)
return
}
start := util.ByteOrder.Uint64(m.Data[:8])
end := util.ByteOrder.Uint64(m.Data[8:])
for seq := start; seq <= end; seq++ {
sendingTheFirstAvail := false
msg, err := c.store.Msgs.Lookup(seq)
if err != nil {
s.log.Errorf("Snapshot restore request error for channel %q, error looking up message %v: %v", c.name, seq, err)
return
}
// If the requestor is a server 0.14.2+, we will send the first
// available message.
if msg == nil && v2 {
msg, err = c.store.Msgs.FirstMsg()
if err != nil {
s.log.Errorf("Snapshot restore request error for channel %q, error looking up first message: %v", c.name, err)
return
}
sendingTheFirstAvail = msg != nil
if sendingTheFirstAvail && msg.Sequence < seq {
// It could be that a follower tries to restore from
// a snapshot of previous version of the same channel name
// that had more messages than current channel. So
// bail out if the first avail is below the last seq
// we dealt with.
msg = nil
}
}
if msg == nil {
// We don't have this message because of channel limits.
// Return nil to caller to signal this state.
buf = nil
} else {
msgBuf = util.EnsureBufBigEnough(msgBuf, msg.Size())
n, err := msg.MarshalTo(msgBuf)
if err != nil {
panic(err)
}
buf = msgBuf[:n]
}
// This will be empty string for old requestors, or restoreMsgsV2
// for servers 0.14.2+
respReply := reply
if sendingTheFirstAvail {
// Use the reply subject that contains information that this
// is the first available message. Requestors 0.16.1+ will
// make use of that.
respReply = replyFirstAvail
}
if err := s.ncsr.PublishRequest(m.Reply, respReply, buf); err != nil {
s.log.Errorf("Snapshot restore request error for channel %q, unable to send response for seq %v: %v", c.name, seq, err)
return
}
// If we sent a message and seq was not the expected one,
// reset seq to proper value for the next iteration.
if msg != nil && msg.Sequence != seq {
seq = msg.Sequence
} else if buf == nil {
// Regardless of the version of the requestor, if we are
// here we can stop. If it is a pre 0.14.2, the follower
// would have break out of its for-loop when getting the
// nil message, so there is no point in continuing.
// And if it is a 0.14.2+, we have sent the first avail
// message and so if we are here it means that there
// is none, so we really have nothing else to do.
return
}
select {
case <-s.shutdownCh:
return
default:
}
}
})
if err != nil {
return err
}
sub.SetPendingLimits(-1, -1)
s.snapReqSub = sub
return nil
}
// StanServer structure represents the NATS Streaming Server
type StanServer struct {
// Keep all members for which we use atomic at the beginning of the
// struct and make sure they are all 64bits (or use padding if necessary).
// atomic.* functions crash on 32bit machines if operand is not aligned
// at 64bit. See https://github.com/golang/go/issues/599
ioChannelStatsMaxBatchSize int64 // stats of the max number of messages than went into a single batch
stats struct {
inMsgs int64
inBytes int64
outMsgs int64
outBytes int64
}
mu sync.RWMutex
shutdown bool
shutdownCh chan struct{}
serverID string
info spb.ServerInfo // Contains cluster ID and subjects
natsServer *server.Server
opts *Options
natsOpts *server.Options
startTime time.Time
// For scalability, a dedicated connection is used to publish
// messages to subscribers and for replication.
nc *nats.Conn // used for most protocol messages
ncs *nats.Conn // used for sending to subscribers and acking publishers
nca *nats.Conn // used to receive subscriptions acks
ncr *nats.Conn // used for raft messages
ncsr *nats.Conn // used for raft snapshot replication
wg sync.WaitGroup // Wait on go routines during shutdown
// Used when processing connect requests for client ID already registered
dupCIDTimeout time.Duration
// Clients
clients *clientStore
cliDupCIDsMu sync.Mutex
cliDipCIDsMap map[string]struct{}
// channels
channels *channelStore
// Store
store stores.Store
// IO Channel
ioChannel chan *ioPendingMsg
ioChannelQuit chan struct{}
ioChannelWG sync.WaitGroup
// To protect some close related requests
closeMu sync.Mutex
tmpBuf []byte // Used to marshal protocols (right now, only PubAck)
subStartCh chan *subStartInfo
subStartQuit chan struct{}
// For FT mode
ftnc *nats.Conn
ftSubject string
ftHBInterval time.Duration
ftHBMissedInterval time.Duration
ftHBCh chan *nats.Msg
ftQuit chan struct{}
state State
// This is in cases where a fatal error occurs after the server was
// started. We call Fatalf, but for users starting the server
// programmatically, it is a way to report what the error was.
lastError error
// Will be created only when running in partitioning mode.
partitions *partitions
// Use these flags for Debug/Trace in places where speed matters.
// Normally, Debugf and Tracef will check an internal variable to
// figure out if the statement should be logged, however, the
// cost of calling Debugf/Tracef is still significant since there
// may be memory allocations to format the string passed to these
// calls. So in those situations, use these flags to surround the
// calls to Debugf/Tracef.
trace bool
debug bool
log *logger.StanLogger
// Specific to clustering
raft *raftNode
raftLogging bool
isClustered bool
ssarepl *subsSentAndAckReplication
snapReqSub *nats.Subscription
// Our internal subscriptions
connectSub *nats.Subscription
closeSub *nats.Subscription
pubSub *nats.Subscription
subSub *nats.Subscription
subCloseSub *nats.Subscription
subUnsubSub *nats.Subscription
cliPingSub *nats.Subscription
addNodeSub *nats.Subscription
rmNodeSub *nats.Subscription
// For sending responses to client PINGS. Used to be global but would
// cause races when running more than 1 server in a program or test.
pingResponseOKBytes []byte
pingResponseInvalidClientBytes []byte
// If using an external server, capture the URL that was given for return in ClientURL().
providedServerURL string
}
type subsSentAndAckReplication struct {
ready *sync.Map
waiting *sync.Map
gates *sync.Map
notifyCh chan struct{}
}
func (s *StanServer) isLeader() bool {
return atomic.LoadInt64(&s.raft.leader) == 1
}
func (s *StanServer) isStandaloneOrLeader() bool {
return !s.isClustered || s.isLeader()
}
// subStore holds all known state for all subscriptions
type subStore struct {
sync.RWMutex
psubs []*subState // plain subscribers
qsubs map[string]*queueState // queue subscribers
durables map[string]*subState // durables lookup
acks map[string]*subState // ack inbox lookup
stan *StanServer // back link to the server
}
// Holds all queue subsribers for a subject/group and
// tracks lastSent for the group.
type queueState struct {
sync.RWMutex
lastSent uint64
subs []*subState
rdlvCount map[uint64]uint32
shadow *subState // For durable case, when last member leaves and group is not closed.
stalledSubCount int // number of stalled members
newOnHold bool
}
// When doing message redelivery due to ack expiration, the function
// makeSortedPendingMsgs return an array of pendingMsg objects,
// ordered by their expiration date.
type pendingMsg struct {
seq uint64
expire int64
}
// Holds Subscription state
type subState struct {
sync.RWMutex
spb.SubState // Embedded protobuf. Used for storage.
subject string
qstate *queueState
ackWait time.Duration // SubState.AckWaitInSecs expressed as a time.Duration
ackTimer *time.Timer
ackSub *nats.Subscription
acksPending map[uint64]int64 // key is message sequence, value is expiration time.
store stores.SubStore // for easy access to the store interface
savedClientID string // Used only for closed durables in Clustering mode and monitoring endpoints.
replicate *subSentAndAck // Used in Clustering mode
norepl bool // When a sub is being closed, prevents collectSentOrAck to recreate `replicate`.
rdlvCount map[uint64]uint32 // Used only when not a queue sub, otherwise queueState's rldvCount is used.
// So far, compacting these booleans into a byte flag would not save space.
// May change if we need to add more.
initialized bool // false until the subscription response has been sent to prevent data to be sent too early.
stalled bool
newOnHold bool // Prevents delivery of new msgs until old are redelivered (on restart)
hasFailedHB bool // This is set when server sends heartbeat to this subscriber's client.
}
type subSentAndAck struct {
sent []uint64
ack []uint64
applying bool
}
// Returns the total number of subscriptions (including offline (queue) durables).
// This is used essentially for the monitoring endpoint /streaming/serverz.
func (s *StanServer) numSubs() int {
total := 0
s.channels.RLock()
for _, c := range s.channels.channels {
c.ss.RLock()
total += len(c.ss.psubs)
// Need to add offline durables
for _, sub := range c.ss.durables {
if sub.ClientID == "" {
total++
}
}
for _, qsub := range c.ss.qsubs {
qsub.RLock()
total += len(qsub.subs)
// If this is a durable queue subscription and all members
// are offline, qsub.shadow will be not nil. Report this one.
if qsub.shadow != nil {
total++
}
qsub.RUnlock()
}
c.ss.RUnlock()
}
s.channels.RUnlock()
return total
}
// Looks up, or create a new channel if it does not exist
func (s *StanServer) lookupOrCreateChannel(name string) (*channel, error) {
cs := s.channels
cs.RLock()
c := cs.channels[name]
if c != nil {
if c.activity != nil && c.activity.deleteInProgress {
cs.RUnlock()
return nil, ErrChanDelInProgress
}
cs.RUnlock()
return c, nil
}
cs.RUnlock()
return cs.createChannel(s, name)
}
func (s *StanServer) lookupOrCreateChannelPreventDelete(name string) (*channel, bool, error) {
cs := s.channels
cs.Lock()
c := cs.channels[name]
if c != nil {
if c.activity != nil && c.activity.deleteInProgress {
cs.Unlock()
return nil, false, ErrChanDelInProgress
}
} else {
var err error
c, err = cs.createChannelLocked(s, name, 0)
if err != nil {
cs.Unlock()
return nil, false, err
}
}
preventDelete := false
if c.activity != nil {
c.activity.preventDelete = true
c.stopDeleteTimer()
preventDelete = true
}
cs.Unlock()
return c, preventDelete, nil
}
// createSubStore creates a new instance of `subStore`.
func (s *StanServer) createSubStore() *subStore {
subs := &subStore{
psubs: make([]*subState, 0, 4),
qsubs: make(map[string]*queueState),
durables: make(map[string]*subState),
acks: make(map[string]*subState),
stan: s,
}
return subs
}
// Store adds this subscription to the server's `subStore` and also in storage
func (ss *subStore) Store(sub *subState) error {
if sub == nil {
return nil
}
// Adds to storage.
// Use sub lock to avoid race with waitForAcks in some tests
sub.Lock()
err := sub.store.CreateSub(&sub.SubState)
sub.Unlock()
if err == nil {
err = sub.store.Flush()
}
if err != nil {
ss.stan.log.Errorf("Unable to store subscription [%v:%v] on [%s]: %v", sub.ClientID, sub.Inbox, sub.subject, err)
return err
}
ss.updateState(sub)
return nil
}
// Updates the subStore state with this sub.
// The subStore is locked on entry (or does not need, as during server restart).
// However, `sub` does not need locking since it has just been created.
func (ss *subStore) updateState(sub *subState) {
// Store by type
if sub.isQueueSubscriber() {
// Queue subscriber.
qs := ss.qsubs[sub.QGroup]
if qs == nil {
qs = &queueState{
subs: make([]*subState, 0, 4),
}
ss.qsubs[sub.QGroup] = qs
}
qs.Lock()
// The recovered shadow queue sub will have ClientID=="",
// keep a reference to it until a member re-joins the group.
if sub.ClientID == "" {
// There should be only one shadow queue subscriber, but
// we found in https://github.com/nats-io/nats-streaming-server/issues/322
// that some datastore had 2 of those (not sure how this happened except
// maybe due to upgrades from much older releases that had bugs?).
// So don't panic and use as the shadow the one with the highest LastSent
// value.
if qs.shadow != nil {
ss.stan.log.Warnf("Duplicate shadow durable queue consumer (subid=%v) for group %q", sub.ID, sub.QGroup)
}
if qs.shadow == nil || sub.LastSent > qs.lastSent {
qs.shadow = sub
}
} else {
// Store by ackInbox for ack direct lookup
ss.acks[sub.AckInbox] = sub
qs.subs = append(qs.subs, sub)
// If the added sub has newOnHold it means that we are doing recovery and
// that this member had unacknowledged messages. Mark the queue group
// with newOnHold
if sub.newOnHold {
qs.newOnHold = true
}
// Update stalled (on recovery)
if sub.stalled {
qs.stalledSubCount++
}
}