forked from influxdata/influxdb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
raft_server.go
701 lines (607 loc) · 19 KB
/
raft_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
package coordinator
import (
"bytes"
"cluster"
"common"
"configuration"
"encoding/binary"
"encoding/json"
"errors"
"fmt"
"io/ioutil"
"math/rand"
"net"
"net/http"
"os"
"parser"
"path/filepath"
"protocol"
"strings"
"sync"
"time"
log "code.google.com/p/log4go"
"github.com/goraft/raft"
"github.com/gorilla/mux"
)
const (
DEFAULT_ROOT_PWD = "root"
DEFAULT_ROOT_PWD_ENVKEY = "INFLUXDB_INIT_PWD"
RAFT_NAME_SIZE = 8
)
// The raftd server is a combination of the Raft server and an HTTP
// server which acts as the transport.
type RaftServer struct {
name string
path string
bind_address string
router *mux.Router
raftServer raft.Server
httpServer *http.Server
clusterConfig *cluster.ClusterConfiguration
mutex sync.RWMutex
listener net.Listener
closing bool
config *configuration.Configuration
notLeader chan bool
coordinator *CoordinatorImpl
processContinuousQueries bool
}
var registeredCommands bool
// Creates a new server.
func NewRaftServer(config *configuration.Configuration, clusterConfig *cluster.ClusterConfiguration) *RaftServer {
// raft.SetLogLevel(raft.Debug)
if !registeredCommands {
registeredCommands = true
for _, command := range internalRaftCommands {
raft.RegisterCommand(command)
}
}
s := &RaftServer{
path: config.RaftDir,
clusterConfig: clusterConfig,
notLeader: make(chan bool, 1),
router: mux.NewRouter(),
config: config,
}
// Read existing name or generate a new one.
if b, err := ioutil.ReadFile(filepath.Join(s.path, "name")); err == nil {
s.name = string(b)
} else {
var i uint64
if _, err := os.Stat("/dev/urandom"); err == nil {
log.Info("Using /dev/urandom to initialize the raft server name")
f, err := os.Open("/dev/urandom")
if err != nil {
panic(err)
}
defer f.Close()
readBytes := 0
b := make([]byte, RAFT_NAME_SIZE)
for readBytes < RAFT_NAME_SIZE {
n, err := f.Read(b[readBytes:])
if err != nil {
panic(err)
}
readBytes += n
}
err = binary.Read(bytes.NewBuffer(b), binary.BigEndian, &i)
if err != nil {
panic(err)
}
} else {
log.Info("Using rand package to generate raft server name")
rand.Seed(time.Now().UnixNano())
i = uint64(rand.Int())
}
s.name = fmt.Sprintf("%08x", i)
log.Info("Setting raft name to %s", s.name)
if err = ioutil.WriteFile(filepath.Join(s.path, "name"), []byte(s.name), 0644); err != nil {
panic(err)
}
}
return s
}
func (s *RaftServer) GetRaftName() string {
return s.name
}
func (s *RaftServer) leaderConnectString() (string, bool) {
leader := s.raftServer.Leader()
peers := s.raftServer.Peers()
if peer, ok := peers[leader]; !ok {
return "", false
} else {
return peer.ConnectionString, true
}
}
func (s *RaftServer) doOrProxyCommand(command raft.Command) (interface{}, error) {
var err error
var value interface{}
for i := 0; i < 3; i++ {
value, err = s.doOrProxyCommandOnce(command)
if err == nil {
return value, nil
}
if strings.Contains(err.Error(), "node failure") {
continue
}
return nil, err
}
return nil, err
}
func (s *RaftServer) doOrProxyCommandOnce(command raft.Command) (interface{}, error) {
if s.raftServer.State() == raft.Leader {
value, err := s.raftServer.Do(command)
if err != nil {
log.Error("Cannot run command %#v. %s", command, err)
}
return value, err
} else {
if leader, ok := s.leaderConnectString(); !ok {
return nil, errors.New("Couldn't connect to the cluster leader...")
} else {
return SendCommandToServer(leader, command)
}
}
return nil, nil
}
func SendCommandToServer(url string, command raft.Command) (interface{}, error) {
var b bytes.Buffer
if err := json.NewEncoder(&b).Encode(command); err != nil {
return nil, err
}
resp, err := http.Post(url+"/process_command/"+command.CommandName(), "application/json", &b)
if err != nil {
return nil, err
}
defer resp.Body.Close()
body, err2 := ioutil.ReadAll(resp.Body)
if resp.StatusCode != 200 {
return nil, errors.New(strings.TrimSpace(string(body)))
}
var js interface{}
json.Unmarshal(body, &js)
return js, err2
}
func (s *RaftServer) CreateDatabase(name string) error {
command := NewCreateDatabaseCommand(name)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) DropDatabase(name string) error {
command := NewDropDatabaseCommand(name)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) SaveDbUser(u *cluster.DbUser) error {
command := NewSaveDbUserCommand(u)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) ChangeDbUserPassword(db, username string, hash []byte) error {
command := NewChangeDbUserPasswordCommand(db, username, string(hash))
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) ChangeDbUserPermissions(db, username, readPermissions, writePermissions string) error {
command := NewChangeDbUserPermissionsCommand(db, username, readPermissions, writePermissions)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) SaveClusterAdminUser(u *cluster.ClusterAdmin) error {
command := NewSaveClusterAdminCommand(u)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) CreateRootUser() error {
u := &cluster.ClusterAdmin{cluster.CommonUser{"root", "", false, "root"}}
password := os.Getenv(DEFAULT_ROOT_PWD_ENVKEY)
if password == "" {
password = DEFAULT_ROOT_PWD
}
hash, _ := cluster.HashPassword(password)
u.ChangePassword(string(hash))
return s.SaveClusterAdminUser(u)
}
func (s *RaftServer) SetContinuousQueryTimestamp(timestamp time.Time) error {
command := NewSetContinuousQueryTimestampCommand(timestamp)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) CreateContinuousQuery(db string, query string) error {
selectQuery, err := parser.ParseSelectQuery(query)
if err != nil {
return fmt.Errorf("Failed to parse continuous query: %s", query)
}
if !selectQuery.IsValidContinuousQuery() {
return fmt.Errorf("Continuous queries with a group by clause must include time(...) as one of the elements")
}
if !selectQuery.IsNonRecursiveContinuousQuery() {
return fmt.Errorf("Continuous queries with :series_name interpolation must use a regular expression in the from clause that prevents recursion")
}
duration, err := selectQuery.GetGroupByClause().GetGroupByTime()
if err != nil {
return fmt.Errorf("Couldn't get group by time for continuous query: %s", err)
}
// if there are already-running queries, we need to initiate a backfill
if duration != nil && !s.clusterConfig.LastContinuousQueryRunTime().IsZero() {
zeroTime := time.Time{}
currentBoundary := time.Now().Truncate(*duration)
go s.runContinuousQuery(db, selectQuery, zeroTime, currentBoundary)
} else {
// TODO: make continuous queries backfill for queries that don't have a group by time
}
command := NewCreateContinuousQueryCommand(db, query)
_, err = s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) DeleteContinuousQuery(db string, id uint32) error {
command := NewDeleteContinuousQueryCommand(db, id)
_, err := s.doOrProxyCommand(command)
return err
}
func (s *RaftServer) ChangeConnectionString(raftName, protobufConnectionString, raftConnectionString string, forced bool) error {
command := &InfluxChangeConnectionStringCommand{
Force: true,
Name: raftName,
ConnectionString: raftConnectionString,
ProtobufConnectionString: protobufConnectionString,
}
for _, s := range s.raftServer.Peers() {
// send the command and ignore errors in case a server is down
SendCommandToServer(s.ConnectionString, command)
}
// make the change permament
command.Force = false
_, err := s.doOrProxyCommand(command)
log.Info("Running the actual command")
return err
}
func (s *RaftServer) AssignCoordinator(coordinator *CoordinatorImpl) error {
s.coordinator = coordinator
return nil
}
const (
MAX_SIZE = 10 * MEGABYTE
)
func (s *RaftServer) ForceLogCompaction() error {
err := s.raftServer.TakeSnapshot()
if err != nil {
log.Error("Cannot take snapshot: %s", err)
return err
}
return nil
}
func (s *RaftServer) CompactLog() {
checkSizeTicker := time.Tick(time.Minute)
forceCompactionTicker := time.Tick(time.Hour * 24)
for {
select {
case <-checkSizeTicker:
log.Debug("Testing if we should compact the raft logs")
path := s.raftServer.LogPath()
size, err := common.GetFileSize(path)
if err != nil {
log.Error("Error getting size of file '%s': %s", path, err)
}
if size < MAX_SIZE {
continue
}
s.ForceLogCompaction()
case <-forceCompactionTicker:
s.ForceLogCompaction()
}
}
}
func (s *RaftServer) CommittedAllChanges() bool {
entries := s.raftServer.LogEntries()
if len(entries) == 0 {
if s.raftServer.CommitIndex() == 0 {
// commit index should never be zero, we have at least one
// raft join command that should be committed, something is
// wrong, return false to be safe
return false
}
// may be we recovered from a snapshot ?
return true
}
lastIndex := entries[len(entries)-1].Index()
return s.raftServer.CommitIndex() == lastIndex
}
func (s *RaftServer) startRaft() error {
log.Info("Initializing Raft Server: %s", s.config.RaftConnectionString())
// Initialize and start Raft server.
transporter := raft.NewHTTPTransporter("/raft")
var err error
s.raftServer, err = raft.NewServer(s.name, s.path, transporter, s.clusterConfig, s.clusterConfig, "")
if err != nil {
return err
}
s.raftServer.SetElectionTimeout(s.config.RaftTimeout.Duration)
s.raftServer.LoadSnapshot() // ignore errors
s.raftServer.AddEventListener(raft.StateChangeEventType, s.raftEventHandler)
transporter.Install(s.raftServer, s)
s.raftServer.Start()
go s.CompactLog()
if !s.raftServer.IsLogEmpty() {
log.Info("Recovered from log")
return nil
}
potentialLeaders := s.config.SeedServers
if len(potentialLeaders) == 0 {
log.Info("Starting as new Raft leader...")
name := s.raftServer.Name()
_, err := s.raftServer.Do(&InfluxJoinCommand{
Name: name,
ConnectionString: s.config.RaftConnectionString(),
ProtobufConnectionString: s.config.ProtobufConnectionString(),
})
if err != nil {
log.Error(err)
}
err = s.CreateRootUser()
return err
}
for {
for _, leader := range potentialLeaders {
log.Info("(raft:%s) Attempting to join leader: %s", s.raftServer.Name(), leader)
if err := s.Join(leader); err == nil {
log.Info("Joined: %s", leader)
return nil
}
}
log.Warn("Couldn't join any of the seeds, sleeping and retrying...")
time.Sleep(100 * time.Millisecond)
}
return nil
}
func (s *RaftServer) raftEventHandler(e raft.Event) {
if e.Value() == "leader" {
log.Info("(raft:%s) Selected as leader. Starting leader loop.", s.raftServer.Name())
go s.raftLeaderLoop(time.NewTicker(1 * time.Second))
}
if e.PrevValue() == "leader" {
log.Info("(raft:%s) Demoted from leader. Ending leader loop.", s.raftServer.Name())
s.notLeader <- true
}
}
func (s *RaftServer) raftLeaderLoop(loopTimer *time.Ticker) {
for {
select {
case <-loopTimer.C:
log.Debug("(raft:%s) Executing leader loop.", s.raftServer.Name())
s.checkContinuousQueries()
break
case <-s.notLeader:
log.Debug("(raft:%s) Exiting leader loop.", s.raftServer.Name())
return
}
}
}
func (s *RaftServer) StartProcessingContinuousQueries() {
s.processContinuousQueries = true
}
func (s *RaftServer) checkContinuousQueries() {
if !s.processContinuousQueries {
return
}
if !s.clusterConfig.HasContinuousQueries() {
return
}
runTime := time.Now()
queriesDidRun := false
for db, queries := range s.clusterConfig.ParsedContinuousQueries {
for _, query := range queries {
groupByClause := query.GetGroupByClause()
// if there's no group by clause, it's handled as a fanout query
if groupByClause.Elems == nil {
continue
}
duration, err := query.GetGroupByClause().GetGroupByTime()
if err != nil {
log.Error("Couldn't get group by time for continuous query:", err)
continue
}
currentBoundary := runTime.Truncate(*duration)
lastRun := s.clusterConfig.LastContinuousQueryRunTime()
lastBoundary := lastRun.Truncate(*duration)
if currentBoundary.After(lastRun) {
s.runContinuousQuery(db, query, lastBoundary, currentBoundary)
queriesDidRun = true
}
}
}
if queriesDidRun {
s.clusterConfig.SetLastContinuousQueryRunTime(runTime)
s.SetContinuousQueryTimestamp(runTime)
}
}
func (s *RaftServer) runContinuousQuery(db string, query *parser.SelectQuery, start time.Time, end time.Time) {
adminName := s.clusterConfig.GetClusterAdmins()[0]
clusterAdmin := s.clusterConfig.GetClusterAdmin(adminName)
intoClause := query.GetIntoClause()
targetName := intoClause.Target.Name
queryString := query.GetQueryStringWithTimesAndNoIntoClause(start, end)
f := func(series *protocol.Series) error {
return s.coordinator.InterpolateValuesAndCommit(query.GetQueryString(), db, series, targetName, true)
}
writer := NewContinuousQueryWriter(f)
s.coordinator.RunQuery(clusterAdmin, db, queryString, writer)
}
func (s *RaftServer) ListenAndServe() error {
l, err := net.Listen("tcp", s.config.RaftListenString())
if err != nil {
panic(err)
}
return s.Serve(l)
}
func (s *RaftServer) Serve(l net.Listener) error {
s.listener = l
log.Info("Initializing Raft HTTP server")
// Initialize and start HTTP server.
s.httpServer = &http.Server{
Handler: s.router,
}
s.router.HandleFunc("/cluster_config", s.configHandler).Methods("GET")
s.router.HandleFunc("/join", s.joinHandler).Methods("POST")
s.router.HandleFunc("/process_command/{command_type}", s.processCommandHandler).Methods("POST")
log.Info("Raft Server Listening at %s", s.config.RaftListenString())
go func() {
err := s.httpServer.Serve(l)
if !strings.Contains(err.Error(), "closed network") {
panic(err)
}
}()
started := make(chan error)
go func() {
started <- s.startRaft()
}()
err := <-started
// time.Sleep(3 * time.Second)
return err
}
func (self *RaftServer) Close() {
if !self.closing || self.raftServer == nil {
self.closing = true
self.raftServer.Stop()
self.listener.Close()
self.notLeader <- true
}
}
// This is a hack around Gorilla mux not providing the correct net/http
// HandleFunc() interface.
func (s *RaftServer) HandleFunc(pattern string, handler func(http.ResponseWriter, *http.Request)) {
s.router.HandleFunc(pattern, handler)
}
// Joins to the leader of an existing cluster.
func (s *RaftServer) Join(leader string) error {
command := &InfluxJoinCommand{
Name: s.raftServer.Name(),
ConnectionString: s.config.RaftConnectionString(),
ProtobufConnectionString: s.config.ProtobufConnectionString(),
}
connectUrl := leader
if !strings.HasPrefix(connectUrl, "http://") {
connectUrl = "http://" + connectUrl
}
if !strings.HasSuffix(connectUrl, "/join") {
connectUrl = connectUrl + "/join"
}
var b bytes.Buffer
json.NewEncoder(&b).Encode(command)
log.Debug("(raft:%s) Posting to seed server %s", s.raftServer.Name(), connectUrl)
tr := &http.Transport{
ResponseHeaderTimeout: time.Second,
}
client := &http.Client{Transport: tr}
resp, err := client.Post(connectUrl, "application/json", &b)
if err != nil {
log.Error(err)
return err
}
defer resp.Body.Close()
if resp.StatusCode == http.StatusTemporaryRedirect {
address := resp.Header.Get("Location")
log.Debug("Redirected to %s to join leader", address)
return s.Join(address)
}
log.Debug("(raft:%s) Posted to seed server %s", s.raftServer.Name(), connectUrl)
return nil
}
func (s *RaftServer) retryCommand(command raft.Command, retries int) (ret interface{}, err error) {
for retries = retries; retries > 0; retries-- {
ret, err = s.raftServer.Do(command)
if err == nil {
return ret, nil
}
time.Sleep(50 * time.Millisecond)
log.Info("Retrying RAFT command...")
}
return
}
func (s *RaftServer) joinHandler(w http.ResponseWriter, req *http.Request) {
// if this is the leader, process the command
if s.raftServer.State() == raft.Leader {
command := &InfluxJoinCommand{}
if err := json.NewDecoder(req.Body).Decode(&command); err != nil {
http.Error(w, err.Error(), http.StatusInternalServerError)
return
}
log.Debug("ON RAFT LEADER - JOIN: %v", command)
// during the test suite the join command will sometimes time out.. just retry a few times
if _, err := s.raftServer.Do(command); err != nil {
log.Error("Can't process %v: %s", command, err)
http.Error(w, err.Error(), http.StatusInternalServerError)
}
return
}
leader, ok := s.leaderConnectString()
log.Debug("Non-leader redirecting to: (%v, %v)", leader, ok)
if ok {
log.Debug("redirecting to leader to join...")
http.Redirect(w, req, leader+"/join", http.StatusTemporaryRedirect)
} else {
http.Error(w, errors.New("Couldn't find leader of the cluster to join").Error(), http.StatusInternalServerError)
}
}
func (s *RaftServer) configHandler(w http.ResponseWriter, req *http.Request) {
js, err := json.Marshal(s.clusterConfig.GetMapForJsonSerialization())
if err != nil {
log.Error("ERROR marshalling config: ", err)
}
w.Write(js)
}
func (s *RaftServer) marshalAndDoCommandFromBody(command raft.Command, req *http.Request) (interface{}, error) {
if err := json.NewDecoder(req.Body).Decode(&command); err != nil {
return nil, err
}
if c, ok := command.(*InfluxChangeConnectionStringCommand); ok && c.Force {
// if this is a forced change, just do it now and return. Note
// that this isn't a permanent change, since on restart the old
// connection strings will be used
return c.Apply(s.raftServer)
}
if result, err := s.raftServer.Do(command); err != nil {
return nil, err
} else {
return result, nil
}
}
func (s *RaftServer) processCommandHandler(w http.ResponseWriter, req *http.Request) {
vars := mux.Vars(req)
value := vars["command_type"]
command := internalRaftCommands[value]
if result, err := s.marshalAndDoCommandFromBody(command, req); err != nil {
log.Error("command %T failed: %s", command, err)
http.Error(w, err.Error(), http.StatusInternalServerError)
} else {
if result != nil {
js, _ := json.Marshal(result)
w.Write(js)
}
}
}
func (self *RaftServer) CreateShards(shards []*cluster.NewShardData) ([]*cluster.ShardData, error) {
log.Debug("RAFT: CreateShards")
command := NewCreateShardsCommand(shards)
createShardsResult, err := self.doOrProxyCommand(command)
if err != nil {
log.Error("RAFT: CreateShards: ", err)
return nil, err
}
js, err := json.Marshal(createShardsResult)
if err != nil {
return nil, err
}
newShards := make([]*cluster.NewShardData, 0)
err = json.Unmarshal(js, &newShards)
if err != nil {
return nil, err
}
log.Debug("NEW SHARDS: ", newShards)
return self.clusterConfig.MarshalNewShardArrayToShards(newShards)
}
func (self *RaftServer) DropShard(id uint32, serverIds []uint32) error {
command := NewDropShardCommand(id, serverIds)
_, err := self.doOrProxyCommand(command)
return err
}