/
wal.go
138 lines (119 loc) · 3.5 KB
/
wal.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
package raftwal
import (
"encoding/binary"
"github.com/coreos/etcd/raft"
"github.com/coreos/etcd/raft/raftpb"
"github.com/dgraph-io/dgraph/store"
"github.com/dgraph-io/dgraph/x"
)
type Wal struct {
wals *store.Store
id uint64
}
func Init(walStore *store.Store, id uint64) *Wal {
return &Wal{wals: walStore, id: id}
}
func (w *Wal) snapshotKey(gid uint32) []byte {
b := make([]byte, 14)
binary.BigEndian.PutUint64(b[0:8], w.id)
copy(b[8:10], []byte("ss"))
binary.BigEndian.PutUint32(b[10:14], gid)
return b
}
func (w *Wal) hardStateKey(gid uint32) []byte {
b := make([]byte, 14)
binary.BigEndian.PutUint64(b[0:8], w.id)
copy(b[8:10], []byte("hs"))
binary.BigEndian.PutUint32(b[10:14], gid)
return b
}
func (w *Wal) entryKey(gid uint32, term, idx uint64) []byte {
b := make([]byte, 28)
binary.BigEndian.PutUint64(b[0:8], w.id)
binary.BigEndian.PutUint32(b[8:12], gid)
binary.BigEndian.PutUint64(b[12:20], term)
binary.BigEndian.PutUint64(b[20:28], idx)
return b
}
func (w *Wal) prefix(gid uint32) []byte {
b := make([]byte, 12)
binary.BigEndian.PutUint64(b[0:8], w.id)
binary.BigEndian.PutUint32(b[8:12], gid)
return b
}
// Store stores the snapshot, hardstate and entries for a given RAFT group.
func (w *Wal) Store(gid uint32, s raftpb.Snapshot, h raftpb.HardState, es []raftpb.Entry) error {
b := w.wals.NewWriteBatch()
defer b.Destroy()
if !raft.IsEmptySnap(s) {
data, err := s.Marshal()
if err != nil {
return x.Wrapf(err, "wal.Store: While marshal snapshot")
}
b.Put(w.snapshotKey(gid), data)
}
if !raft.IsEmptyHardState(h) {
data, err := h.Marshal()
if err != nil {
return x.Wrapf(err, "wal.Store: While marshal hardstate")
}
b.Put(w.hardStateKey(gid), data)
}
var t, i uint64
for _, e := range es {
t, i = e.Term, e.Index
data, err := e.Marshal()
if err != nil {
return x.Wrapf(err, "wal.Store: While marshal entry")
}
k := w.entryKey(gid, e.Term, e.Index)
b.Put(k, data)
}
// If we get no entries, then the default value of t and i would be zero. That would
// end up deleting all the previous valid raft entry logs. This check avoids that.
if t > 0 || i > 0 {
// Delete all keys above this index.
start := w.entryKey(gid, t, i+1)
prefix := w.prefix(gid)
itr := w.wals.NewIterator()
defer itr.Close()
for itr.Seek(start); itr.ValidForPrefix(prefix); itr.Next() {
b.Delete(itr.Key().Data())
}
}
err := w.wals.WriteBatch(b)
return x.Wrapf(err, "wal.Store: While WriteBatch")
}
func (w *Wal) Snapshot(gid uint32) (snap raftpb.Snapshot, rerr error) {
slice, err := w.wals.Get(w.snapshotKey(gid))
if err != nil || slice == nil {
return snap, x.Wrapf(err, "While getting snapshot")
}
rerr = x.Wrapf(snap.Unmarshal(slice.Data()), "While unmarshal snapshot")
slice.Free()
return
}
func (w *Wal) HardState(gid uint32) (hd raftpb.HardState, rerr error) {
slice, err := w.wals.Get(w.hardStateKey(gid))
if err != nil || slice == nil {
return hd, x.Wrapf(err, "While getting hardstate")
}
rerr = x.Wrapf(hd.Unmarshal(slice.Data()), "While unmarshal hardstate")
slice.Free()
return
}
func (w *Wal) Entries(gid uint32, fromTerm, fromIndex uint64) (es []raftpb.Entry, rerr error) {
start := w.entryKey(gid, fromTerm, fromIndex)
prefix := w.prefix(gid)
itr := w.wals.NewIterator()
defer itr.Close()
for itr.Seek(start); itr.ValidForPrefix(prefix); itr.Next() {
data := itr.Value().Data()
var e raftpb.Entry
if err := e.Unmarshal(data); err != nil {
return es, x.Wrapf(err, "While unmarshal raftpb.Entry")
}
es = append(es, e)
}
return
}