forked from hypermodeinc/dgraph
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathoracle.go
247 lines (214 loc) · 5.82 KB
/
oracle.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
/*
* Copyright 2017-2018 Dgraph Labs, Inc. and Contributors
*
* 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 posting
import (
"context"
"math"
"sync"
"sync/atomic"
"time"
"github.com/dgraph-io/dgraph/protos/pb"
"github.com/dgraph-io/dgraph/x"
)
var o *oracle
// TODO: Oracle should probably be located in worker package, instead of posting
// package now that we don't run inSnapshot anymore.
func Oracle() *oracle {
return o
}
func init() {
o = new(oracle)
o.init()
}
type Txn struct {
StartTs uint64
// atomic
shouldAbort uint32
// Fields which can changed after init
sync.Mutex
// Deltas keeps track of the posting list keys, and whether they should be considered for
// conflict detection or not. When a txn is marked committed or aborted, we use the keys stored
// here to determine which posting lists to get and update.
deltas map[string]struct{}
// Keeps track of conflict keys that should be used to determine if this
// transaction conflicts with another.
conflicts map[string]struct{}
// Keeps track of last update wall clock. We use this fact later to
// determine unhealthy, stale txns.
lastUpdate time.Time
cache *LocalCache
}
func NewTxn(startTs uint64) *Txn {
return &Txn{
StartTs: startTs,
cache: NewLocalCache(),
lastUpdate: time.Now(),
}
}
func (txn *Txn) Get(key []byte) (*List, error) {
return txn.cache.Get(key)
}
func (txn *Txn) Store(pl *List) *List {
return txn.cache.Set(string(pl.key), pl)
}
type oracle struct {
x.SafeMutex
// max start ts given out by Zero. Do not use mutex on this, only use atomics.
maxAssigned uint64
// Keeps track of all the startTs we have seen so far, based on the mutations. Then as
// transactions are committed or aborted, we delete entries from the startTs map. When taking a
// snapshot, we need to know the minimum start ts present in the map, which represents a
// mutation which has not yet been committed or aborted. As we iterate over entries, we should
// only discard those whose StartTs is below this minimum pending start ts.
pendingTxns map[uint64]*Txn
// Used for waiting logic for transactions with startTs > maxpending so that we don't read an
// uncommitted transaction.
waiters map[uint64][]chan struct{}
}
func (o *oracle) init() {
o.waiters = make(map[uint64][]chan struct{})
o.pendingTxns = make(map[uint64]*Txn)
}
func (o *oracle) RegisterStartTs(ts uint64) *Txn {
o.Lock()
defer o.Unlock()
txn, ok := o.pendingTxns[ts]
if ok {
txn.lastUpdate = time.Now()
} else {
txn = NewTxn(ts)
o.pendingTxns[ts] = txn
}
return txn
}
func (o *oracle) CacheAt(ts uint64) *LocalCache {
o.RLock()
defer o.RUnlock()
txn, ok := o.pendingTxns[ts]
if !ok {
return nil
}
return txn.cache
}
// MinPendingStartTs returns the min start ts which is currently pending a commit or abort decision.
func (o *oracle) MinPendingStartTs() uint64 {
o.RLock()
defer o.RUnlock()
min := uint64(math.MaxUint64)
for ts := range o.pendingTxns {
if ts < min {
min = ts
}
}
return min
}
func (o *oracle) TxnOlderThan(dur time.Duration) (res []uint64) {
o.RLock()
defer o.RUnlock()
cutoff := time.Now().Add(-dur)
for startTs, txn := range o.pendingTxns {
if txn.lastUpdate.Before(cutoff) {
res = append(res, startTs)
}
}
return res
}
func (o *oracle) addToWaiters(startTs uint64) (chan struct{}, bool) {
if startTs <= o.MaxAssigned() {
return nil, false
}
o.Lock()
defer o.Unlock()
// Check again after acquiring lock, because o.waiters is being processed serially. So, if we
// don't check here, then it's possible that we add to waiters here, but MaxAssigned has already
// moved past startTs.
if startTs <= o.MaxAssigned() {
return nil, false
}
ch := make(chan struct{})
o.waiters[startTs] = append(o.waiters[startTs], ch)
return ch, true
}
func (o *oracle) MaxAssigned() uint64 {
return atomic.LoadUint64(&o.maxAssigned)
}
func (o *oracle) WaitForTs(ctx context.Context, startTs uint64) error {
ch, ok := o.addToWaiters(startTs)
if !ok {
return nil
}
select {
case <-ch:
return nil
case <-ctx.Done():
return ctx.Err()
}
}
func (o *oracle) ProcessDelta(delta *pb.OracleDelta) {
o.Lock()
defer o.Unlock()
for _, txn := range delta.Txns {
delete(o.pendingTxns, txn.StartTs)
}
curMax := o.MaxAssigned()
if delta.MaxAssigned < curMax {
return
}
// Notify the waiting cattle.
for startTs, toNotify := range o.waiters {
if startTs > delta.MaxAssigned {
continue
}
for _, ch := range toNotify {
close(ch)
}
delete(o.waiters, startTs)
}
x.AssertTrue(atomic.CompareAndSwapUint64(&o.maxAssigned, curMax, delta.MaxAssigned))
}
func (o *oracle) ResetTxns() {
o.Lock()
defer o.Unlock()
o.pendingTxns = make(map[uint64]*Txn)
}
func (o *oracle) GetTxn(startTs uint64) *Txn {
o.RLock()
defer o.RUnlock()
return o.pendingTxns[startTs]
}
func (txn *Txn) matchesDelta(ok func(key []byte) bool) bool {
txn.Lock()
defer txn.Unlock()
for key := range txn.deltas {
if ok([]byte(key)) {
return true
}
}
return false
}
// IterateTxns returns a list of start timestamps for currently pending transactions, which match
// the provided function.
func (o *oracle) IterateTxns(ok func(key []byte) bool) []uint64 {
o.RLock()
defer o.RUnlock()
var timestamps []uint64
for startTs, txn := range o.pendingTxns {
if txn.matchesDelta(ok) {
timestamps = append(timestamps, startTs)
}
}
return timestamps
}