diff --git a/contrib/scripts/functions.sh b/contrib/scripts/functions.sh index f44b062b87e..23e46159409 100755 --- a/contrib/scripts/functions.sh +++ b/contrib/scripts/functions.sh @@ -10,7 +10,7 @@ function runCluster { go build . && go install . && md5sum dgraph $GOPATH/bin/dgraph DATA="/tmp/dg" docker-compose up --force-recreate --remove-orphans --detach popd - $basedir/contrib/wait-for-it.sh localhost:6080 - $basedir/contrib/wait-for-it.sh localhost:9180 + $basedir/contrib/wait-for-it.sh -t 60 localhost:6080 + $basedir/contrib/wait-for-it.sh -t 60 localhost:9180 sleep 10 # Sleep 10 seconds to get things ready. } diff --git a/dgraph/cmd/zero/oracle.go b/dgraph/cmd/zero/oracle.go index 8ab496a847c..6452362c648 100644 --- a/dgraph/cmd/zero/oracle.go +++ b/dgraph/cmd/zero/oracle.go @@ -28,9 +28,9 @@ type Oracle struct { x.SafeMutex commits map[uint64]uint64 // startTs -> commitTs // TODO: Check if we need LRU. - rowCommit map[string]uint64 // fp(key) -> commitTs. Used to detect conflict. - aborts map[uint64]struct{} // key is startTs - maxPending uint64 // max transaction startTs given out by us. + rowCommit map[string]uint64 // fp(key) -> commitTs. Used to detect conflict. + aborts map[uint64]struct{} // key is startTs + maxAssigned uint64 // max transaction assigned by us. // timestamp at the time of start of server or when it became leader. Used to detect conflicts. tmax uint64 @@ -131,7 +131,7 @@ func (o *Oracle) currentState() *intern.OracleDelta { for abort := range o.aborts { resp.Aborts = append(resp.Aborts, abort) } - resp.MaxPending = o.maxPending + resp.MaxAssigned = o.maxAssigned return resp } @@ -169,8 +169,8 @@ func (o *Oracle) sendDeltasToSubscribers() { slurp_loop: for { // Consume tctx. - if update.MaxPending > delta.MaxPending { - delta.MaxPending = update.MaxPending + if update.MaxAssigned > delta.MaxAssigned { + delta.MaxAssigned = update.MaxAssigned } for _, startTs := range update.Aborts { delta.Aborts = append(delta.Aborts, startTs) @@ -244,19 +244,19 @@ func (o *Oracle) storePending(ids *api.AssignedIds) { // Wait to finish up processing everything before start id. o.doneUntil.WaitForMark(context.Background(), ids.EndId) // Now send it out to updates. - o.updates <- &intern.OracleDelta{MaxPending: ids.EndId} + o.updates <- &intern.OracleDelta{MaxAssigned: ids.EndId} o.Lock() defer o.Unlock() max := ids.EndId - if o.maxPending < max { - o.maxPending = max + if o.maxAssigned < max { + o.maxAssigned = max } } func (o *Oracle) MaxPending() uint64 { o.RLock() defer o.RUnlock() - return o.maxPending + return o.maxAssigned } var errConflict = errors.New("Transaction conflict") @@ -317,8 +317,7 @@ func (s *Server) commit(ctx context.Context, src *api.TxnContext) error { // map to be keyed by uint64, which would be cheaper. But, unsure about the repurcussions of // that. It would save some memory. So, worth a try. - var num intern.Num - num.Val = 1 + num := intern.Num{Val: 1} assigned, err := s.lease(ctx, &num, true) if err != nil { return err diff --git a/dgraph/docker-compose.yml b/dgraph/docker-compose.yml index 9ed2f9b755a..e17d12f1c2c 100644 --- a/dgraph/docker-compose.yml +++ b/dgraph/docker-compose.yml @@ -20,42 +20,42 @@ services: # - type: bind # source: "${DATA}" # target: /data - zero2: - image: debian:latest - container_name: bank-dg0.2 - command: /gobin/dgraph zero --my=zero2:5080 --replicas 3 --peer=zero1:5080 --idx 2 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - zero3: - image: debian:latest - container_name: bank-dg0.3 - command: /gobin/dgraph zero --my=zero3:5080 --replicas 3 --peer=zero1:5080 --idx 3 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - zero4: - image: debian:latest - container_name: bank-dg0.4 - command: /gobin/dgraph zero --my=zero4:5080 --replicas 3 --peer=zero1:5080 --idx 4 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - zero5: - image: debian:latest - container_name: bank-dg0.5 - command: /gobin/dgraph zero --my=zero5:5080 --replicas 3 --peer=zero1:5080 --idx 5 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true + # zero2: + # image: debian:latest + # container_name: bank-dg0.2 + # command: /gobin/dgraph zero --my=zero2:5080 --replicas 3 --peer=zero1:5080 --idx 2 + # volumes: + # - type: bind + # source: $GOPATH/bin + # target: /gobin + # read_only: true + # zero3: + # image: debian:latest + # container_name: bank-dg0.3 + # command: /gobin/dgraph zero --my=zero3:5080 --replicas 3 --peer=zero1:5080 --idx 3 + # volumes: + # - type: bind + # source: $GOPATH/bin + # target: /gobin + # read_only: true + # zero4: + # image: debian:latest + # container_name: bank-dg0.4 + # command: /gobin/dgraph zero --my=zero4:5080 --replicas 3 --peer=zero1:5080 --idx 4 + # volumes: + # - type: bind + # source: $GOPATH/bin + # target: /gobin + # read_only: true + # zero5: + # image: debian:latest + # container_name: bank-dg0.5 + # command: /gobin/dgraph zero --my=zero5:5080 --replicas 3 --peer=zero1:5080 --idx 5 + # volumes: + # - type: bind + # source: $GOPATH/bin + # target: /gobin + # read_only: true dg1: image: debian:latest @@ -73,6 +73,7 @@ services: - 8180:8180 - 9180:9180 command: /gobin/dgraph server --my=dg1:7180 --lru_mb=1024 --zero=zero1:5080 -o 100 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 + dg2: image: debian:latest container_name: bank-dg2 @@ -108,37 +109,37 @@ services: command: /gobin/dgraph server --my=dg3:7183 --lru_mb=1024 --zero=zero1:5080 -o 103 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 - dg4: - image: debian:latest - container_name: bank-dg4 - working_dir: /data/dg4 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - # - type: bind - # source: "${DATA}" - # target: /data - ports: - - 8184:8184 - - 9184:9184 - command: /gobin/dgraph server --my=dg4:7184 --lru_mb=1024 --zero=zero1:5080 -o 104 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 + # dg4: + # image: debian:latest + # container_name: bank-dg4 + # working_dir: /data/dg4 + # volumes: + # - type: bind + # source: $GOPATH/bin + # target: /gobin + # read_only: true + # # - type: bind + # # source: "${DATA}" + # # target: /data + # ports: + # - 8184:8184 + # - 9184:9184 + # command: /gobin/dgraph server --my=dg4:7184 --lru_mb=1024 --zero=zero1:5080 -o 104 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 - dg5: - image: debian:latest - container_name: bank-dg5 - working_dir: /data/dg5 - volumes: - - type: bind - source: $GOPATH/bin - target: /gobin - read_only: true - # - type: bind - # source: "${DATA}" - # target: /data - ports: - - 8185:8185 - - 9185:9185 - command: /gobin/dgraph server --my=dg5:7185 --lru_mb=1024 --zero=zero1:5080 -o 105 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 + # dg5: + # image: debian:latest + # container_name: bank-dg5 + # working_dir: /data/dg5 + # volumes: + # - type: bind + # source: $GOPATH/bin + # target: /gobin + # read_only: true + # # - type: bind + # # source: "${DATA}" + # # target: /data + # ports: + # - 8185:8185 + # - 9185:9185 + # command: /gobin/dgraph server --my=dg5:7185 --lru_mb=1024 --zero=zero1:5080 -o 105 --expose_trace --trace 1.0 --profile_mode block --block_rate 10 diff --git a/posting/mvcc.go b/posting/mvcc.go index 23ff10f6af7..bb7078fb591 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -29,17 +29,15 @@ var ( txnMarks *x.WaterMark // Used to find out till what RAFT index we can snapshot entries. ) +type transactions struct { + x.SafeMutex + m map[uint64]*Txn +} + func init() { txns = new(transactions) txns.m = make(map[uint64]*Txn) - txnMarks = &x.WaterMark{Name: "Transaction watermark"} - txnMarks.Init() -} - -func TxnMarks() *x.WaterMark { - return txnMarks } - func Txns() *transactions { return txns } @@ -52,6 +50,8 @@ type delta struct { posting *intern.Posting checkConflict bool // Check conflict detection. } + +// TODO: This structure can be merged back into Oracle. type Txn struct { StartTs uint64 @@ -66,11 +66,7 @@ type Txn struct { nextKeyIdx int } -type transactions struct { - x.SafeMutex - m map[uint64]*Txn -} - +// TODO: What is this for? func (t *transactions) MinTs() uint64 { t.Lock() var minTs uint64 @@ -94,28 +90,9 @@ func (t *transactions) MinTs() uint64 { return minTs } -func (t *transactions) TxnsSinceSnapshot(pending uint64) []uint64 { - lastSnapshotIdx := TxnMarks().DoneUntil() - var timestamps []uint64 - t.Lock() - defer t.Unlock() - var oldest float64 = 0.2 * float64(pending) - for _, txn := range t.m { - index := txn.startIdx() - // We abort oldest 20% of the transactions. - if index-lastSnapshotIdx <= uint64(oldest) { - timestamps = append(timestamps, txn.StartTs) - } - } - return timestamps -} - func (t *transactions) Reset() { t.Lock() defer t.Unlock() - for _, txn := range t.m { - txn.done() - } t.m = make(map[uint64]*Txn) } @@ -158,21 +135,9 @@ func (t *transactions) Get(startTs uint64) *Txn { func (t *transactions) Done(startTs uint64) { t.Lock() defer t.Unlock() - txn, ok := t.m[startTs] - if !ok { - return - } - txn.done() delete(t.m, startTs) } -func (t *Txn) done() { - t.Lock() - defer t.Unlock() - // All indices should have been added by now. - TxnMarks().DoneMany(t.Indices) -} - // LastIndex returns the index of last prewrite proposal associated with // the transaction. func (t *Txn) LastIndex() uint64 { diff --git a/posting/oracle.go b/posting/oracle.go index 0cdc1f8b3c2..51b09a90841 100644 --- a/posting/oracle.go +++ b/posting/oracle.go @@ -9,6 +9,8 @@ package posting import ( "context" + "math" + "time" "github.com/dgraph-io/dgraph/protos/intern" "github.com/dgraph-io/dgraph/x" @@ -27,12 +29,21 @@ func init() { type oracle struct { x.SafeMutex + + // TODO: Remove commits and aborts map from here. We don't need this, if we're doing transaction + // tracking correctly and applying the txn status back to posting lists correctly. commits map[uint64]uint64 // startTs => commitTs map aborts map[uint64]struct{} // key is startTs - // We know for sure that transactions with startTs <= maxpending have either been - // aborted/committed. - maxpending uint64 + // max start ts given out by Zero. + 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. + pendingStartTs map[uint64]time.Time // Used for waiting logic for transactions with startTs > maxpending so that we don't read an // uncommitted transaction. @@ -43,6 +54,7 @@ func (o *oracle) init() { o.commits = make(map[uint64]uint64) o.aborts = make(map[uint64]struct{}) o.waiters = make(map[uint64][]chan struct{}) + o.pendingStartTs = make(map[uint64]time.Time) } func (o *oracle) Done(startTs uint64) { @@ -65,10 +77,42 @@ func (o *oracle) Aborted(startTs uint64) bool { return ok } +func (o *oracle) RegisterStartTs(ts uint64) { + o.Lock() + defer o.Unlock() + o.pendingStartTs[ts] = time.Now() +} + +// 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.pendingStartTs { + 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, clockTs := range o.pendingStartTs { + if clockTs.Before(cutoff) { + res = append(res, startTs) + } + } + return res +} + func (o *oracle) addToWaiters(startTs uint64) (chan struct{}, bool) { o.Lock() defer o.Unlock() - if o.maxpending >= startTs { + if o.maxAssigned >= startTs { return nil, false } ch := make(chan struct{}) @@ -79,13 +123,13 @@ func (o *oracle) addToWaiters(startTs uint64) (chan struct{}, bool) { func (o *oracle) MaxPending() uint64 { o.RLock() defer o.RUnlock() - return o.maxpending + return o.maxAssigned } func (o *oracle) SetMaxPending(maxPending uint64) { o.Lock() defer o.Unlock() - o.maxpending = maxPending + o.maxAssigned = maxPending } func (o *oracle) CurrentState() *intern.OracleDelta { @@ -115,20 +159,28 @@ func (o *oracle) WaitForTs(ctx context.Context, startTs uint64) error { } } -func (o *oracle) ProcessOracleDelta(od *intern.OracleDelta) { +func (o *oracle) ProcessOracleDelta(delta *intern.OracleDelta) { o.Lock() defer o.Unlock() - for startTs, commitTs := range od.Commits { + for startTs, commitTs := range delta.Commits { o.commits[startTs] = commitTs + delete(o.pendingStartTs, startTs) } - for _, startTs := range od.Aborts { + for _, startTs := range delta.Aborts { o.aborts[startTs] = struct{}{} + delete(o.pendingStartTs, startTs) } - if od.MaxPending <= o.maxpending { + // We should always be moving forward with Zero and with Raft logs. A move + // back should not be possible, unless there's a bigger issue in + // understanding or the codebase. + if delta.MaxAssigned == 0 { return } + x.AssertTrue(delta.MaxAssigned >= o.maxAssigned) + + // Notify the waiting cattle. for startTs, toNotify := range o.waiters { - if startTs > od.MaxPending { + if startTs > delta.MaxAssigned { continue } for _, ch := range toNotify { @@ -136,5 +188,5 @@ func (o *oracle) ProcessOracleDelta(od *intern.OracleDelta) { } delete(o.waiters, startTs) } - o.maxpending = od.MaxPending + o.maxAssigned = delta.MaxAssigned } diff --git a/protos/intern/internal.pb.go b/protos/intern/internal.pb.go index 6291afe9f42..26b672cda33 100644 --- a/protos/intern/internal.pb.go +++ b/protos/intern/internal.pb.go @@ -28,6 +28,7 @@ DirectedEdge Mutations KeyValues + Snapshot Proposal KVS KV @@ -141,7 +142,7 @@ var Posting_ValType_value = map[string]int32{ func (x Posting_ValType) String() string { return proto.EnumName(Posting_ValType_name, int32(x)) } -func (Posting_ValType) EnumDescriptor() ([]byte, []int) { return fileDescriptorInternal, []int{23, 0} } +func (Posting_ValType) EnumDescriptor() ([]byte, []int) { return fileDescriptorInternal, []int{24, 0} } type Posting_PostingType int32 @@ -166,7 +167,7 @@ func (x Posting_PostingType) String() string { return proto.EnumName(Posting_PostingType_name, int32(x)) } func (Posting_PostingType) EnumDescriptor() ([]byte, []int) { - return fileDescriptorInternal, []int{23, 1} + return fileDescriptorInternal, []int{24, 1} } type SchemaUpdate_Directive int32 @@ -195,7 +196,7 @@ func (x SchemaUpdate_Directive) String() string { return proto.EnumName(SchemaUpdate_Directive_name, int32(x)) } func (SchemaUpdate_Directive) EnumDescriptor() ([]byte, []int) { - return fileDescriptorInternal, []int{33, 0} + return fileDescriptorInternal, []int{34, 0} } type ExportPayload_Status int32 @@ -224,7 +225,7 @@ func (x ExportPayload_Status) String() string { return proto.EnumName(ExportPayload_Status_name, int32(x)) } func (ExportPayload_Status) EnumDescriptor() ([]byte, []int) { - return fileDescriptorInternal, []int{36, 0} + return fileDescriptorInternal, []int{37, 0} } type List struct { @@ -1131,6 +1132,38 @@ func (m *KeyValues) GetKv() []*KV { return nil } +type Snapshot struct { + Context *RaftContext `protobuf:"bytes,1,opt,name=context" json:"context,omitempty"` + Index uint64 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"` + MinPendingStartTs uint64 `protobuf:"varint,3,opt,name=min_pending_start_ts,json=minPendingStartTs,proto3" json:"min_pending_start_ts,omitempty"` +} + +func (m *Snapshot) Reset() { *m = Snapshot{} } +func (m *Snapshot) String() string { return proto.CompactTextString(m) } +func (*Snapshot) ProtoMessage() {} +func (*Snapshot) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{20} } + +func (m *Snapshot) GetContext() *RaftContext { + if m != nil { + return m.Context + } + return nil +} + +func (m *Snapshot) GetIndex() uint64 { + if m != nil { + return m.Index + } + return 0 +} + +func (m *Snapshot) GetMinPendingStartTs() uint64 { + if m != nil { + return m.MinPendingStartTs + } + return 0 +} + type Proposal struct { DeprecatedId uint32 `protobuf:"varint,1,opt,name=deprecated_id,json=deprecatedId,proto3" json:"deprecated_id,omitempty"` DeprecatedTxnContext *api.TxnContext `protobuf:"bytes,3,opt,name=deprecated_txn_context,json=deprecatedTxnContext" json:"deprecated_txn_context,omitempty"` @@ -1140,12 +1173,13 @@ type Proposal struct { CleanPredicate string `protobuf:"bytes,6,opt,name=clean_predicate,json=cleanPredicate,proto3" json:"clean_predicate,omitempty"` Key string `protobuf:"bytes,7,opt,name=key,proto3" json:"key,omitempty"` Delta *OracleDelta `protobuf:"bytes,8,opt,name=delta" json:"delta,omitempty"` + Snapshot *Snapshot `protobuf:"bytes,9,opt,name=snapshot" json:"snapshot,omitempty"` } func (m *Proposal) Reset() { *m = Proposal{} } func (m *Proposal) String() string { return proto.CompactTextString(m) } func (*Proposal) ProtoMessage() {} -func (*Proposal) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{20} } +func (*Proposal) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{21} } func (m *Proposal) GetDeprecatedId() uint32 { if m != nil { @@ -1203,6 +1237,13 @@ func (m *Proposal) GetDelta() *OracleDelta { return nil } +func (m *Proposal) GetSnapshot() *Snapshot { + if m != nil { + return m.Snapshot + } + return nil +} + type KVS struct { Kv []*KV `protobuf:"bytes,1,rep,name=kv" json:"kv,omitempty"` } @@ -1210,7 +1251,7 @@ type KVS struct { func (m *KVS) Reset() { *m = KVS{} } func (m *KVS) String() string { return proto.CompactTextString(m) } func (*KVS) ProtoMessage() {} -func (*KVS) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{21} } +func (*KVS) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{22} } func (m *KVS) GetKv() []*KV { if m != nil { @@ -1229,7 +1270,7 @@ type KV struct { func (m *KV) Reset() { *m = KV{} } func (m *KV) String() string { return proto.CompactTextString(m) } func (*KV) ProtoMessage() {} -func (*KV) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{22} } +func (*KV) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{23} } func (m *KV) GetKey() []byte { if m != nil { @@ -1277,7 +1318,7 @@ type Posting struct { func (m *Posting) Reset() { *m = Posting{} } func (m *Posting) String() string { return proto.CompactTextString(m) } func (*Posting) ProtoMessage() {} -func (*Posting) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{23} } +func (*Posting) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{24} } func (m *Posting) GetUid() uint64 { if m != nil { @@ -1359,7 +1400,7 @@ type PostingList struct { func (m *PostingList) Reset() { *m = PostingList{} } func (m *PostingList) String() string { return proto.CompactTextString(m) } func (*PostingList) ProtoMessage() {} -func (*PostingList) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{24} } +func (*PostingList) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{25} } func (m *PostingList) GetPostings() []*Posting { if m != nil { @@ -1397,7 +1438,7 @@ type FacetParam struct { func (m *FacetParam) Reset() { *m = FacetParam{} } func (m *FacetParam) String() string { return proto.CompactTextString(m) } func (*FacetParam) ProtoMessage() {} -func (*FacetParam) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{25} } +func (*FacetParam) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{26} } func (m *FacetParam) GetKey() string { if m != nil { @@ -1421,7 +1462,7 @@ type FacetParams struct { func (m *FacetParams) Reset() { *m = FacetParams{} } func (m *FacetParams) String() string { return proto.CompactTextString(m) } func (*FacetParams) ProtoMessage() {} -func (*FacetParams) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{26} } +func (*FacetParams) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{27} } func (m *FacetParams) GetAllKeys() bool { if m != nil { @@ -1444,7 +1485,7 @@ type Facets struct { func (m *Facets) Reset() { *m = Facets{} } func (m *Facets) String() string { return proto.CompactTextString(m) } func (*Facets) ProtoMessage() {} -func (*Facets) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{27} } +func (*Facets) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{28} } func (m *Facets) GetFacets() []*api.Facet { if m != nil { @@ -1460,7 +1501,7 @@ type FacetsList struct { func (m *FacetsList) Reset() { *m = FacetsList{} } func (m *FacetsList) String() string { return proto.CompactTextString(m) } func (*FacetsList) ProtoMessage() {} -func (*FacetsList) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{28} } +func (*FacetsList) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{29} } func (m *FacetsList) GetFacetsList() []*Facets { if m != nil { @@ -1478,7 +1519,7 @@ type Function struct { func (m *Function) Reset() { *m = Function{} } func (m *Function) String() string { return proto.CompactTextString(m) } func (*Function) ProtoMessage() {} -func (*Function) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{29} } +func (*Function) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{30} } func (m *Function) GetName() string { if m != nil { @@ -1511,7 +1552,7 @@ type FilterTree struct { func (m *FilterTree) Reset() { *m = FilterTree{} } func (m *FilterTree) String() string { return proto.CompactTextString(m) } func (*FilterTree) ProtoMessage() {} -func (*FilterTree) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{30} } +func (*FilterTree) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{31} } func (m *FilterTree) GetOp() string { if m != nil { @@ -1545,7 +1586,7 @@ type SchemaRequest struct { func (m *SchemaRequest) Reset() { *m = SchemaRequest{} } func (m *SchemaRequest) String() string { return proto.CompactTextString(m) } func (*SchemaRequest) ProtoMessage() {} -func (*SchemaRequest) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{31} } +func (*SchemaRequest) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{32} } func (m *SchemaRequest) GetGroupId() uint32 { if m != nil { @@ -1575,7 +1616,7 @@ type SchemaResult struct { func (m *SchemaResult) Reset() { *m = SchemaResult{} } func (m *SchemaResult) String() string { return proto.CompactTextString(m) } func (*SchemaResult) ProtoMessage() {} -func (*SchemaResult) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{32} } +func (*SchemaResult) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{33} } func (m *SchemaResult) GetSchema() []*api.SchemaNode { if m != nil { @@ -1598,7 +1639,7 @@ type SchemaUpdate struct { func (m *SchemaUpdate) Reset() { *m = SchemaUpdate{} } func (m *SchemaUpdate) String() string { return proto.CompactTextString(m) } func (*SchemaUpdate) ProtoMessage() {} -func (*SchemaUpdate) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{33} } +func (*SchemaUpdate) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{34} } func (m *SchemaUpdate) GetPredicate() string { if m != nil { @@ -1667,7 +1708,7 @@ type MapEntry struct { func (m *MapEntry) Reset() { *m = MapEntry{} } func (m *MapEntry) String() string { return proto.CompactTextString(m) } func (*MapEntry) ProtoMessage() {} -func (*MapEntry) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{34} } +func (*MapEntry) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{35} } func (m *MapEntry) GetKey() []byte { if m != nil { @@ -1700,7 +1741,7 @@ type MovePredicatePayload struct { func (m *MovePredicatePayload) Reset() { *m = MovePredicatePayload{} } func (m *MovePredicatePayload) String() string { return proto.CompactTextString(m) } func (*MovePredicatePayload) ProtoMessage() {} -func (*MovePredicatePayload) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{35} } +func (*MovePredicatePayload) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{36} } func (m *MovePredicatePayload) GetPredicate() string { if m != nil { @@ -1743,7 +1784,7 @@ type ExportPayload struct { func (m *ExportPayload) Reset() { *m = ExportPayload{} } func (m *ExportPayload) String() string { return proto.CompactTextString(m) } func (*ExportPayload) ProtoMessage() {} -func (*ExportPayload) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{36} } +func (*ExportPayload) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{37} } func (m *ExportPayload) GetReqId() uint64 { if m != nil { @@ -1774,15 +1815,15 @@ func (m *ExportPayload) GetReadTs() uint64 { } type OracleDelta struct { - Commits map[uint64]uint64 `protobuf:"bytes,1,rep,name=commits" json:"commits,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` - Aborts []uint64 `protobuf:"varint,2,rep,packed,name=aborts" json:"aborts,omitempty"` - MaxPending uint64 `protobuf:"varint,3,opt,name=max_pending,json=maxPending,proto3" json:"max_pending,omitempty"` + Commits map[uint64]uint64 `protobuf:"bytes,1,rep,name=commits" json:"commits,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Aborts []uint64 `protobuf:"varint,2,rep,packed,name=aborts" json:"aborts,omitempty"` + MaxAssigned uint64 `protobuf:"varint,3,opt,name=max_assigned,json=maxAssigned,proto3" json:"max_assigned,omitempty"` } func (m *OracleDelta) Reset() { *m = OracleDelta{} } func (m *OracleDelta) String() string { return proto.CompactTextString(m) } func (*OracleDelta) ProtoMessage() {} -func (*OracleDelta) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{37} } +func (*OracleDelta) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{38} } func (m *OracleDelta) GetCommits() map[uint64]uint64 { if m != nil { @@ -1798,9 +1839,9 @@ func (m *OracleDelta) GetAborts() []uint64 { return nil } -func (m *OracleDelta) GetMaxPending() uint64 { +func (m *OracleDelta) GetMaxAssigned() uint64 { if m != nil { - return m.MaxPending + return m.MaxAssigned } return 0 } @@ -1812,7 +1853,7 @@ type TxnTimestamps struct { func (m *TxnTimestamps) Reset() { *m = TxnTimestamps{} } func (m *TxnTimestamps) String() string { return proto.CompactTextString(m) } func (*TxnTimestamps) ProtoMessage() {} -func (*TxnTimestamps) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{38} } +func (*TxnTimestamps) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{39} } func (m *TxnTimestamps) GetTs() []uint64 { if m != nil { @@ -1828,7 +1869,7 @@ type PeerResponse struct { func (m *PeerResponse) Reset() { *m = PeerResponse{} } func (m *PeerResponse) String() string { return proto.CompactTextString(m) } func (*PeerResponse) ProtoMessage() {} -func (*PeerResponse) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{39} } +func (*PeerResponse) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{40} } func (m *PeerResponse) GetStatus() bool { if m != nil { @@ -1845,7 +1886,7 @@ type RaftBatch struct { func (m *RaftBatch) Reset() { *m = RaftBatch{} } func (m *RaftBatch) String() string { return proto.CompactTextString(m) } func (*RaftBatch) ProtoMessage() {} -func (*RaftBatch) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{40} } +func (*RaftBatch) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{41} } func (m *RaftBatch) GetContext() *RaftContext { if m != nil { @@ -1868,7 +1909,7 @@ type Num struct { func (m *Num) Reset() { *m = Num{} } func (m *Num) String() string { return proto.CompactTextString(m) } func (*Num) ProtoMessage() {} -func (*Num) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{41} } +func (*Num) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{42} } func (m *Num) GetVal() uint64 { if m != nil { @@ -1885,7 +1926,7 @@ type SnapshotMeta struct { func (m *SnapshotMeta) Reset() { *m = SnapshotMeta{} } func (m *SnapshotMeta) String() string { return proto.CompactTextString(m) } func (*SnapshotMeta) ProtoMessage() {} -func (*SnapshotMeta) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{42} } +func (*SnapshotMeta) Descriptor() ([]byte, []int) { return fileDescriptorInternal, []int{43} } func (m *SnapshotMeta) GetClientTs() uint64 { if m != nil { @@ -1922,6 +1963,7 @@ func init() { proto.RegisterType((*DirectedEdge)(nil), "intern.DirectedEdge") proto.RegisterType((*Mutations)(nil), "intern.Mutations") proto.RegisterType((*KeyValues)(nil), "intern.KeyValues") + proto.RegisterType((*Snapshot)(nil), "intern.Snapshot") proto.RegisterType((*Proposal)(nil), "intern.Proposal") proto.RegisterType((*KVS)(nil), "intern.KVS") proto.RegisterType((*KV)(nil), "intern.KV") @@ -4083,6 +4125,44 @@ func (m *KeyValues) MarshalTo(dAtA []byte) (int, error) { return i, nil } +func (m *Snapshot) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *Snapshot) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.Context != nil { + dAtA[i] = 0xa + i++ + i = encodeVarintInternal(dAtA, i, uint64(m.Context.Size())) + n20, err := m.Context.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n20 + } + if m.Index != 0 { + dAtA[i] = 0x10 + i++ + i = encodeVarintInternal(dAtA, i, uint64(m.Index)) + } + if m.MinPendingStartTs != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintInternal(dAtA, i, uint64(m.MinPendingStartTs)) + } + return i, nil +} + func (m *Proposal) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -4107,21 +4187,21 @@ func (m *Proposal) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x12 i++ i = encodeVarintInternal(dAtA, i, uint64(m.Mutations.Size())) - n20, err := m.Mutations.MarshalTo(dAtA[i:]) + n21, err := m.Mutations.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n20 + i += n21 } if m.DeprecatedTxnContext != nil { dAtA[i] = 0x1a i++ i = encodeVarintInternal(dAtA, i, uint64(m.DeprecatedTxnContext.Size())) - n21, err := m.DeprecatedTxnContext.MarshalTo(dAtA[i:]) + n22, err := m.DeprecatedTxnContext.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n21 + i += n22 } if len(m.Kv) > 0 { for _, msg := range m.Kv { @@ -4139,11 +4219,11 @@ func (m *Proposal) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x2a i++ i = encodeVarintInternal(dAtA, i, uint64(m.State.Size())) - n22, err := m.State.MarshalTo(dAtA[i:]) + n23, err := m.State.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n22 + i += n23 } if len(m.CleanPredicate) > 0 { dAtA[i] = 0x32 @@ -4161,11 +4241,21 @@ func (m *Proposal) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x42 i++ i = encodeVarintInternal(dAtA, i, uint64(m.Delta.Size())) - n23, err := m.Delta.MarshalTo(dAtA[i:]) + n24, err := m.Delta.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n23 + i += n24 + } + if m.Snapshot != nil { + dAtA[i] = 0x4a + i++ + i = encodeVarintInternal(dAtA, i, uint64(m.Snapshot.Size())) + n25, err := m.Snapshot.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n25 } return i, nil } @@ -4579,11 +4669,11 @@ func (m *FilterTree) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintInternal(dAtA, i, uint64(m.Func.Size())) - n24, err := m.Func.MarshalTo(dAtA[i:]) + n26, err := m.Func.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n24 + i += n26 } return i, nil } @@ -4791,11 +4881,11 @@ func (m *MapEntry) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x1a i++ i = encodeVarintInternal(dAtA, i, uint64(m.Posting.Size())) - n25, err := m.Posting.MarshalTo(dAtA[i:]) + n27, err := m.Posting.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n25 + i += n27 } return i, nil } @@ -4835,11 +4925,11 @@ func (m *MovePredicatePayload) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0x22 i++ i = encodeVarintInternal(dAtA, i, uint64(m.State.Size())) - n26, err := m.State.MarshalTo(dAtA[i:]) + n28, err := m.State.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n26 + i += n28 } return i, nil } @@ -4913,26 +5003,26 @@ func (m *OracleDelta) MarshalTo(dAtA []byte) (int, error) { } } if len(m.Aborts) > 0 { - dAtA28 := make([]byte, len(m.Aborts)*10) - var j27 int + dAtA30 := make([]byte, len(m.Aborts)*10) + var j29 int for _, num := range m.Aborts { for num >= 1<<7 { - dAtA28[j27] = uint8(uint64(num)&0x7f | 0x80) + dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j27++ + j29++ } - dAtA28[j27] = uint8(num) - j27++ + dAtA30[j29] = uint8(num) + j29++ } dAtA[i] = 0x12 i++ - i = encodeVarintInternal(dAtA, i, uint64(j27)) - i += copy(dAtA[i:], dAtA28[:j27]) + i = encodeVarintInternal(dAtA, i, uint64(j29)) + i += copy(dAtA[i:], dAtA30[:j29]) } - if m.MaxPending != 0 { + if m.MaxAssigned != 0 { dAtA[i] = 0x18 i++ - i = encodeVarintInternal(dAtA, i, uint64(m.MaxPending)) + i = encodeVarintInternal(dAtA, i, uint64(m.MaxAssigned)) } return i, nil } @@ -4953,21 +5043,21 @@ func (m *TxnTimestamps) MarshalTo(dAtA []byte) (int, error) { var l int _ = l if len(m.Ts) > 0 { - dAtA30 := make([]byte, len(m.Ts)*10) - var j29 int + dAtA32 := make([]byte, len(m.Ts)*10) + var j31 int for _, num := range m.Ts { for num >= 1<<7 { - dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80) + dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j29++ + j31++ } - dAtA30[j29] = uint8(num) - j29++ + dAtA32[j31] = uint8(num) + j31++ } dAtA[i] = 0xa i++ - i = encodeVarintInternal(dAtA, i, uint64(j29)) - i += copy(dAtA[i:], dAtA30[:j29]) + i = encodeVarintInternal(dAtA, i, uint64(j31)) + i += copy(dAtA[i:], dAtA32[:j31]) } return i, nil } @@ -5019,21 +5109,21 @@ func (m *RaftBatch) MarshalTo(dAtA []byte) (int, error) { dAtA[i] = 0xa i++ i = encodeVarintInternal(dAtA, i, uint64(m.Context.Size())) - n31, err := m.Context.MarshalTo(dAtA[i:]) + n33, err := m.Context.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n31 + i += n33 } if m.Payload != nil { dAtA[i] = 0x12 i++ i = encodeVarintInternal(dAtA, i, uint64(m.Payload.Size())) - n32, err := m.Payload.MarshalTo(dAtA[i:]) + n34, err := m.Payload.MarshalTo(dAtA[i:]) if err != nil { return 0, err } - i += n32 + i += n34 } return i, nil } @@ -5614,6 +5704,22 @@ func (m *KeyValues) Size() (n int) { return n } +func (m *Snapshot) Size() (n int) { + var l int + _ = l + if m.Context != nil { + l = m.Context.Size() + n += 1 + l + sovInternal(uint64(l)) + } + if m.Index != 0 { + n += 1 + sovInternal(uint64(m.Index)) + } + if m.MinPendingStartTs != 0 { + n += 1 + sovInternal(uint64(m.MinPendingStartTs)) + } + return n +} + func (m *Proposal) Size() (n int) { var l int _ = l @@ -5650,6 +5756,10 @@ func (m *Proposal) Size() (n int) { l = m.Delta.Size() n += 1 + l + sovInternal(uint64(l)) } + if m.Snapshot != nil { + l = m.Snapshot.Size() + n += 1 + l + sovInternal(uint64(l)) + } return n } @@ -5984,8 +6094,8 @@ func (m *OracleDelta) Size() (n int) { } n += 1 + sovInternal(uint64(l)) + l } - if m.MaxPending != 0 { - n += 1 + sovInternal(uint64(m.MaxPending)) + if m.MaxAssigned != 0 { + n += 1 + sovInternal(uint64(m.MaxAssigned)) } return n } @@ -9731,6 +9841,127 @@ func (m *KeyValues) Unmarshal(dAtA []byte) error { } return nil } +func (m *Snapshot) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Snapshot: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Snapshot: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthInternal + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Context == nil { + m.Context = &RaftContext{} + } + if err := m.Context.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Index", wireType) + } + m.Index = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Index |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field MinPendingStartTs", wireType) + } + m.MinPendingStartTs = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.MinPendingStartTs |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipInternal(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthInternal + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *Proposal) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -10000,6 +10231,39 @@ func (m *Proposal) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 9: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Snapshot", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowInternal + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthInternal + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Snapshot == nil { + m.Snapshot = &Snapshot{} + } + if err := m.Snapshot.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipInternal(dAtA[iNdEx:]) @@ -12377,9 +12641,9 @@ func (m *OracleDelta) Unmarshal(dAtA []byte) error { } case 3: if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field MaxPending", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field MaxAssigned", wireType) } - m.MaxPending = 0 + m.MaxAssigned = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowInternal @@ -12389,7 +12653,7 @@ func (m *OracleDelta) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - m.MaxPending |= (uint64(b) & 0x7F) << shift + m.MaxAssigned |= (uint64(b) & 0x7F) << shift if b < 0x80 { break } @@ -12978,198 +13242,202 @@ var ( func init() { proto.RegisterFile("internal.proto", fileDescriptorInternal) } var fileDescriptorInternal = []byte{ - // 3075 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x39, 0x49, 0x6f, 0x23, 0xc7, - 0xd5, 0xec, 0x26, 0xd9, 0x6c, 0x3e, 0x92, 0x1a, 0xba, 0x3c, 0x9e, 0xa1, 0x69, 0x7f, 0xb2, 0x5c, - 0xe3, 0x6f, 0x46, 0xde, 0x64, 0x8f, 0x3c, 0x5e, 0x32, 0x89, 0x13, 0x68, 0x44, 0xce, 0x98, 0x1e, - 0x6d, 0x2e, 0x52, 0xe3, 0xd8, 0x87, 0x10, 0x25, 0x76, 0x89, 0x6a, 0xa8, 0xd9, 0xdd, 0xee, 0x6a, - 0x0a, 0x94, 0x8f, 0xb9, 0x05, 0xb9, 0xe5, 0x94, 0x73, 0x80, 0xdc, 0x02, 0x04, 0xc8, 0x31, 0x87, - 0xdc, 0x02, 0x24, 0x40, 0x80, 0x24, 0xb7, 0x1c, 0x13, 0xe7, 0x12, 0xe4, 0x96, 0x7f, 0x10, 0xd4, - 0xd2, 0x0b, 0x29, 0x8e, 0xac, 0x6c, 0x27, 0xf6, 0x7b, 0xf5, 0x5e, 0x2d, 0x6f, 0x7f, 0x8f, 0xb0, - 0xe2, 0xfa, 0x31, 0x8b, 0x7c, 0xea, 0x6d, 0x84, 0x51, 0x10, 0x07, 0xc8, 0x52, 0x70, 0xbb, 0x4a, - 0x43, 0x57, 0xa1, 0x70, 0x1b, 0x4a, 0x3b, 0x2e, 0x8f, 0x11, 0x82, 0xd2, 0xd4, 0x75, 0x78, 0xcb, - 0x58, 0x2b, 0xae, 0x5b, 0x44, 0x7e, 0xe3, 0x4f, 0xa0, 0x3a, 0xa0, 0xfc, 0xf4, 0x09, 0xf5, 0xa6, - 0x0c, 0x35, 0xa1, 0x78, 0x46, 0xbd, 0x96, 0xb1, 0x66, 0xac, 0xd7, 0x89, 0xf8, 0x44, 0x9b, 0x60, - 0x9f, 0x51, 0x6f, 0x18, 0x9f, 0x87, 0xac, 0x65, 0xae, 0x19, 0xeb, 0x2b, 0x9b, 0x37, 0x37, 0xd4, - 0x01, 0x1b, 0x07, 0x01, 0x8f, 0x5d, 0x7f, 0xbc, 0xf1, 0x84, 0x7a, 0x83, 0xf3, 0x90, 0x91, 0xca, - 0x99, 0xfa, 0xc0, 0xfb, 0x50, 0xeb, 0x47, 0xa3, 0x87, 0x53, 0x7f, 0x14, 0xbb, 0x81, 0x2f, 0x4e, - 0xf5, 0xe9, 0x84, 0xc9, 0x5d, 0xab, 0x44, 0x7e, 0x0b, 0x1c, 0x8d, 0xc6, 0xbc, 0x55, 0x5c, 0x2b, - 0x0a, 0x9c, 0xf8, 0x46, 0x2d, 0xa8, 0xb8, 0x7c, 0x3b, 0x98, 0xfa, 0x71, 0xab, 0xb4, 0x66, 0xac, - 0xdb, 0x24, 0x01, 0xf1, 0xcf, 0x8a, 0x50, 0xfe, 0x64, 0xca, 0xa2, 0x73, 0xc9, 0x17, 0xc7, 0x51, - 0xb2, 0x97, 0xf8, 0x46, 0xd7, 0xa1, 0xec, 0x51, 0x7f, 0xcc, 0x5b, 0xa6, 0xdc, 0x4c, 0x01, 0xe8, - 0x05, 0xa8, 0xd2, 0xe3, 0x98, 0x45, 0xc3, 0xa9, 0xeb, 0xb4, 0x8a, 0x6b, 0xc6, 0xba, 0x45, 0x6c, - 0x89, 0x38, 0x74, 0x1d, 0xf4, 0x3c, 0xd8, 0x4e, 0x30, 0x1c, 0xe5, 0xcf, 0x72, 0x02, 0x79, 0x16, - 0xba, 0x03, 0xf6, 0xd4, 0x75, 0x86, 0x9e, 0xcb, 0xe3, 0x56, 0x79, 0xcd, 0x58, 0xaf, 0x6d, 0xd6, - 0x93, 0x07, 0x0b, 0x19, 0x92, 0xca, 0xd4, 0x75, 0xa4, 0x30, 0x37, 0xc0, 0xe6, 0xd1, 0x68, 0x78, - 0x3c, 0xf5, 0x47, 0x2d, 0x4b, 0x12, 0x3e, 0x9b, 0x10, 0xe6, 0x5e, 0x4f, 0x2a, 0x5c, 0x01, 0xe2, - 0x79, 0x11, 0x3b, 0x63, 0x11, 0x67, 0xad, 0x8a, 0x3a, 0x52, 0x83, 0xe8, 0x1e, 0xd4, 0x8e, 0xe9, - 0x88, 0xc5, 0xc3, 0x90, 0x46, 0x74, 0xd2, 0xb2, 0xe7, 0x37, 0x7b, 0x28, 0x96, 0x0e, 0xc4, 0x0a, - 0x27, 0x70, 0x9c, 0x02, 0xe8, 0x7d, 0x68, 0x48, 0x88, 0x0f, 0x8f, 0x5d, 0x2f, 0x66, 0x51, 0xab, - 0x2a, 0xf9, 0x50, 0xca, 0x27, 0xb1, 0x83, 0x88, 0x31, 0x52, 0x57, 0x84, 0x0a, 0x83, 0xfe, 0x0f, - 0x80, 0xcd, 0x42, 0xea, 0x3b, 0x43, 0xea, 0x79, 0x2d, 0x90, 0x77, 0xa9, 0x2a, 0xcc, 0x96, 0xe7, - 0xa1, 0x9b, 0xe2, 0x9e, 0xd4, 0x19, 0xc6, 0xbc, 0xd5, 0x58, 0x33, 0xd6, 0x4b, 0xc4, 0x12, 0xe0, - 0x80, 0x0b, 0xc9, 0x78, 0xae, 0x3f, 0x14, 0x50, 0x6b, 0x45, 0x4b, 0x46, 0xd8, 0xd8, 0x8e, 0xeb, - 0x13, 0x46, 0x1d, 0x52, 0xf1, 0xd4, 0x07, 0x7e, 0x0f, 0xaa, 0xd2, 0x9c, 0xa4, 0x98, 0x5e, 0x05, - 0xeb, 0x4c, 0x00, 0xca, 0xea, 0x6a, 0x9b, 0xcf, 0x24, 0xf7, 0x4b, 0xad, 0x8e, 0x68, 0x02, 0xbc, - 0x0a, 0xf6, 0x0e, 0xf5, 0xc7, 0x89, 0xa9, 0x0a, 0x3d, 0x4a, 0xa6, 0x2a, 0x91, 0xdf, 0xf8, 0x8f, - 0x26, 0x58, 0x84, 0xf1, 0xa9, 0x17, 0xa3, 0xd7, 0x01, 0x84, 0x96, 0x26, 0x34, 0x8e, 0xdc, 0x99, - 0xde, 0x79, 0x5e, 0x4f, 0xd5, 0xa9, 0xeb, 0xec, 0xca, 0x65, 0x74, 0x0f, 0xea, 0xf2, 0x84, 0x84, - 0xdc, 0x9c, 0xbf, 0x48, 0x7a, 0x57, 0x52, 0x93, 0x64, 0x9a, 0xeb, 0x06, 0x58, 0xd2, 0x40, 0x94, - 0x91, 0x36, 0x88, 0x86, 0xd0, 0xff, 0x6b, 0x8f, 0xe3, 0x6c, 0x14, 0x0f, 0x1d, 0xc6, 0x13, 0x0b, - 0x6a, 0xa4, 0xd8, 0x0e, 0xe3, 0x31, 0x7a, 0x17, 0x94, 0xd4, 0x93, 0x43, 0xcb, 0xf2, 0x50, 0x34, - 0xa7, 0x55, 0xae, 0x4e, 0x95, 0x74, 0xfa, 0xd4, 0xbb, 0x50, 0x13, 0x6f, 0x4d, 0xb8, 0x2c, 0xc9, - 0xd5, 0x4c, 0x5f, 0xa6, 0xc5, 0x43, 0x40, 0x10, 0x69, 0x16, 0x21, 0x2a, 0x61, 0xad, 0xca, 0xaa, - 0xe4, 0xf7, 0xd5, 0x75, 0xd5, 0x85, 0xf2, 0x7e, 0xe4, 0xb0, 0x68, 0xa9, 0x67, 0x21, 0x28, 0x39, - 0x8c, 0x8f, 0xa4, 0xe3, 0xdb, 0x44, 0x7e, 0x67, 0xde, 0x56, 0xcc, 0x79, 0x1b, 0xfe, 0xbd, 0x01, - 0xb5, 0x7e, 0x10, 0xc5, 0xbb, 0x8c, 0x73, 0x3a, 0x66, 0xe8, 0x16, 0x94, 0x03, 0xb1, 0xad, 0x56, - 0x4d, 0x23, 0x79, 0x80, 0x3c, 0x8b, 0xa8, 0xb5, 0x05, 0x25, 0x9a, 0x97, 0x2b, 0xf1, 0x3a, 0x94, - 0x95, 0xbf, 0x0a, 0x5f, 0x2e, 0x13, 0x05, 0x08, 0x25, 0x05, 0xc7, 0xc7, 0x9c, 0x29, 0x25, 0x94, - 0x89, 0x86, 0xfe, 0x0b, 0x46, 0x7c, 0x04, 0x20, 0x1e, 0xf4, 0xef, 0xd8, 0xdb, 0x95, 0xcf, 0x38, - 0x81, 0x1a, 0xa1, 0xc7, 0xf1, 0x76, 0xe0, 0xc7, 0x6c, 0x16, 0xa3, 0x15, 0x30, 0x5d, 0x47, 0x2a, - 0xc0, 0x22, 0xa6, 0xeb, 0x88, 0x27, 0x8f, 0xa3, 0x60, 0x1a, 0x4a, 0xf9, 0x37, 0x88, 0x02, 0xa4, - 0xa2, 0x1c, 0x27, 0x92, 0x72, 0x10, 0x8a, 0x72, 0x9c, 0x08, 0xbd, 0x04, 0x35, 0xee, 0xd3, 0x90, - 0x9f, 0x04, 0xb1, 0x78, 0x72, 0x49, 0x3e, 0x19, 0x12, 0xd4, 0x80, 0xe3, 0x5f, 0x1b, 0x60, 0xed, - 0xb2, 0xc9, 0x11, 0x8b, 0x2e, 0x9c, 0xf2, 0x3c, 0xd8, 0x72, 0xe3, 0xa1, 0xeb, 0xe8, 0x83, 0x2a, - 0x12, 0xee, 0x39, 0x4b, 0x8f, 0xba, 0x01, 0x96, 0xc7, 0xa8, 0x50, 0xad, 0x32, 0x7b, 0x0d, 0x09, - 0x89, 0xd3, 0xc9, 0xd0, 0x11, 0x6f, 0x2e, 0xab, 0x05, 0x3a, 0xe9, 0x30, 0xea, 0x88, 0xbb, 0x79, - 0x94, 0xc7, 0xc3, 0x69, 0xe8, 0xd0, 0x98, 0xc9, 0x50, 0x59, 0x12, 0xf6, 0xcb, 0xe3, 0x43, 0x89, - 0x41, 0xaf, 0xc1, 0x33, 0x23, 0x6f, 0xca, 0x45, 0xac, 0x76, 0xfd, 0xe3, 0x60, 0x18, 0xf8, 0xde, - 0xb9, 0xd4, 0x9a, 0x4d, 0xae, 0xe9, 0x85, 0x9e, 0x7f, 0x1c, 0xec, 0xfb, 0xde, 0x39, 0xfe, 0xa1, - 0x09, 0xe5, 0x47, 0x52, 0x0c, 0xf7, 0xa0, 0x32, 0x91, 0x0f, 0x4a, 0x02, 0x4b, 0x3b, 0x51, 0x87, - 0x5c, 0xdf, 0x50, 0xaf, 0xe5, 0x5d, 0x3f, 0x8e, 0xce, 0x49, 0x42, 0x2a, 0xb8, 0x62, 0x7a, 0xe4, - 0xb1, 0x98, 0x6b, 0x7b, 0x5b, 0xe0, 0x1a, 0xa8, 0x45, 0xcd, 0xa5, 0x49, 0xdb, 0x1f, 0x43, 0x3d, - 0xbf, 0x9d, 0x48, 0x93, 0xa7, 0xec, 0x5c, 0xca, 0xb0, 0x44, 0xc4, 0x27, 0x7a, 0x05, 0xca, 0x32, - 0x76, 0x48, 0x09, 0xd6, 0x36, 0x57, 0x92, 0x5d, 0x15, 0x1b, 0x51, 0x8b, 0xf7, 0xcd, 0x0f, 0x0c, - 0xb1, 0x57, 0xfe, 0x90, 0xfc, 0x5e, 0xd5, 0xcb, 0xf7, 0x52, 0x6c, 0xb9, 0xbd, 0xf0, 0x8f, 0x4c, - 0xa8, 0x7f, 0xce, 0xa2, 0xe0, 0x20, 0x0a, 0xc2, 0x80, 0x53, 0x0f, 0xdd, 0x82, 0x86, 0xc3, 0xc2, - 0x88, 0x8d, 0x68, 0xcc, 0x9c, 0xa1, 0x56, 0x73, 0x83, 0xd4, 0x33, 0x64, 0xcf, 0x41, 0xb7, 0xc1, - 0x52, 0xe2, 0x78, 0xca, 0x65, 0xf5, 0xaa, 0xa0, 0x53, 0x02, 0x90, 0xfa, 0xbf, 0x78, 0x11, 0xbd, - 0x8a, 0x56, 0x01, 0x26, 0x74, 0xb6, 0xc3, 0x28, 0x67, 0x3d, 0x27, 0xb1, 0xbd, 0x0c, 0x83, 0xda, - 0x60, 0x4f, 0xe8, 0x6c, 0x30, 0xf3, 0x07, 0x5c, 0x9a, 0x46, 0x89, 0xa4, 0x30, 0x7a, 0x11, 0xaa, - 0x13, 0x3a, 0x13, 0x4e, 0xd0, 0x73, 0xb4, 0x69, 0x64, 0x08, 0xf4, 0x32, 0x14, 0xe3, 0x99, 0x2f, - 0x03, 0x5b, 0x6d, 0xf3, 0x9a, 0xf4, 0xa1, 0xc1, 0xcc, 0xd7, 0xee, 0x42, 0xc4, 0x5a, 0x22, 0x3e, - 0x3b, 0x15, 0x1f, 0xfe, 0x65, 0x11, 0xae, 0x69, 0x6d, 0x9d, 0xb8, 0x61, 0x3f, 0x16, 0x26, 0xd6, - 0x82, 0x8a, 0x8c, 0x17, 0x2c, 0xd2, 0x4a, 0x4b, 0x40, 0xf4, 0x4d, 0xb0, 0xa4, 0xb5, 0x27, 0xf6, - 0x70, 0x6b, 0x5e, 0x18, 0xe9, 0x16, 0xca, 0x3e, 0xb4, 0x61, 0x68, 0x16, 0xf4, 0x01, 0x94, 0xbf, - 0x64, 0x51, 0xa0, 0x62, 0x61, 0x6d, 0x13, 0x3f, 0x8d, 0x57, 0xe8, 0x48, 0xb3, 0x2a, 0x86, 0xff, - 0xa1, 0xcc, 0xd6, 0x45, 0xe4, 0x9b, 0x04, 0x67, 0xcc, 0x69, 0x55, 0xe4, 0xad, 0x16, 0xd5, 0x9b, - 0x2c, 0xb7, 0x3f, 0x82, 0x5a, 0xee, 0x51, 0x79, 0x43, 0x6c, 0x28, 0x43, 0xbc, 0x35, 0x6f, 0x88, - 0x8d, 0x39, 0x57, 0xc9, 0xdb, 0xf4, 0x47, 0x00, 0xd9, 0x13, 0xff, 0x13, 0xef, 0xc0, 0x3f, 0x30, - 0xe0, 0xda, 0x76, 0xe0, 0xfb, 0x4c, 0x16, 0x4f, 0x4a, 0x79, 0x99, 0xbd, 0x1a, 0x97, 0xda, 0xeb, - 0x9b, 0x50, 0xe6, 0x82, 0x41, 0x9f, 0x72, 0xf3, 0x29, 0xda, 0x20, 0x8a, 0x4a, 0xc4, 0xa5, 0x09, - 0x9d, 0x0d, 0x43, 0xe6, 0x3b, 0xae, 0x3f, 0x96, 0x36, 0xae, 0x74, 0x70, 0xa0, 0x30, 0xf8, 0x27, - 0x06, 0x58, 0xca, 0xd4, 0xe7, 0x62, 0xa4, 0x31, 0x1f, 0x23, 0x5f, 0x84, 0x6a, 0x18, 0x31, 0xc7, - 0x1d, 0x25, 0x27, 0x57, 0x49, 0x86, 0x10, 0x21, 0xfc, 0x38, 0x88, 0x46, 0x4c, 0x6e, 0x6f, 0x13, - 0x05, 0x88, 0xda, 0x54, 0x66, 0x27, 0x19, 0xe9, 0x54, 0x18, 0xb5, 0x05, 0x42, 0x84, 0x38, 0xc1, - 0xc2, 0x43, 0x3a, 0x52, 0x55, 0x62, 0x91, 0x28, 0x40, 0x84, 0x5d, 0xa5, 0x37, 0x69, 0xea, 0x36, - 0xd1, 0x10, 0xfe, 0x85, 0x09, 0xf5, 0x8e, 0x1b, 0xb1, 0x51, 0xcc, 0x9c, 0xae, 0x33, 0x96, 0x84, - 0xcc, 0x8f, 0xdd, 0xf8, 0x5c, 0x87, 0x78, 0x0d, 0xa5, 0xf9, 0xdd, 0x9c, 0xaf, 0x9c, 0x95, 0x5e, - 0x8a, 0xb2, 0xe0, 0x57, 0x00, 0x7a, 0x0f, 0x40, 0x95, 0x4b, 0xb2, 0xe8, 0x2f, 0x5d, 0x5e, 0xf4, - 0x57, 0x25, 0xa9, 0xf8, 0x14, 0x42, 0x52, 0x7c, 0xae, 0x4a, 0x01, 0x96, 0xec, 0x08, 0xa6, 0xc2, - 0x9c, 0x65, 0xd1, 0x70, 0xc4, 0x3c, 0x69, 0xae, 0xb2, 0x68, 0x38, 0x62, 0x5e, 0x5a, 0xe3, 0x55, - 0xd4, 0x95, 0xc4, 0x37, 0xba, 0x03, 0x66, 0x10, 0xca, 0x37, 0xe6, 0x0e, 0xcd, 0x3f, 0x70, 0x63, - 0x3f, 0x24, 0x66, 0x10, 0x22, 0x0c, 0x96, 0xaa, 0x6a, 0x5b, 0x55, 0x69, 0xe6, 0x20, 0xc3, 0x83, - 0x2c, 0xab, 0x88, 0x5e, 0xc1, 0x37, 0xc0, 0xdc, 0x0f, 0x51, 0x05, 0x8a, 0xfd, 0xee, 0xa0, 0x59, - 0x10, 0x1f, 0x9d, 0xee, 0x4e, 0xd3, 0xc0, 0x7f, 0x37, 0xa0, 0xba, 0x3b, 0x8d, 0xa9, 0xb0, 0x31, - 0x7e, 0x99, 0x72, 0x9f, 0x07, 0x9b, 0xc7, 0x34, 0x92, 0x49, 0xd5, 0x54, 0x81, 0x43, 0xc2, 0x03, - 0x8e, 0x5e, 0x83, 0x32, 0x73, 0xc6, 0x2c, 0xf1, 0xfd, 0xeb, 0xcb, 0xee, 0x4a, 0x14, 0x09, 0x7a, - 0x03, 0x2c, 0x3e, 0x3a, 0x61, 0x13, 0xda, 0x2a, 0xcd, 0x13, 0xf7, 0x25, 0x56, 0xe5, 0x41, 0xa2, - 0x69, 0x64, 0x73, 0x12, 0x05, 0xa1, 0xac, 0xce, 0xcb, 0xba, 0x39, 0x89, 0x82, 0x50, 0xd4, 0xe6, - 0x9b, 0xf0, 0x9c, 0x3b, 0xf6, 0x83, 0x88, 0x0d, 0x5d, 0xdf, 0x61, 0xb3, 0xe1, 0x28, 0xf0, 0x8f, - 0x3d, 0x77, 0x14, 0x4b, 0xb9, 0xda, 0xe4, 0x59, 0xb5, 0xd8, 0x13, 0x6b, 0xdb, 0x7a, 0x09, 0xdf, - 0x81, 0xea, 0x63, 0x76, 0x2e, 0x8b, 0x5c, 0x8e, 0xda, 0x60, 0x9e, 0x9e, 0xe9, 0x84, 0x09, 0xc9, - 0x2d, 0x1e, 0x3f, 0x21, 0xe6, 0xe9, 0x19, 0xfe, 0x93, 0x09, 0xf6, 0xbf, 0x96, 0x49, 0xde, 0x82, - 0xea, 0x24, 0x11, 0xa3, 0xf6, 0xba, 0xb4, 0xaa, 0x4e, 0xe5, 0x4b, 0x32, 0x1a, 0xd4, 0x85, 0x1b, - 0xb9, 0x5d, 0xe3, 0x99, 0x2f, 0x5e, 0x20, 0x82, 0xb9, 0x4e, 0x31, 0x17, 0x62, 0xfc, 0xf5, 0x8c, - 0x3c, 0xc3, 0xea, 0x57, 0x94, 0x96, 0xbd, 0x22, 0x8b, 0x02, 0xe5, 0x2b, 0x45, 0x81, 0x3b, 0x70, - 0x6d, 0xe4, 0x31, 0xea, 0x0f, 0x33, 0x27, 0x56, 0x36, 0xba, 0x22, 0xd1, 0x07, 0xa9, 0x27, 0xeb, - 0xa8, 0x56, 0xc9, 0xf2, 0xf4, 0xab, 0x50, 0x76, 0x98, 0x17, 0xd3, 0xc5, 0x86, 0x6d, 0x3f, 0xa2, - 0x23, 0x8f, 0x75, 0xc4, 0x12, 0x51, 0x14, 0xf8, 0x65, 0x28, 0x3e, 0x7e, 0xd2, 0xbf, 0x54, 0xfa, - 0xdf, 0x03, 0xf3, 0xf1, 0x93, 0x7c, 0xec, 0xac, 0xab, 0x53, 0x74, 0x4b, 0x6e, 0x66, 0x2d, 0x79, - 0x1b, 0xec, 0x29, 0x67, 0xd1, 0x2e, 0x8b, 0xa9, 0x76, 0xdc, 0x14, 0x16, 0x89, 0x4e, 0xf4, 0x94, - 0x6e, 0xe0, 0xeb, 0xa4, 0x92, 0x80, 0xf8, 0x1f, 0x45, 0xa8, 0x68, 0xe7, 0x15, 0x7b, 0x4e, 0xd3, - 0x1a, 0x50, 0x7c, 0x66, 0x91, 0xc0, 0xcc, 0x47, 0x82, 0x7c, 0xf3, 0x5f, 0xbc, 0x5a, 0xf3, 0x8f, - 0xbe, 0x0d, 0xf5, 0x50, 0xad, 0xe5, 0xe3, 0xc7, 0x0b, 0x8b, 0x7c, 0xfa, 0x57, 0xf2, 0xd6, 0xc2, - 0x0c, 0x10, 0xd6, 0x2f, 0x1b, 0xa0, 0x98, 0x8e, 0xa5, 0x0a, 0xeb, 0xa4, 0x22, 0xe0, 0x01, 0x1d, - 0x3f, 0x25, 0x8a, 0x5c, 0x21, 0x10, 0x88, 0x9a, 0x37, 0x08, 0x5b, 0x75, 0x69, 0xc2, 0x22, 0x78, - 0xe4, 0xfd, 0xba, 0x31, 0xef, 0xd7, 0x2f, 0x40, 0x75, 0x14, 0x4c, 0x26, 0xae, 0x5c, 0x5b, 0x51, - 0xa9, 0x57, 0x21, 0x06, 0x1c, 0x7f, 0x09, 0x15, 0xfd, 0x60, 0x54, 0x83, 0x4a, 0xa7, 0xfb, 0x70, - 0xeb, 0x70, 0x47, 0x44, 0x16, 0x00, 0xeb, 0x41, 0x6f, 0x6f, 0x8b, 0x7c, 0xd6, 0x34, 0x44, 0x94, - 0xe9, 0xed, 0x0d, 0x9a, 0x26, 0xaa, 0x42, 0xf9, 0xe1, 0xce, 0xfe, 0xd6, 0xa0, 0x59, 0x44, 0x36, - 0x94, 0x1e, 0xec, 0xef, 0xef, 0x34, 0x4b, 0xa8, 0x0e, 0x76, 0x67, 0x6b, 0xd0, 0x1d, 0xf4, 0x76, - 0xbb, 0xcd, 0xb2, 0xa0, 0x7d, 0xd4, 0xdd, 0x6f, 0x5a, 0xe2, 0xe3, 0xb0, 0xd7, 0x69, 0x56, 0xc4, - 0xfa, 0xc1, 0x56, 0xbf, 0xff, 0xe9, 0x3e, 0xe9, 0x34, 0x6d, 0xb1, 0x6f, 0x7f, 0x40, 0x7a, 0x7b, - 0x8f, 0x9a, 0x55, 0x7c, 0x17, 0x6a, 0x39, 0xa1, 0x09, 0x0e, 0xd2, 0x7d, 0xd8, 0x2c, 0x88, 0x63, - 0x9e, 0x6c, 0xed, 0x1c, 0x76, 0x9b, 0x06, 0x5a, 0x01, 0x90, 0x9f, 0xc3, 0x9d, 0xad, 0xbd, 0x47, - 0x4d, 0x13, 0x7f, 0xdf, 0x48, 0x79, 0x64, 0x53, 0xfd, 0x3a, 0xd8, 0x5a, 0xd4, 0x49, 0xd1, 0x7c, - 0x6d, 0x41, 0x2f, 0x24, 0x25, 0x10, 0x66, 0x36, 0x3a, 0x61, 0xa3, 0x53, 0x3e, 0x9d, 0x68, 0xab, - 0x48, 0x61, 0xd5, 0x1b, 0x0b, 0x99, 0xe8, 0xb4, 0xa9, 0xa1, 0x74, 0xc0, 0x54, 0x92, 0xf4, 0x6a, - 0xc0, 0x74, 0x0f, 0x20, 0x1b, 0x61, 0x2c, 0x29, 0x77, 0xaf, 0x43, 0x99, 0x7a, 0x2e, 0xe5, 0x3a, - 0x33, 0x29, 0x00, 0x13, 0xa8, 0xe5, 0x06, 0x1f, 0x42, 0x61, 0xd4, 0xf3, 0x86, 0xa7, 0xec, 0x9c, - 0x4b, 0x5e, 0x9b, 0x54, 0xa8, 0xe7, 0x3d, 0x66, 0xe7, 0x1c, 0xad, 0x43, 0x59, 0xcd, 0x4d, 0xcc, - 0x25, 0x1d, 0xb6, 0x64, 0x27, 0x8a, 0x00, 0xbf, 0x01, 0x96, 0x6a, 0xbb, 0x73, 0x36, 0x63, 0x3c, - 0x35, 0x79, 0x7c, 0xa8, 0xef, 0x2d, 0x9b, 0x74, 0xf4, 0x96, 0x9e, 0xd1, 0x70, 0x35, 0x19, 0x32, - 0xe6, 0x4b, 0x2b, 0x45, 0xa8, 0xc7, 0x33, 0x92, 0x01, 0x77, 0xc0, 0xbe, 0x74, 0x02, 0xa6, 0x05, - 0x61, 0x66, 0x82, 0x58, 0x32, 0x13, 0xc3, 0x11, 0x40, 0x36, 0xc7, 0xd1, 0x66, 0xac, 0x76, 0x11, - 0x66, 0xbc, 0x21, 0x54, 0xe4, 0x7a, 0x4e, 0xc4, 0xfc, 0x0b, 0xaf, 0xcf, 0xa6, 0x3f, 0x29, 0x0d, - 0x7a, 0x05, 0x4a, 0x72, 0x5c, 0xa5, 0x82, 0x6d, 0x3a, 0x55, 0x48, 0x67, 0x55, 0x72, 0x15, 0x1f, - 0x41, 0x43, 0xe5, 0x25, 0xc2, 0xbe, 0x98, 0x32, 0x7e, 0x69, 0xf5, 0xb3, 0x0a, 0x90, 0x06, 0xce, - 0x64, 0x00, 0x97, 0xc3, 0x08, 0x43, 0x39, 0x76, 0x99, 0xe7, 0x24, 0xaf, 0xd2, 0x10, 0x7e, 0x1f, - 0xea, 0xc9, 0x19, 0xb2, 0xbf, 0xbe, 0x93, 0x66, 0xc8, 0xc4, 0x2e, 0x85, 0x42, 0x14, 0xc9, 0x5e, - 0xe0, 0xa4, 0xc9, 0x11, 0xff, 0xc5, 0x4c, 0x38, 0x75, 0xf7, 0x38, 0x57, 0x7f, 0x19, 0x8b, 0xf5, - 0xd7, 0x7c, 0x2d, 0x63, 0x5e, 0xb9, 0x96, 0xf9, 0x16, 0x54, 0x1d, 0x99, 0xc8, 0xdd, 0xb3, 0x24, - 0xf4, 0xad, 0x2e, 0x4b, 0xda, 0x3a, 0xdd, 0xbb, 0x67, 0x8c, 0x64, 0x0c, 0xe2, 0x4e, 0x71, 0x70, - 0xca, 0x7c, 0xf7, 0x4b, 0xd9, 0x26, 0x8b, 0x87, 0x67, 0x88, 0x6c, 0x92, 0xa1, 0x92, 0xbb, 0x9e, - 0x64, 0x24, 0x53, 0x1c, 0x2b, 0x37, 0xc5, 0xb9, 0x01, 0xd6, 0x34, 0xe4, 0x2c, 0x8a, 0x93, 0xa2, - 0x4f, 0x41, 0x69, 0xe1, 0x54, 0xd5, 0xb4, 0xd4, 0x1f, 0xe3, 0x6f, 0x40, 0x35, 0xbd, 0x8b, 0x88, - 0x37, 0x7b, 0xfb, 0x7b, 0x5d, 0x15, 0x1d, 0x7a, 0x7b, 0x9d, 0xee, 0x77, 0x9b, 0x86, 0x88, 0x58, - 0xa4, 0xfb, 0xa4, 0x4b, 0xfa, 0xdd, 0xa6, 0x29, 0x22, 0x4b, 0xa7, 0xbb, 0xd3, 0x1d, 0x74, 0x9b, - 0xc5, 0x8f, 0x4b, 0x76, 0xa5, 0x69, 0x13, 0x9b, 0xcd, 0x42, 0xcf, 0x1d, 0xb9, 0x31, 0xfe, 0x0c, - 0xec, 0x5d, 0x1a, 0x5e, 0x28, 0xe6, 0xb3, 0x84, 0x34, 0xd5, 0xa3, 0x02, 0x9d, 0x3c, 0x5e, 0x85, - 0x8a, 0x8e, 0x1a, 0x69, 0x1a, 0x5f, 0x88, 0x2a, 0xc9, 0x3a, 0xfe, 0xb9, 0x01, 0xd7, 0x77, 0x83, - 0x33, 0x96, 0xe6, 0xd5, 0x03, 0x7a, 0xee, 0x05, 0xd4, 0xf9, 0x1a, 0x35, 0xde, 0x86, 0x6b, 0x3c, - 0x98, 0x46, 0x23, 0x36, 0x5c, 0x18, 0x55, 0x34, 0x14, 0xfa, 0x91, 0x36, 0x47, 0x2c, 0xaa, 0x16, - 0x1e, 0x67, 0x54, 0x45, 0x49, 0x55, 0x13, 0xc8, 0x84, 0x26, 0x2d, 0x10, 0x4a, 0x57, 0x29, 0x10, - 0xf0, 0xef, 0x0c, 0x68, 0x74, 0x67, 0x61, 0x10, 0xc5, 0xc9, 0x55, 0x9f, 0x13, 0xa5, 0xf8, 0x17, - 0x89, 0x33, 0x94, 0x48, 0x39, 0x62, 0x5f, 0xf4, 0x2e, 0x9d, 0xa3, 0xdc, 0x03, 0x4b, 0x6c, 0x36, - 0xe5, 0xda, 0x94, 0x5e, 0x4c, 0xce, 0x9c, 0xdb, 0x78, 0xa3, 0x2f, 0x69, 0x88, 0xa6, 0xcd, 0xcf, - 0xb0, 0x4a, 0xf9, 0x19, 0x16, 0xbe, 0x0f, 0x96, 0x22, 0xcd, 0xe9, 0xb9, 0x06, 0x95, 0xfe, 0xe1, - 0xf6, 0x76, 0xb7, 0xdf, 0x6f, 0x1a, 0xa8, 0x01, 0xd5, 0xce, 0xe1, 0xc1, 0x4e, 0x6f, 0x7b, 0x6b, - 0xa0, 0x75, 0xfd, 0x70, 0xab, 0xb7, 0xd3, 0xed, 0x34, 0x8b, 0xf8, 0x57, 0x06, 0xd4, 0x72, 0x05, - 0x0a, 0xba, 0x2f, 0x3a, 0x63, 0x11, 0xbb, 0x93, 0x50, 0xb8, 0xb6, 0xa4, 0x8c, 0xd9, 0xd8, 0x56, - 0x24, 0x7a, 0x2c, 0xa2, 0x19, 0x84, 0x79, 0xd2, 0xa3, 0x20, 0xd2, 0xb3, 0x94, 0x12, 0xd1, 0xd0, - 0xd7, 0x76, 0x56, 0xed, 0xfb, 0x50, 0xcf, 0xef, 0xb8, 0xa4, 0x63, 0x9c, 0xab, 0x47, 0x4a, 0xf9, - 0x0e, 0xf1, 0x25, 0x68, 0x88, 0x36, 0xd8, 0x9d, 0x30, 0x1e, 0xd3, 0x49, 0x28, 0x73, 0xbb, 0xbe, - 0x7c, 0x89, 0x98, 0x31, 0xc7, 0xb7, 0xa1, 0x7e, 0xc0, 0x58, 0x44, 0x18, 0x0f, 0x03, 0x9f, 0xcb, - 0x86, 0x48, 0x0b, 0x5f, 0x25, 0x0e, 0x0d, 0xe1, 0x23, 0xa8, 0x8a, 0x96, 0xf9, 0x01, 0x8d, 0x47, - 0x27, 0xe8, 0x4d, 0x21, 0x06, 0x55, 0x89, 0x1a, 0xf3, 0xd5, 0x5c, 0x6e, 0x40, 0x47, 0x12, 0x1a, - 0x74, 0x1b, 0x2a, 0xa1, 0x52, 0x9a, 0x2e, 0x7b, 0xd5, 0x80, 0x4f, 0x2b, 0x92, 0x24, 0x8b, 0xf8, - 0x26, 0x14, 0xf7, 0xa6, 0x93, 0xfc, 0xdf, 0x2a, 0x25, 0x59, 0xc3, 0xe1, 0x87, 0x50, 0xef, 0xeb, - 0xe9, 0x9c, 0xac, 0xdb, 0x44, 0xd5, 0xe1, 0xb9, 0xcc, 0x97, 0x55, 0x87, 0xa1, 0xab, 0x0e, 0x89, - 0x18, 0xf0, 0x4b, 0x2c, 0x6b, 0xf3, 0xb7, 0x06, 0x94, 0xc4, 0x0d, 0x45, 0x68, 0xef, 0x8e, 0x4e, - 0x02, 0x34, 0x77, 0x91, 0xf6, 0x1c, 0x84, 0x0b, 0xe8, 0x6d, 0x35, 0x70, 0x4c, 0xa6, 0xb4, 0xcf, - 0xe4, 0x1f, 0x29, 0x05, 0x71, 0x81, 0x63, 0x13, 0x6a, 0x1f, 0x07, 0xae, 0xbf, 0xad, 0xe6, 0x70, - 0x68, 0x99, 0x58, 0x2e, 0xf0, 0xbc, 0x0b, 0x56, 0x8f, 0x0b, 0x1d, 0x2c, 0x27, 0x4f, 0xbb, 0x9f, - 0xbc, 0x9a, 0x70, 0x61, 0xf3, 0xa7, 0x45, 0x28, 0x7d, 0xce, 0xa2, 0x00, 0xdd, 0x83, 0x8a, 0x9e, - 0x01, 0xa0, 0x85, 0x5e, 0xbf, 0x9d, 0x7a, 0xeb, 0xc2, 0x90, 0x00, 0x17, 0xd0, 0x7b, 0x60, 0xe9, - 0x94, 0x30, 0x3f, 0xa8, 0x68, 0x3f, 0xcd, 0xc3, 0x71, 0x61, 0xdd, 0x78, 0xdb, 0x40, 0x6f, 0x81, - 0xa5, 0x4c, 0x7d, 0x41, 0x76, 0xcb, 0xea, 0x79, 0x5c, 0x90, 0x0c, 0xb5, 0xfe, 0x49, 0x30, 0xf5, - 0x9c, 0x3e, 0x8b, 0xce, 0x18, 0x5a, 0x18, 0x8b, 0xb5, 0x17, 0x60, 0x5c, 0x40, 0x6f, 0x02, 0x6c, - 0x71, 0xee, 0x8e, 0xfd, 0x43, 0xd7, 0xe1, 0xa8, 0x96, 0xac, 0xef, 0x4d, 0x27, 0xed, 0xa6, 0x3c, - 0x52, 0xad, 0x8a, 0x96, 0x8a, 0x2b, 0xf2, 0x9c, 0x79, 0x7f, 0x2d, 0xf9, 0x3b, 0xd0, 0x50, 0xce, - 0xb4, 0x1f, 0x6d, 0x09, 0xff, 0x43, 0x8b, 0x4d, 0x54, 0x7b, 0x11, 0x81, 0x0b, 0xe8, 0x3e, 0xd8, - 0x83, 0xe8, 0x5c, 0xd1, 0x3f, 0x97, 0x5e, 0x38, 0xef, 0x57, 0xed, 0xe5, 0x68, 0x5c, 0xd8, 0xfc, - 0x5b, 0x11, 0xac, 0x4f, 0x83, 0xe8, 0x94, 0x45, 0x68, 0x03, 0x2c, 0xd9, 0xe7, 0xe5, 0x4c, 0x29, - 0xed, 0xfb, 0x96, 0x1d, 0xfb, 0x06, 0x54, 0xa5, 0xd0, 0x06, 0x94, 0x9f, 0x66, 0x6a, 0x92, 0x7f, - 0xed, 0x65, 0x72, 0x53, 0x25, 0x01, 0x2e, 0xa0, 0xef, 0xc0, 0x8d, 0x34, 0x4f, 0x6c, 0xf9, 0x8e, - 0xca, 0xbb, 0x1d, 0x1a, 0x53, 0x94, 0x35, 0xd0, 0x39, 0x27, 0x6a, 0xd7, 0xb2, 0x96, 0xaa, 0x2f, - 0x35, 0x75, 0x17, 0x4a, 0x7d, 0xf1, 0xc2, 0xec, 0x8f, 0xb9, 0xec, 0x2f, 0x8a, 0x36, 0xca, 0x23, - 0xd3, 0x33, 0xdf, 0x07, 0x4b, 0x9d, 0x93, 0x89, 0x65, 0xae, 0x18, 0x6a, 0x5f, 0x5f, 0x44, 0x6b, - 0xc6, 0x3b, 0x60, 0xef, 0xba, 0xbe, 0x9a, 0xd0, 0xcd, 0x1b, 0x52, 0x5e, 0x83, 0xb8, 0x80, 0x3e, - 0x00, 0x4b, 0x85, 0xfd, 0xec, 0x84, 0xb9, 0x34, 0xd0, 0x5e, 0x8e, 0xc6, 0x05, 0x74, 0x17, 0x9a, - 0x84, 0x8d, 0x98, 0x9b, 0x4b, 0x9f, 0x28, 0xff, 0xe6, 0x45, 0x47, 0x5c, 0x37, 0xd0, 0x87, 0xd0, - 0x98, 0x4b, 0xb7, 0x28, 0x4d, 0x3d, 0xcb, 0xb2, 0xf0, 0xe2, 0x06, 0x0f, 0x9a, 0xbf, 0xf9, 0x6a, - 0xd5, 0xf8, 0xc3, 0x57, 0xab, 0xc6, 0x9f, 0xbf, 0x5a, 0x35, 0x7e, 0xfc, 0xd7, 0xd5, 0xc2, 0x91, - 0x25, 0xff, 0x51, 0x7e, 0xe7, 0x9f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x31, 0x23, 0x67, 0x76, - 0x1e, 0x00, 0x00, + // 3147 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4b, 0x6f, 0x23, 0xc7, + 0xf1, 0xd7, 0x0c, 0xc9, 0xe1, 0xb0, 0x48, 0x6a, 0xb9, 0x6d, 0x79, 0x97, 0xa6, 0xfd, 0x97, 0xe5, + 0x5e, 0xff, 0x77, 0x65, 0x7b, 0xad, 0xf5, 0xca, 0xeb, 0x47, 0x36, 0x71, 0x02, 0xad, 0xc8, 0x5d, + 0xd3, 0xab, 0x97, 0x9b, 0xd4, 0x3a, 0xf6, 0x21, 0x44, 0x8b, 0xd3, 0xa2, 0x06, 0x1a, 0xce, 0x8c, + 0xa7, 0x87, 0x02, 0xe5, 0x53, 0x90, 0x43, 0x80, 0x20, 0xb7, 0x9c, 0x72, 0x0e, 0x90, 0x5b, 0x80, + 0x00, 0x39, 0xe6, 0x94, 0x4b, 0x80, 0x04, 0x08, 0x90, 0xe4, 0x1b, 0x24, 0xce, 0x25, 0xc8, 0x2d, + 0xdf, 0x20, 0xe8, 0xc7, 0x3c, 0xc8, 0xe5, 0xca, 0x72, 0x1e, 0x27, 0x4e, 0x55, 0x57, 0xf5, 0xa3, + 0xaa, 0xba, 0xea, 0xd7, 0x45, 0x58, 0x76, 0xfd, 0x98, 0x45, 0x3e, 0xf5, 0x36, 0xc2, 0x28, 0x88, + 0x03, 0x64, 0x29, 0xba, 0x55, 0xa1, 0xa1, 0xab, 0x58, 0xb8, 0x05, 0xc5, 0x1d, 0x97, 0xc7, 0x08, + 0x41, 0x71, 0xe2, 0x3a, 0xbc, 0x69, 0xac, 0x15, 0xd6, 0x2d, 0x22, 0xbf, 0xf1, 0xc7, 0x50, 0xe9, + 0x53, 0x7e, 0xfa, 0x84, 0x7a, 0x13, 0x86, 0x1a, 0x50, 0x38, 0xa3, 0x5e, 0xd3, 0x58, 0x33, 0xd6, + 0x6b, 0x44, 0x7c, 0xa2, 0x4d, 0xb0, 0xcf, 0xa8, 0x37, 0x88, 0xcf, 0x43, 0xd6, 0x34, 0xd7, 0x8c, + 0xf5, 0xe5, 0xcd, 0xeb, 0x1b, 0x6a, 0x81, 0x8d, 0x83, 0x80, 0xc7, 0xae, 0x3f, 0xda, 0x78, 0x42, + 0xbd, 0xfe, 0x79, 0xc8, 0x48, 0xf9, 0x4c, 0x7d, 0xe0, 0x7d, 0xa8, 0xf6, 0xa2, 0xe1, 0xc3, 0x89, + 0x3f, 0x8c, 0xdd, 0xc0, 0x17, 0xab, 0xfa, 0x74, 0xcc, 0xe4, 0xac, 0x15, 0x22, 0xbf, 0x05, 0x8f, + 0x46, 0x23, 0xde, 0x2c, 0xac, 0x15, 0x04, 0x4f, 0x7c, 0xa3, 0x26, 0x94, 0x5d, 0xbe, 0x1d, 0x4c, + 0xfc, 0xb8, 0x59, 0x5c, 0x33, 0xd6, 0x6d, 0x92, 0x90, 0xf8, 0x17, 0x05, 0x28, 0x7d, 0x3c, 0x61, + 0xd1, 0xb9, 0xd4, 0x8b, 0xe3, 0x28, 0x99, 0x4b, 0x7c, 0xa3, 0x15, 0x28, 0x79, 0xd4, 0x1f, 0xf1, + 0xa6, 0x29, 0x27, 0x53, 0x04, 0x7a, 0x11, 0x2a, 0xf4, 0x38, 0x66, 0xd1, 0x60, 0xe2, 0x3a, 0xcd, + 0xc2, 0x9a, 0xb1, 0x6e, 0x11, 0x5b, 0x32, 0x0e, 0x5d, 0x07, 0xbd, 0x00, 0xb6, 0x13, 0x0c, 0x86, + 0xf9, 0xb5, 0x9c, 0x40, 0xae, 0x85, 0x6e, 0x81, 0x3d, 0x71, 0x9d, 0x81, 0xe7, 0xf2, 0xb8, 0x59, + 0x5a, 0x33, 0xd6, 0xab, 0x9b, 0xb5, 0xe4, 0xc0, 0xc2, 0x86, 0xa4, 0x3c, 0x71, 0x1d, 0x69, 0xcc, + 0x0d, 0xb0, 0x79, 0x34, 0x1c, 0x1c, 0x4f, 0xfc, 0x61, 0xd3, 0x92, 0x82, 0xcf, 0x25, 0x82, 0xb9, + 0xd3, 0x93, 0x32, 0x57, 0x84, 0x38, 0x5e, 0xc4, 0xce, 0x58, 0xc4, 0x59, 0xb3, 0xac, 0x96, 0xd4, + 0x24, 0xba, 0x07, 0xd5, 0x63, 0x3a, 0x64, 0xf1, 0x20, 0xa4, 0x11, 0x1d, 0x37, 0xed, 0xd9, 0xc9, + 0x1e, 0x8a, 0xa1, 0x03, 0x31, 0xc2, 0x09, 0x1c, 0xa7, 0x04, 0x7a, 0x0f, 0xea, 0x92, 0xe2, 0x83, + 0x63, 0xd7, 0x8b, 0x59, 0xd4, 0xac, 0x48, 0x3d, 0x94, 0xea, 0x49, 0x6e, 0x3f, 0x62, 0x8c, 0xd4, + 0x94, 0xa0, 0xe2, 0xa0, 0xff, 0x03, 0x60, 0xd3, 0x90, 0xfa, 0xce, 0x80, 0x7a, 0x5e, 0x13, 0xe4, + 0x5e, 0x2a, 0x8a, 0xb3, 0xe5, 0x79, 0xe8, 0xba, 0xd8, 0x27, 0x75, 0x06, 0x31, 0x6f, 0xd6, 0xd7, + 0x8c, 0xf5, 0x22, 0xb1, 0x04, 0xd9, 0xe7, 0xc2, 0x32, 0x9e, 0xeb, 0x0f, 0x04, 0xd5, 0x5c, 0xd6, + 0x96, 0x11, 0x31, 0xb6, 0xe3, 0xfa, 0x84, 0x51, 0x87, 0x94, 0x3d, 0xf5, 0x81, 0xdf, 0x85, 0x8a, + 0x0c, 0x27, 0x69, 0xa6, 0xd7, 0xc0, 0x3a, 0x13, 0x84, 0x8a, 0xba, 0xea, 0xe6, 0xd5, 0x64, 0x7f, + 0x69, 0xd4, 0x11, 0x2d, 0x80, 0x57, 0xc1, 0xde, 0xa1, 0xfe, 0x28, 0x09, 0x55, 0xe1, 0x47, 0xa9, + 0x54, 0x21, 0xf2, 0x1b, 0xff, 0xd9, 0x04, 0x8b, 0x30, 0x3e, 0xf1, 0x62, 0xf4, 0x06, 0x80, 0xf0, + 0xd2, 0x98, 0xc6, 0x91, 0x3b, 0xd5, 0x33, 0xcf, 0xfa, 0xa9, 0x32, 0x71, 0x9d, 0x5d, 0x39, 0x8c, + 0xee, 0x41, 0x4d, 0xae, 0x90, 0x88, 0x9b, 0xb3, 0x1b, 0x49, 0xf7, 0x4a, 0xaa, 0x52, 0x4c, 0x6b, + 0x5d, 0x03, 0x4b, 0x06, 0x88, 0x0a, 0xd2, 0x3a, 0xd1, 0x14, 0xfa, 0x7f, 0x7d, 0xe3, 0x38, 0x1b, + 0xc6, 0x03, 0x87, 0xf1, 0x24, 0x82, 0xea, 0x29, 0xb7, 0xcd, 0x78, 0x8c, 0xde, 0x01, 0x65, 0xf5, + 0x64, 0xd1, 0x92, 0x5c, 0x14, 0xcd, 0x78, 0x95, 0xab, 0x55, 0xa5, 0x9c, 0x5e, 0xf5, 0x2e, 0x54, + 0xc5, 0x59, 0x13, 0x2d, 0x4b, 0x6a, 0x35, 0xd2, 0x93, 0x69, 0xf3, 0x10, 0x10, 0x42, 0x5a, 0x45, + 0x98, 0x4a, 0x44, 0xab, 0x8a, 0x2a, 0xf9, 0x7d, 0x79, 0x5f, 0x75, 0xa0, 0xb4, 0x1f, 0x39, 0x2c, + 0x5a, 0x78, 0xb3, 0x10, 0x14, 0x1d, 0xc6, 0x87, 0xf2, 0xe2, 0xdb, 0x44, 0x7e, 0x67, 0xb7, 0xad, + 0x90, 0xbb, 0x6d, 0xf8, 0x8f, 0x06, 0x54, 0x7b, 0x41, 0x14, 0xef, 0x32, 0xce, 0xe9, 0x88, 0xa1, + 0x1b, 0x50, 0x0a, 0xc4, 0xb4, 0xda, 0x35, 0xf5, 0xe4, 0x00, 0x72, 0x2d, 0xa2, 0xc6, 0xe6, 0x9c, + 0x68, 0x5e, 0xec, 0xc4, 0x15, 0x28, 0xa9, 0xfb, 0x2a, 0xee, 0x72, 0x89, 0x28, 0x42, 0x38, 0x29, + 0x38, 0x3e, 0xe6, 0x4c, 0x39, 0xa1, 0x44, 0x34, 0xf5, 0x5f, 0x08, 0xe2, 0x23, 0x00, 0x71, 0xa0, + 0x7f, 0x27, 0xde, 0x2e, 0xbd, 0xc6, 0x09, 0x54, 0x09, 0x3d, 0x8e, 0xb7, 0x03, 0x3f, 0x66, 0xd3, + 0x18, 0x2d, 0x83, 0xe9, 0x3a, 0xd2, 0x01, 0x16, 0x31, 0x5d, 0x47, 0x1c, 0x79, 0x14, 0x05, 0x93, + 0x50, 0xda, 0xbf, 0x4e, 0x14, 0x21, 0x1d, 0xe5, 0x38, 0x91, 0xb4, 0x83, 0x70, 0x94, 0xe3, 0x44, + 0xe8, 0x65, 0xa8, 0x72, 0x9f, 0x86, 0xfc, 0x24, 0x88, 0xc5, 0x91, 0x8b, 0xf2, 0xc8, 0x90, 0xb0, + 0xfa, 0x1c, 0xff, 0xd6, 0x00, 0x6b, 0x97, 0x8d, 0x8f, 0x58, 0xf4, 0xd4, 0x2a, 0x2f, 0x80, 0x2d, + 0x27, 0x1e, 0xb8, 0x8e, 0x5e, 0xa8, 0x2c, 0xe9, 0xae, 0xb3, 0x70, 0xa9, 0x6b, 0x60, 0x79, 0x8c, + 0x0a, 0xd7, 0xaa, 0xb0, 0xd7, 0x94, 0xb0, 0x38, 0x1d, 0x0f, 0x1c, 0x71, 0xe6, 0x92, 0x1a, 0xa0, + 0xe3, 0x36, 0xa3, 0x8e, 0xd8, 0x9b, 0x47, 0x79, 0x3c, 0x98, 0x84, 0x0e, 0x8d, 0x99, 0x4c, 0x95, + 0x45, 0x11, 0xbf, 0x3c, 0x3e, 0x94, 0x1c, 0xf4, 0x3a, 0x5c, 0x1d, 0x7a, 0x13, 0x2e, 0x72, 0xb5, + 0xeb, 0x1f, 0x07, 0x83, 0xc0, 0xf7, 0xce, 0xa5, 0xd7, 0x6c, 0x72, 0x45, 0x0f, 0x74, 0xfd, 0xe3, + 0x60, 0xdf, 0xf7, 0xce, 0xf1, 0x8f, 0x4d, 0x28, 0x3d, 0x92, 0x66, 0xb8, 0x07, 0xe5, 0xb1, 0x3c, + 0x50, 0x92, 0x58, 0x5a, 0x89, 0x3b, 0xe4, 0xf8, 0x86, 0x3a, 0x2d, 0xef, 0xf8, 0x71, 0x74, 0x4e, + 0x12, 0x51, 0xa1, 0x15, 0xd3, 0x23, 0x8f, 0xc5, 0x5c, 0xc7, 0xdb, 0x9c, 0x56, 0x5f, 0x0d, 0x6a, + 0x2d, 0x2d, 0xda, 0xfa, 0x08, 0x6a, 0xf9, 0xe9, 0x44, 0x99, 0x3c, 0x65, 0xe7, 0xd2, 0x86, 0x45, + 0x22, 0x3e, 0xd1, 0xab, 0x50, 0x92, 0xb9, 0x43, 0x5a, 0xb0, 0xba, 0xb9, 0x9c, 0xcc, 0xaa, 0xd4, + 0x88, 0x1a, 0xbc, 0x6f, 0xbe, 0x6f, 0x88, 0xb9, 0xf2, 0x8b, 0xe4, 0xe7, 0xaa, 0x5c, 0x3c, 0x97, + 0x52, 0xcb, 0xcd, 0x85, 0x7f, 0x62, 0x42, 0xed, 0x33, 0x16, 0x05, 0x07, 0x51, 0x10, 0x06, 0x9c, + 0x7a, 0xe8, 0x06, 0xd4, 0x1d, 0x16, 0x46, 0x6c, 0x48, 0x63, 0xe6, 0x0c, 0xb4, 0x9b, 0xeb, 0xa4, + 0x96, 0x31, 0xbb, 0x0e, 0xba, 0x09, 0x96, 0x32, 0xc7, 0x33, 0x36, 0xab, 0x47, 0x85, 0x9c, 0x32, + 0x80, 0xf4, 0xff, 0xd3, 0x1b, 0xd1, 0xa3, 0x68, 0x15, 0x60, 0x4c, 0xa7, 0x3b, 0x8c, 0x72, 0xd6, + 0x75, 0x92, 0xd8, 0xcb, 0x38, 0xa8, 0x05, 0xf6, 0x98, 0x4e, 0xfb, 0x53, 0xbf, 0xcf, 0x65, 0x68, + 0x14, 0x49, 0x4a, 0xa3, 0x97, 0xa0, 0x32, 0xa6, 0x53, 0x71, 0x09, 0xba, 0x8e, 0x0e, 0x8d, 0x8c, + 0x81, 0x5e, 0x81, 0x42, 0x3c, 0xf5, 0x65, 0x62, 0xab, 0x6e, 0x5e, 0x91, 0x77, 0xa8, 0x3f, 0xf5, + 0xf5, 0x75, 0x21, 0x62, 0x2c, 0x31, 0x9f, 0x9d, 0x9a, 0x0f, 0xff, 0xba, 0x00, 0x57, 0xb4, 0xb7, + 0x4e, 0xdc, 0xb0, 0x17, 0x8b, 0x10, 0x6b, 0x42, 0x59, 0xe6, 0x0b, 0x16, 0x69, 0xa7, 0x25, 0x24, + 0xfa, 0x26, 0x58, 0x32, 0xda, 0x93, 0x78, 0xb8, 0x31, 0x6b, 0x8c, 0x74, 0x0a, 0x15, 0x1f, 0x3a, + 0x30, 0xb4, 0x0a, 0x7a, 0x1f, 0x4a, 0x5f, 0xb0, 0x28, 0x50, 0xb9, 0xb0, 0xba, 0x89, 0x9f, 0xa5, + 0x2b, 0x7c, 0xa4, 0x55, 0x95, 0xc2, 0xff, 0xd0, 0x66, 0xeb, 0x22, 0xf3, 0x8d, 0x83, 0x33, 0xe6, + 0x34, 0xcb, 0x72, 0x57, 0xf3, 0xee, 0x4d, 0x86, 0x5b, 0x1f, 0x42, 0x35, 0x77, 0xa8, 0x7c, 0x20, + 0xd6, 0x55, 0x20, 0xde, 0x98, 0x0d, 0xc4, 0xfa, 0xcc, 0x55, 0xc9, 0xc7, 0xf4, 0x87, 0x00, 0xd9, + 0x11, 0xff, 0x93, 0xdb, 0x81, 0x7f, 0x64, 0xc0, 0x95, 0xed, 0xc0, 0xf7, 0x99, 0x04, 0x4f, 0xca, + 0x79, 0x59, 0xbc, 0x1a, 0x17, 0xc6, 0xeb, 0x9b, 0x50, 0xe2, 0x42, 0x41, 0xaf, 0x72, 0xfd, 0x19, + 0xde, 0x20, 0x4a, 0x4a, 0xe4, 0xa5, 0x31, 0x9d, 0x0e, 0x42, 0xe6, 0x3b, 0xae, 0x3f, 0x92, 0x31, + 0xae, 0x7c, 0x70, 0xa0, 0x38, 0xf8, 0x67, 0x06, 0x58, 0x2a, 0xd4, 0x67, 0x72, 0xa4, 0x31, 0x9b, + 0x23, 0x5f, 0x82, 0x4a, 0x18, 0x31, 0xc7, 0x1d, 0x26, 0x2b, 0x57, 0x48, 0xc6, 0x10, 0x29, 0xfc, + 0x38, 0x88, 0x86, 0x4c, 0x4e, 0x6f, 0x13, 0x45, 0x08, 0x6c, 0x2a, 0xab, 0x93, 0xcc, 0x74, 0x2a, + 0x8d, 0xda, 0x82, 0x21, 0x52, 0x9c, 0x50, 0xe1, 0x21, 0x1d, 0x2a, 0x94, 0x58, 0x20, 0x8a, 0x10, + 0x69, 0x57, 0xf9, 0x4d, 0x86, 0xba, 0x4d, 0x34, 0x85, 0x7f, 0x65, 0x42, 0xad, 0xed, 0x46, 0x6c, + 0x18, 0x33, 0xa7, 0xe3, 0x8c, 0xa4, 0x20, 0xf3, 0x63, 0x37, 0x3e, 0xd7, 0x29, 0x5e, 0x53, 0x69, + 0x7d, 0x37, 0x67, 0x91, 0xb3, 0xf2, 0x4b, 0x41, 0x02, 0x7e, 0x45, 0xa0, 0x77, 0x01, 0x14, 0x5c, + 0x92, 0xa0, 0xbf, 0x78, 0x31, 0xe8, 0xaf, 0x48, 0x51, 0xf1, 0x29, 0x8c, 0xa4, 0xf4, 0x5c, 0x55, + 0x02, 0x2c, 0xf9, 0x22, 0x98, 0x88, 0x70, 0x96, 0xa0, 0xe1, 0x88, 0x79, 0x32, 0x5c, 0x25, 0x68, + 0x38, 0x62, 0x5e, 0x8a, 0xf1, 0xca, 0x6a, 0x4b, 0xe2, 0x1b, 0xdd, 0x02, 0x33, 0x08, 0xe5, 0x19, + 0x73, 0x8b, 0xe6, 0x0f, 0xb8, 0xb1, 0x1f, 0x12, 0x33, 0x08, 0x11, 0x06, 0x4b, 0xa1, 0xda, 0x66, + 0x45, 0x86, 0x39, 0xc8, 0xf4, 0x20, 0x61, 0x15, 0xd1, 0x23, 0xf8, 0x1a, 0x98, 0xfb, 0x21, 0x2a, + 0x43, 0xa1, 0xd7, 0xe9, 0x37, 0x96, 0xc4, 0x47, 0xbb, 0xb3, 0xd3, 0x30, 0xf0, 0x3f, 0x0c, 0xa8, + 0xec, 0x4e, 0x62, 0x2a, 0x62, 0x8c, 0x5f, 0xe4, 0xdc, 0x17, 0xc0, 0xe6, 0x31, 0x8d, 0x64, 0x51, + 0x35, 0x55, 0xe2, 0x90, 0x74, 0x9f, 0xa3, 0xd7, 0xa1, 0xc4, 0x9c, 0x11, 0x4b, 0xee, 0xfe, 0xca, + 0xa2, 0xbd, 0x12, 0x25, 0x82, 0x6e, 0x83, 0xc5, 0x87, 0x27, 0x6c, 0x4c, 0x9b, 0xc5, 0x59, 0xe1, + 0x9e, 0xe4, 0xaa, 0x3a, 0x48, 0xb4, 0x8c, 0x7c, 0x9c, 0x44, 0x41, 0x28, 0xd1, 0x79, 0x49, 0x3f, + 0x4e, 0xa2, 0x20, 0x14, 0xd8, 0x7c, 0x13, 0x9e, 0x77, 0x47, 0x7e, 0x10, 0xb1, 0x81, 0xeb, 0x3b, + 0x6c, 0x3a, 0x18, 0x06, 0xfe, 0xb1, 0xe7, 0x0e, 0x63, 0x69, 0x57, 0x9b, 0x3c, 0xa7, 0x06, 0xbb, + 0x62, 0x6c, 0x5b, 0x0f, 0xe1, 0x5b, 0x50, 0x79, 0xcc, 0xce, 0x25, 0xc8, 0xe5, 0xa8, 0x05, 0xe6, + 0xe9, 0x99, 0x2e, 0x98, 0x90, 0xec, 0xe2, 0xf1, 0x13, 0x62, 0x9e, 0x9e, 0xe1, 0xef, 0x1b, 0x60, + 0xf7, 0x34, 0x64, 0x40, 0x6f, 0x8a, 0x8c, 0x29, 0xf3, 0xac, 0xbe, 0x75, 0xe9, 0x7b, 0x24, 0x87, + 0x58, 0x48, 0x22, 0x23, 0x1c, 0x2c, 0x77, 0xa4, 0xad, 0xa4, 0x08, 0x74, 0x07, 0x56, 0xc6, 0xae, + 0x9f, 0x5c, 0xb1, 0x41, 0x6a, 0x4a, 0x75, 0xd7, 0xae, 0x8e, 0x5d, 0x5f, 0xdf, 0xb5, 0x9e, 0x32, + 0x2a, 0xfe, 0x61, 0x01, 0xec, 0xaf, 0x57, 0xcc, 0xee, 0x40, 0x65, 0x9c, 0x78, 0x52, 0x5f, 0xfc, + 0x14, 0xd8, 0xa7, 0x2e, 0x26, 0x99, 0x0c, 0xea, 0xc0, 0xb5, 0xdc, 0xac, 0xf1, 0xd4, 0x1f, 0x24, + 0xe7, 0x2c, 0x2c, 0x2e, 0x33, 0x2b, 0x99, 0x78, 0xc6, 0xd5, 0x86, 0x2c, 0x2e, 0x32, 0x64, 0x96, + 0x88, 0x4a, 0x97, 0x4a, 0x44, 0xb7, 0xe0, 0xca, 0xd0, 0x63, 0xd4, 0x1f, 0x64, 0x79, 0x44, 0x5d, + 0x93, 0x65, 0xc9, 0x3e, 0x48, 0x93, 0x89, 0x4e, 0xac, 0xe5, 0x0c, 0x2a, 0xbc, 0x06, 0x25, 0x87, + 0x79, 0x31, 0x9d, 0x7f, 0x33, 0xee, 0x47, 0x74, 0xe8, 0xb1, 0xb6, 0x18, 0x22, 0x4a, 0x02, 0xdd, + 0x06, 0x3b, 0xc1, 0x83, 0xfa, 0xa5, 0x98, 0xbe, 0x2a, 0x12, 0xa7, 0x93, 0x54, 0x02, 0xbf, 0x02, + 0x85, 0xc7, 0x4f, 0x7a, 0x17, 0x86, 0xcb, 0xf7, 0xc0, 0x7c, 0xfc, 0x24, 0x9f, 0xec, 0x6b, 0x6a, + 0x4f, 0xba, 0x87, 0x60, 0x66, 0x3d, 0x84, 0x16, 0xd8, 0x13, 0xce, 0xa2, 0x5d, 0x16, 0x53, 0x9d, + 0x69, 0x52, 0x5a, 0x54, 0x66, 0xf1, 0x08, 0x76, 0x03, 0x5f, 0x57, 0xc1, 0x84, 0xc4, 0xff, 0x2c, + 0x40, 0x59, 0x67, 0x1b, 0x31, 0xe7, 0x24, 0x05, 0xad, 0xe2, 0x33, 0x4b, 0x5d, 0x66, 0x3e, 0x75, + 0xe5, 0xbb, 0x15, 0x85, 0xcb, 0x75, 0x2b, 0xd0, 0xb7, 0xa1, 0x16, 0xaa, 0xb1, 0x7c, 0xc2, 0x7b, + 0x71, 0x5e, 0x4f, 0xff, 0x4a, 0xdd, 0x6a, 0x98, 0x11, 0xe2, 0xba, 0xca, 0x17, 0x5b, 0x4c, 0x47, + 0xd2, 0xe1, 0x35, 0x52, 0x16, 0x74, 0x9f, 0x8e, 0x9e, 0x91, 0xf6, 0x2e, 0x91, 0xb9, 0x04, 0x48, + 0x0f, 0xc2, 0x66, 0x4d, 0x06, 0xbc, 0xc8, 0x76, 0xf9, 0x44, 0x54, 0x9f, 0x4d, 0x44, 0x2f, 0x42, + 0x65, 0x18, 0x8c, 0xc7, 0xae, 0x1c, 0x5b, 0x56, 0x58, 0x41, 0x31, 0xfa, 0x1c, 0x7f, 0x01, 0x65, + 0x7d, 0x60, 0x54, 0x85, 0x72, 0xbb, 0xf3, 0x70, 0xeb, 0x70, 0x47, 0xa4, 0x42, 0x00, 0xeb, 0x41, + 0x77, 0x6f, 0x8b, 0x7c, 0xda, 0x30, 0x44, 0x5a, 0xec, 0xee, 0xf5, 0x1b, 0x26, 0xaa, 0x40, 0xe9, + 0xe1, 0xce, 0xfe, 0x56, 0xbf, 0x51, 0x40, 0x36, 0x14, 0x1f, 0xec, 0xef, 0xef, 0x34, 0x8a, 0xa8, + 0x06, 0x76, 0x7b, 0xab, 0xdf, 0xe9, 0x77, 0x77, 0x3b, 0x8d, 0x92, 0x90, 0x7d, 0xd4, 0xd9, 0x6f, + 0x58, 0xe2, 0xe3, 0xb0, 0xdb, 0x6e, 0x94, 0xc5, 0xf8, 0xc1, 0x56, 0xaf, 0xf7, 0xc9, 0x3e, 0x69, + 0x37, 0x6c, 0x31, 0x6f, 0xaf, 0x4f, 0xba, 0x7b, 0x8f, 0x1a, 0x15, 0x7c, 0x17, 0xaa, 0x39, 0xa3, + 0x09, 0x0d, 0xd2, 0x79, 0xd8, 0x58, 0x12, 0xcb, 0x3c, 0xd9, 0xda, 0x39, 0xec, 0x34, 0x0c, 0xb4, + 0x0c, 0x20, 0x3f, 0x07, 0x3b, 0x5b, 0x7b, 0x8f, 0x1a, 0x26, 0xfe, 0x81, 0x91, 0xea, 0xc8, 0x2e, + 0xc0, 0x1b, 0x60, 0x6b, 0x53, 0x27, 0x28, 0xff, 0xca, 0x9c, 0x5f, 0x48, 0x2a, 0x20, 0xc2, 0x6c, + 0x78, 0xc2, 0x86, 0xa7, 0x7c, 0x32, 0xd6, 0x51, 0x91, 0xd2, 0xea, 0x31, 0x2f, 0x6c, 0xa2, 0x73, + 0x8f, 0xa6, 0xd2, 0x8e, 0x58, 0x51, 0xca, 0xab, 0x8e, 0xd8, 0x3d, 0x80, 0xac, 0xe7, 0xb2, 0x00, + 0x9f, 0xaf, 0x40, 0x89, 0x7a, 0x2e, 0xe5, 0xba, 0x94, 0x2a, 0x02, 0x13, 0xa8, 0xe6, 0x3a, 0x35, + 0xc2, 0x61, 0xd4, 0xf3, 0x06, 0xa7, 0xec, 0x9c, 0x4b, 0x5d, 0x9b, 0x94, 0xa9, 0xe7, 0x3d, 0x66, + 0xe7, 0x1c, 0xad, 0x43, 0x49, 0x35, 0x7a, 0xcc, 0x05, 0x2d, 0x01, 0xa9, 0x4e, 0x94, 0x00, 0xbe, + 0x0d, 0x96, 0xea, 0x13, 0xe4, 0x62, 0xc6, 0x78, 0x66, 0xb5, 0xfb, 0x40, 0xef, 0x5b, 0x76, 0x15, + 0xd0, 0x1d, 0xdd, 0x54, 0xe2, 0xaa, 0x95, 0x65, 0xcc, 0x62, 0x41, 0x25, 0xa8, 0xfb, 0x49, 0x52, + 0x01, 0xb7, 0xc1, 0xbe, 0xb0, 0x65, 0xa7, 0x0d, 0x61, 0x66, 0x86, 0x58, 0xd0, 0xc4, 0xc3, 0x11, + 0x40, 0xd6, 0x78, 0xd2, 0x61, 0xac, 0x66, 0x11, 0x61, 0xbc, 0x21, 0x5c, 0xe4, 0x7a, 0x4e, 0xc4, + 0xfc, 0xa7, 0x4e, 0x9f, 0xb5, 0xab, 0x52, 0x19, 0xf4, 0x2a, 0x14, 0x65, 0x7f, 0xad, 0x30, 0x9b, + 0xb0, 0xd2, 0xe6, 0x9a, 0x1c, 0xc5, 0x47, 0x50, 0x57, 0x85, 0x94, 0xb0, 0xcf, 0x27, 0x8c, 0x5f, + 0x08, 0xd7, 0x56, 0x01, 0xd2, 0x34, 0x9b, 0x74, 0x0c, 0x73, 0x1c, 0x11, 0x28, 0xc7, 0x2e, 0xf3, + 0x9c, 0xe4, 0x54, 0x9a, 0xc2, 0xef, 0x41, 0x2d, 0x59, 0x43, 0x36, 0x04, 0x6e, 0xa5, 0x25, 0x3d, + 0x89, 0x4b, 0xe1, 0x10, 0x25, 0xb2, 0x17, 0x38, 0x69, 0x35, 0xc7, 0x7f, 0x35, 0x13, 0x4d, 0xfd, + 0xdc, 0x9d, 0x01, 0x8c, 0xc6, 0x3c, 0x60, 0x9c, 0x05, 0x5f, 0xe6, 0xa5, 0xc1, 0xd7, 0xb7, 0xa0, + 0xe2, 0x48, 0xe4, 0xe1, 0x9e, 0x25, 0xa9, 0x6f, 0x75, 0x11, 0xca, 0xd0, 0xf8, 0xc4, 0x3d, 0x63, + 0x24, 0x53, 0x10, 0x7b, 0x8a, 0x83, 0x53, 0xe6, 0xbb, 0x5f, 0xc8, 0x77, 0xbd, 0x38, 0x78, 0xc6, + 0xc8, 0x5a, 0x2f, 0x0a, 0x8d, 0xe8, 0xd6, 0x4b, 0xd2, 0x76, 0xb2, 0x72, 0x6d, 0xa7, 0x6b, 0x60, + 0x4d, 0x42, 0xce, 0xa2, 0x38, 0x41, 0xa9, 0x8a, 0x4a, 0x91, 0x5e, 0x45, 0xcb, 0x52, 0x7f, 0x84, + 0xbf, 0x01, 0x95, 0x74, 0x2f, 0x22, 0xdf, 0xec, 0xed, 0xef, 0x75, 0x54, 0x76, 0xe8, 0xee, 0xb5, + 0x3b, 0xdf, 0x6d, 0x18, 0x22, 0x63, 0x91, 0xce, 0x93, 0x0e, 0xe9, 0x75, 0x1a, 0xa6, 0xc8, 0x2c, + 0xed, 0xce, 0x4e, 0xa7, 0xdf, 0x69, 0x14, 0x3e, 0x2a, 0xda, 0xe5, 0x86, 0x4d, 0x6c, 0x36, 0x0d, + 0x3d, 0x77, 0xe8, 0xc6, 0xf8, 0x53, 0xb0, 0x77, 0x69, 0xf8, 0xd4, 0xeb, 0x23, 0x2b, 0x48, 0x13, + 0xdd, 0xdb, 0xd0, 0xc5, 0xe3, 0x35, 0x28, 0xeb, 0xac, 0x91, 0x16, 0xfd, 0xb9, 0xac, 0x92, 0x8c, + 0xe3, 0x5f, 0x1a, 0xb0, 0xb2, 0x1b, 0x9c, 0xb1, 0xb4, 0x0a, 0x1f, 0xd0, 0x73, 0x2f, 0xa0, 0xce, + 0x57, 0xb8, 0xf1, 0x26, 0x5c, 0xe1, 0xc1, 0x24, 0x1a, 0xb2, 0xc1, 0x5c, 0x6f, 0xa5, 0xae, 0xd8, + 0x8f, 0x74, 0x38, 0x62, 0x81, 0x71, 0x78, 0x9c, 0x49, 0x15, 0xa4, 0x54, 0x55, 0x30, 0x13, 0x99, + 0x14, 0x4e, 0x14, 0x2f, 0x03, 0x27, 0xf0, 0x1f, 0x0c, 0xa8, 0x77, 0xa6, 0x61, 0x10, 0xc5, 0xc9, + 0x56, 0x9f, 0x17, 0x6f, 0x87, 0xcf, 0x93, 0xcb, 0x50, 0x24, 0xa5, 0x88, 0x7d, 0xde, 0xbd, 0xb0, + 0xf1, 0x73, 0x0f, 0x2c, 0x31, 0xd9, 0x84, 0xeb, 0x50, 0x7a, 0x29, 0x59, 0x73, 0x66, 0xe2, 0x8d, + 0x9e, 0x94, 0x21, 0x5a, 0x36, 0xdf, 0x74, 0x2b, 0xe6, 0x9b, 0x6e, 0xf8, 0x3e, 0x58, 0x4a, 0x34, + 0xe7, 0xe7, 0x2a, 0x94, 0x7b, 0x87, 0xdb, 0xdb, 0x9d, 0x5e, 0xaf, 0x61, 0xa0, 0x3a, 0x54, 0xda, + 0x87, 0x07, 0x3b, 0xdd, 0xed, 0xad, 0xbe, 0xf6, 0xf5, 0xc3, 0xad, 0xee, 0x4e, 0xa7, 0xdd, 0x28, + 0xe0, 0xdf, 0x18, 0x50, 0xcd, 0xc1, 0x19, 0x74, 0x5f, 0x00, 0x53, 0x91, 0xbb, 0x93, 0x54, 0xb8, + 0xb6, 0x00, 0xf4, 0x6c, 0x6c, 0x2b, 0x11, 0xdd, 0xc7, 0xd1, 0x0a, 0x22, 0x3c, 0xe9, 0x51, 0x10, + 0xe9, 0xe6, 0x4f, 0x91, 0x68, 0x0a, 0xbd, 0x02, 0x35, 0xf1, 0x14, 0xa4, 0x9c, 0xbb, 0x23, 0x9f, + 0x39, 0xba, 0x46, 0x88, 0xe7, 0xe1, 0x96, 0x66, 0xb5, 0xee, 0x43, 0x2d, 0x3f, 0xe7, 0x82, 0x47, + 0xee, 0x0c, 0x22, 0x29, 0xe6, 0x1f, 0xb5, 0x2f, 0x43, 0x5d, 0xbc, 0xdc, 0xdd, 0x31, 0xe3, 0x31, + 0x1d, 0x87, 0xb2, 0xba, 0xeb, 0xed, 0x17, 0x89, 0x19, 0x73, 0x7c, 0x13, 0x6a, 0x07, 0x8c, 0x45, + 0x84, 0xf1, 0x30, 0xf0, 0xb9, 0x7c, 0xc3, 0x69, 0xf3, 0xab, 0xd2, 0xa1, 0x29, 0x7c, 0x04, 0x15, + 0x81, 0xbe, 0x1f, 0xd0, 0x78, 0x78, 0xf2, 0x75, 0x11, 0xfa, 0x4d, 0x28, 0x87, 0xca, 0x6d, 0x1a, + 0x26, 0xab, 0x9e, 0xa4, 0x76, 0x25, 0x49, 0x06, 0xf1, 0x75, 0x28, 0xec, 0x4d, 0xc6, 0xf9, 0x7f, + 0x82, 0x8a, 0x12, 0xc5, 0xe1, 0x87, 0x50, 0x4b, 0x80, 0xa2, 0x44, 0x6e, 0x02, 0x77, 0x78, 0x2e, + 0xf3, 0x25, 0xee, 0x30, 0x34, 0xee, 0x90, 0x8c, 0x3e, 0xbf, 0x20, 0xb6, 0x36, 0x7f, 0x6f, 0x40, + 0x51, 0xec, 0x50, 0x24, 0xf7, 0xce, 0xf0, 0x24, 0x40, 0x33, 0x1b, 0x69, 0xcd, 0x50, 0x78, 0x09, + 0xbd, 0xa5, 0x7a, 0xa4, 0x49, 0x63, 0xf9, 0x6a, 0xfe, 0x90, 0xd2, 0x10, 0x4f, 0x69, 0x6c, 0x42, + 0xf5, 0xa3, 0xc0, 0xf5, 0xb7, 0x55, 0xeb, 0x10, 0x2d, 0x32, 0xcb, 0x53, 0x3a, 0xef, 0x80, 0xd5, + 0xe5, 0xc2, 0x07, 0x8b, 0xc5, 0xd3, 0x07, 0x5b, 0xde, 0x4d, 0x78, 0x69, 0xf3, 0xe7, 0x05, 0x28, + 0x7e, 0xc6, 0xa2, 0x00, 0xdd, 0x83, 0xb2, 0x6e, 0x5b, 0xa0, 0xb9, 0xf6, 0x44, 0x2b, 0xbd, 0xaf, + 0x73, 0x7d, 0x0d, 0xbc, 0x84, 0xde, 0x05, 0x4b, 0x17, 0x85, 0xd9, 0xde, 0x4a, 0xeb, 0x59, 0x77, + 0x1c, 0x2f, 0xad, 0x1b, 0x6f, 0x19, 0xe8, 0x0e, 0x58, 0x2a, 0xd8, 0xe7, 0x6c, 0xb7, 0x08, 0xff, + 0xe3, 0x25, 0xa9, 0x50, 0xed, 0x9d, 0x04, 0x13, 0xcf, 0xe9, 0xb1, 0xe8, 0x8c, 0xa1, 0xb9, 0x4e, + 0x5e, 0x6b, 0x8e, 0xc6, 0x4b, 0xe8, 0x4d, 0x00, 0x15, 0xfa, 0x87, 0xae, 0xc3, 0x51, 0x35, 0x19, + 0xdf, 0x9b, 0x8c, 0x5b, 0x0d, 0xb9, 0x64, 0x72, 0x31, 0xba, 0x0e, 0x57, 0xe2, 0xb9, 0xf0, 0xfe, + 0x4a, 0xf1, 0xb7, 0xa1, 0xae, 0x2e, 0xd3, 0x7e, 0xb4, 0x25, 0x6e, 0x20, 0x9a, 0x7f, 0x74, 0xb5, + 0xe6, 0x19, 0x78, 0x09, 0xdd, 0x07, 0xbb, 0x1f, 0x9d, 0x2b, 0xf9, 0xe7, 0xd3, 0x0d, 0xe7, 0xef, + 0x55, 0x6b, 0x31, 0x1b, 0x2f, 0x6d, 0xfe, 0xbd, 0x00, 0xd6, 0x27, 0x41, 0x74, 0xca, 0x22, 0xb4, + 0x01, 0x96, 0x7c, 0x17, 0xe6, 0x42, 0x29, 0x7d, 0x27, 0x2e, 0x5a, 0xf6, 0x36, 0x54, 0xa4, 0xd1, + 0xfa, 0x94, 0x9f, 0x66, 0x6e, 0x92, 0xff, 0x46, 0x66, 0x76, 0x53, 0xa0, 0x00, 0x2f, 0xa1, 0xef, + 0xc0, 0xb5, 0xb4, 0x52, 0x6c, 0xf9, 0x8e, 0xaa, 0xbc, 0x6d, 0x1a, 0x53, 0xb4, 0x32, 0xff, 0xda, + 0x12, 0x97, 0xa8, 0x55, 0xcd, 0x1e, 0x55, 0x3d, 0xe9, 0xa9, 0xbb, 0x50, 0xec, 0x89, 0x13, 0x66, + 0xff, 0x25, 0x66, 0xff, 0xaa, 0xb4, 0x50, 0x9e, 0x99, 0xae, 0xf9, 0x1e, 0x58, 0x6a, 0x9d, 0xcc, + 0x2c, 0x33, 0x70, 0xa8, 0xb5, 0x32, 0xcf, 0xd6, 0x8a, 0xb7, 0xc0, 0xde, 0x75, 0x7d, 0xd5, 0x54, + 0x9c, 0x0d, 0xa4, 0xbc, 0x07, 0xf1, 0x12, 0x7a, 0x1f, 0x2c, 0x95, 0xf8, 0xb3, 0x15, 0x66, 0x0a, + 0x41, 0x6b, 0x31, 0x1b, 0x2f, 0xa1, 0xbb, 0xd0, 0x20, 0x6c, 0xc8, 0xdc, 0x5c, 0x01, 0x45, 0xf9, + 0x33, 0xcf, 0x5f, 0xc4, 0x75, 0x03, 0x7d, 0x00, 0xf5, 0x99, 0x82, 0x8b, 0xd2, 0xe2, 0xb3, 0xa8, + 0x0e, 0xcf, 0x4f, 0xf0, 0xa0, 0xf1, 0xbb, 0x2f, 0x57, 0x8d, 0x3f, 0x7d, 0xb9, 0x6a, 0xfc, 0xe5, + 0xcb, 0x55, 0xe3, 0xa7, 0x7f, 0x5b, 0x5d, 0x3a, 0xb2, 0xe4, 0x9f, 0xe0, 0x6f, 0xff, 0x2b, 0x00, + 0x00, 0xff, 0xff, 0x8c, 0xb2, 0x81, 0x11, 0x29, 0x1f, 0x00, 0x00, } diff --git a/protos/internal.proto b/protos/internal.proto index f4328e1edfb..d110da34dc2 100644 --- a/protos/internal.proto +++ b/protos/internal.proto @@ -166,13 +166,13 @@ message Tablet { } message DirectedEdge { - fixed64 entity = 1; // Subject or source node / UID. - string attr = 2; // Attribute or predicate. Labels the edge. - bytes value = 3; // Edge points to a value. + fixed64 entity = 1; // Subject or source node / UID. + string attr = 2; // Attribute or predicate. Labels the edge. + bytes value = 3; // Edge points to a value. Posting.ValType value_type = 4; // The type of the value - fixed64 value_id = 5; // Object or destination node / UID. - string label = 6; - string lang = 7; + fixed64 value_id = 5; // Object or destination node / UID. + string label = 6; + string lang = 7; enum Op { SET = 0; DEL = 1; @@ -182,28 +182,35 @@ message DirectedEdge { } message Mutations { - uint32 group_id = 1; - uint64 start_ts = 2; - repeated DirectedEdge edges = 3; + uint32 group_id = 1; + uint64 start_ts = 2; + repeated DirectedEdge edges = 3; repeated SchemaUpdate schema = 4; - bool drop_all = 5; - bool ignore_index_conflict = 6; + bool drop_all = 5; + bool ignore_index_conflict = 6; } message KeyValues { repeated KV kv = 1; } +message Snapshot { + RaftContext context = 1; + uint64 index = 2; + uint64 min_pending_start_ts = 3; +} + message Proposal { - uint32 deprecated_id = 1; // delete this field in later versions. Its for backward compatibility. + uint32 deprecated_id = 1; // delete this field in later versions. Its for backward compatibility. api.TxnContext deprecated_txn_context = 3; - Mutations mutations = 2; - repeated KV kv = 4; - MembershipState state = 5; - string clean_predicate = 6; // Delete the predicate which was moved to other group. - string key = 7; - OracleDelta delta = 8; + Mutations mutations = 2; + repeated KV kv = 4; + MembershipState state = 5; + string clean_predicate = 6; // Delete the predicate which was moved to other group. + string key = 7; + OracleDelta delta = 8; + Snapshot snapshot = 9; // Used to tell the group when to take snapshot. } message KVS { @@ -211,8 +218,8 @@ message KVS { } message KV { - bytes key = 1; - bytes val = 2; + bytes key = 1; + bytes val = 2; bytes userMeta = 3; uint64 version = 4; } @@ -357,7 +364,7 @@ message ExportPayload { message OracleDelta { map commits = 1; repeated uint64 aborts = 2; - uint64 max_pending = 3; + uint64 max_assigned = 3; // implement tmax. } diff --git a/query/query_test.go b/query/query_test.go index b6456728cb9..7cea867fee6 100644 --- a/query/query_test.go +++ b/query/query_test.go @@ -55,7 +55,7 @@ func commitTs(startTs uint64) uint64 { Commits: map[uint64]uint64{ startTs: commit, }, - MaxPending: atomic.LoadUint64(&ts), + MaxAssigned: atomic.LoadUint64(&ts), } posting.Oracle().ProcessOracleDelta(od) return commit @@ -6100,7 +6100,7 @@ func (c *raftServer) JoinCluster(ctx context.Context, in *intern.RaftContext) (* func updateMaxPending() { for mp := range maxPendingCh { posting.Oracle().ProcessOracleDelta(&intern.OracleDelta{ - MaxPending: mp, + MaxAssigned: mp, }) } diff --git a/systest/loader_test.go b/systest/loader_test.go index 1524050f3bb..432e74b6ab8 100644 --- a/systest/loader_test.go +++ b/systest/loader_test.go @@ -14,11 +14,10 @@ import ( "os" "os/exec" "path/filepath" - "syscall" "testing" - "time" ) +// TODO: Convert this to Docker based test. func TestLoaderXidmap(t *testing.T) { tmpDir, err := ioutil.TempDir("", "") check(t, err) @@ -60,18 +59,18 @@ func TestLoaderXidmap(t *testing.T) { } // Restart Dgraph before taking an export. - cluster.dgraph.Process.Signal(syscall.SIGINT) - if _, err = cluster.dgraph.Process.Wait(); err != nil { - cluster.Close() - t.Fatalf("Error while waiting for Dgraph process to be killed: %v", err) - } + // cluster.dgraph.Process.Signal(syscall.SIGINT) + // if _, err = cluster.dgraph.Process.Wait(); err != nil { + // cluster.Close() + // t.Fatalf("Error while waiting for Dgraph process to be killed: %v", err) + // } - cluster.dgraph.Process = nil - if err := cluster.dgraph.Start(); err != nil { - cluster.Close() - t.Fatalf("Couldn't start Dgraph server again: %v\n", err) - } - time.Sleep(5 * time.Second) + // cluster.dgraph.Process = nil + // if err := cluster.dgraph.Start(); err != nil { + // cluster.Close() + // t.Fatalf("Couldn't start Dgraph server again: %v\n", err) + // } + // time.Sleep(5 * time.Second) resp, err := http.Get(fmt.Sprintf("http://localhost:%d/admin/export", cluster.dgraphPortOffset+8080)) if err != nil { diff --git a/worker/draft.go b/worker/draft.go index 8c8f28197cc..b9ba6a90518 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -12,6 +12,7 @@ import ( "encoding/binary" "errors" "fmt" + "math" "sync" "time" @@ -20,7 +21,6 @@ import ( "golang.org/x/net/context" "golang.org/x/net/trace" - "github.com/dgraph-io/badger/y" "github.com/dgraph-io/dgo/protos/api" dy "github.com/dgraph-io/dgo/y" "github.com/dgraph-io/dgraph/conn" @@ -388,18 +388,15 @@ func (n *node) applyMutations(proposal *intern.Proposal, index uint64) error { posting.Txns().Reset() schema.State().DeleteAll() err := posting.DeleteAll() - posting.TxnMarks().Done(index) return err } if proposal.Mutations.StartTs == 0 { - posting.TxnMarks().Done(index) return errors.New("StartTs must be provided.") } startTs := proposal.Mutations.StartTs if len(proposal.Mutations.Schema) > 0 { - defer posting.TxnMarks().Done(index) for _, supdate := range proposal.Mutations.Schema { // This is neceassry to ensure that there is no race between when we start reading // from badger and new mutation getting commited via raft and getting applied. @@ -438,7 +435,6 @@ func (n *node) applyMutations(proposal *intern.Proposal, index uint64) error { if edge.Entity == 0 && bytes.Equal(edge.Value, []byte(x.Star)) { // We should only have one edge drop in one mutation call. ctx, _ := n.props.CtxAndTxn(proposal.Key) - defer posting.TxnMarks().Done(index) if err := waitForConflictResolution(edge.Attr); err != nil { return err } @@ -466,6 +462,7 @@ func (n *node) applyMutations(proposal *intern.Proposal, index uint64) error { } m := proposal.Mutations + posting.Oracle().RegisterStartTs(m.StartTs) pctx := n.props.pctx(proposal.Key) pctx.txn = updateTxns(index, m.StartTs) for _, edge := range m.Edges { @@ -501,17 +498,15 @@ func (n *node) applyCommitted(proposal *intern.Proposal, index uint64) error { } pctx.index = index - // TODO: We should be able to remove this as well. - posting.TxnMarks().Begin(index) if proposal.Mutations != nil { // syncmarks for this shouldn't be marked done until it's comitted. n.elog.Printf("Applying mutations for key: %s", proposal.Key) return n.applyMutations(proposal, index) } - defer posting.TxnMarks().Done(index) if len(proposal.Kv) > 0 { - return n.processKeyValues(proposal.Key, proposal.Kv) + ctx, _ := n.props.CtxAndTxn(proposal.Key) + return populateKeyValues(ctx, proposal.Kv) } else if proposal.State != nil { n.elog.Printf("Applying state for key: %s", proposal.Key) @@ -521,7 +516,8 @@ func (n *node) applyCommitted(proposal *intern.Proposal, index uint64) error { return nil } else if len(proposal.CleanPredicate) > 0 { - return n.deletePredicate(proposal.Key, proposal.CleanPredicate) + ctx, _ := n.props.CtxAndTxn(proposal.Key) + return posting.DeletePredicate(ctx, proposal.CleanPredicate) } else if proposal.DeprecatedTxnContext != nil { n.elog.Printf("Applying txncontext for key: %s", proposal.Key) @@ -534,9 +530,20 @@ func (n *node) applyCommitted(proposal *intern.Proposal, index uint64) error { delta.Commits[tctx.StartTs] = tctx.CommitTs } return n.commitOrAbort(proposal.Key, delta) + } else if proposal.Delta != nil { n.elog.Printf("Applying Oracle Delta for key: %s", proposal.Key) return n.commitOrAbort(proposal.Key, proposal.Delta) + + } else if proposal.Snapshot != nil { + snap := proposal.Snapshot + n.elog.Printf("Creating snapshot: %+v", snap) + x.Printf("Creating snapshot at index: %d. MinPendingStartTs: %d.\n", + snap.Index, snap.MinPendingStartTs) + data, err := snap.Marshal() + x.Check(err) + return n.Store.CreateSnapshot(snap.Index, n.ConfState(), data) + } else { x.Fatalf("Unknown proposal") } @@ -589,16 +596,6 @@ func (n *node) commitOrAbort(pkey string, delta *intern.OracleDelta) error { return nil } -func (n *node) deletePredicate(pkey string, predicate string) error { - ctx, _ := n.props.CtxAndTxn(pkey) - return posting.DeletePredicate(ctx, predicate) -} - -func (n *node) processKeyValues(pkey string, kvs []*intern.KV) error { - ctx, _ := n.props.CtxAndTxn(pkey) - return populateKeyValues(ctx, kvs) -} - func (n *node) applyAllMarks(ctx context.Context) { // Get index of last committed. lastIndex := n.Applied.LastIndex() @@ -634,6 +631,11 @@ func (n *node) retrieveSnapshot() error { // Need to clear pl's stored in memory for the case when retrieving snapshot with // index greater than this node's last index // Should invalidate/remove pl's to this group only ideally + // + // We can safely evict posting lists from memory. Because, all the updates corresponding to txn + // commits up until then have already been written to pstore. And the way we take snapshots, we + // keep all the pre-writes for a pending transaction, so they will come back to memory, as Raft + // logs are replayed. posting.EvictLRU() if _, err := n.populateShard(pstore, pool); err != nil { return fmt.Errorf("Cannot retrieve snapshot from peer, error: %v\n", err) @@ -654,17 +656,22 @@ func (n *node) Run() { ticker := time.NewTicker(20 * time.Millisecond) defer ticker.Stop() - // Ensure we don't exit unless any snapshot in progress in done. - closer := y.NewCloser(2) - go n.snapshotPeriodically(closer) - - logTicker := time.NewTicker(time.Minute) - defer logTicker.Stop() + slowTicker := time.NewTicker(time.Minute) + defer slowTicker.Stop() for { select { - case <-logTicker.C: + case <-slowTicker.C: n.elog.Printf("Size of applyCh: %d", len(n.applyCh)) + if leader { + // We use disk based storage for Raft. So, we're not too concerned about + // snapshotting. We just need to do enough, so that we don't have a huge backlog of + // entries to process on a restart. + if err := n.calculateSnapshot(1000); err != nil { + x.Errorf("While taking snapshot: %v\n", err) + } + go n.abortOldTransactions() + } case <-ticker.C: n.Raft().Tick() @@ -704,9 +711,10 @@ func (n *node) Run() { // We don't send snapshots to other nodes. But, if we get one, that means // either the leader is trying to bring us up to state; or this is the // snapshot that I created. Only the former case should be handled. - var rc intern.RaftContext - x.Check(rc.Unmarshal(rd.Snapshot.Data)) - x.AssertTrue(rc.Group == n.gid) + var snap intern.Snapshot + x.Check(snap.Unmarshal(rd.Snapshot.Data)) + rc := snap.GetContext() + x.AssertTrue(rc.GetGroup() == n.gid) if rc.Id != n.Id { // NOTE: Retrieving snapshot here is OK, after storing it above in WAL, because // rc.Id != n.Id. @@ -796,12 +804,10 @@ func (n *node) Run() { } } n.Raft().Stop() - closer.SignalAndWait() close(n.done) }() } else { n.Raft().Stop() - closer.SignalAndWait() close(n.done) } case <-n.done: @@ -820,71 +826,136 @@ func (n *node) Stop() { <-n.done // wait for Run to respond. } -func (n *node) snapshotPeriodically(closer *y.Closer) { - ticker := time.NewTicker(30 * time.Second) - defer ticker.Stop() - - for { - select { - case <-ticker.C: - // Some proposals like predicate move can consume around 32MB per proposal, so keeping - // too many proposals would increase the memory usage so snapshot as soon as - // possible - n.snapshot(10) - - case <-closer.HasBeenClosed(): - closer.Done() - return - } - } -} - -func (n *node) abortOldTransactions(pending uint64) { +// abortOldTransactions would find txns which have done pre-writes, but have been pending for a +// while. The time that is used is based on the last pre-write seen, so if a txn is doing a +// pre-write multiple times, we'll pick the timestamp of the last pre-write. Thus, this function +// would only act on the txns which have not been active in the last N minutes, and send them for +// abort. Note that only the leader runs this function. +// NOTE: We might need to get the results of TryAbort and propose them. But, it's unclear if we need +// to, because Zero should stream out the aborts anyway. +func (n *node) abortOldTransactions() { pl := groups().Leader(0) if pl == nil { return } zc := intern.NewZeroClient(pl.Get()) // Aborts if not already committed. - startTimestamps := posting.Txns().TxnsSinceSnapshot(pending) + startTimestamps := posting.Oracle().TxnOlderThan(5 * time.Minute) + if len(startTimestamps) == 0 { + return + } req := &intern.TxnTimestamps{Ts: startTimestamps} - zc.TryAbort(context.Background(), req) -} - -func (n *node) snapshot(skip uint64) { - txnWatermark := posting.TxnMarks().DoneUntil() - existing, err := n.Store.Snapshot() - x.Checkf(err, "Unable to get existing snapshot") - - lastSnapshotIdx := existing.Metadata.Index - if txnWatermark <= lastSnapshotIdx+skip { - appliedWatermark := n.Applied.DoneUntil() - // If difference grows above 1.5 * ForceAbortDifference we try to abort old transactions - if appliedWatermark-txnWatermark > 1.5*x.ForceAbortDifference && skip != 0 { - // Print warning if difference grows above 3 * x.ForceAbortDifference. Shouldn't ideally - // happen as we abort oldest 20% when it grows above 1.5 times. - if appliedWatermark-txnWatermark > 3*x.ForceAbortDifference { - x.Printf("Couldn't take snapshot, txn watermark: [%d], applied watermark: [%d]\n", - txnWatermark, appliedWatermark) - } - // Try aborting pending transactions here. - n.abortOldTransactions(appliedWatermark - txnWatermark) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + _, err := zc.TryAbort(ctx, req) + x.Printf("Aborted txns with start ts: %v. Error: %v\n", startTimestamps, err) +} + +// calculateSnapshot would calculate a snapshot index, considering these factors: +// - We still keep at least keepN number of Raft entries. If we cut too short, +// then the chances that a crashed follower needs to retrieve the entire state +// from leader increases. So, we keep a buffer to allow a follower to catch up. +// - We can discard at least half of keepN number of entries. +// - We are not overshooting the max applied entry. That is, we're not removing +// Raft entries before they get applied. +// - We are considering the minimum start ts that has yet to be committed or +// aborted. This way, we still keep all the mutations corresponding to this +// start ts in the Raft logs. This is important, because we don't persist +// pre-writes to disk in pstore. +// - Finally, this function would propose this snapshot index, so the entire +// group can apply it to their Raft stores. +func (n *node) calculateSnapshot(keepN int) error { + tr := trace.New("Dgraph.Internal", "Propose.Snapshot") + defer tr.Finish() + + // Each member of the Raft group is taking snapshots independently, as mentioned in Section 7 of + // Raft paper. We want to avoid taking snapshots too close to the LastIndex, so that in case the + // leader changes, and the followers haven't yet caught up to this index, they would need to + // retrieve the entire state (snapshot) of the DB. So, we should always wait to accumulate skip + // entries before we start taking a snapshot. + count, err := n.Store.NumEntries() + if err != nil { + tr.LazyPrintf("Error: %v", err) + tr.SetError() + return err + } + tr.LazyPrintf("Found Raft entries: %d", count) + if count < 2*keepN { + // We wait to build up at least 2*keepN entries, and then discard keepN entries. + tr.LazyPrintf("Skipping due to insufficient entries") + return nil + } + discard := count - keepN + + first, err := n.Store.FirstIndex() + if err != nil { + tr.LazyPrintf("Error: %v", err) + tr.SetError() + return err + } + tr.LazyPrintf("First index: %d", first) + last := first + uint64(discard) + if last > n.Applied.DoneUntil() { + tr.LazyPrintf("Skipping because last index: %d > applied", last) + return nil + } + entries, err := n.Store.Entries(first, last, math.MaxUint64) + if err != nil { + tr.LazyPrintf("Error: %v", err) + tr.SetError() + return err + } + + // We find the minimum start ts for which a decision to commit or abort is still pending. We + // should not discard mutations corresponding to this start ts, because we don't persist + // mutations until they are committed. + minPending := posting.Oracle().MinPendingStartTs() + tr.LazyPrintf("Found min pending start ts: %d", minPending) + var snapshotIdx uint64 + for _, entry := range entries { + if entry.Type != raftpb.EntryNormal { + continue } - return + var proposal intern.Proposal + if err := proposal.Unmarshal(entry.Data); err != nil { + tr.LazyPrintf("Error: %v", err) + tr.SetError() + return err + } + mu := proposal.Mutations + if mu != nil && mu.StartTs >= minPending { + break + } + snapshotIdx = entry.Index + } + tr.LazyPrintf("Got snapshotIdx: %d. Discarding: %d", snapshotIdx, snapshotIdx-first) + // We should discard at least half of keepN. Otherwise, why bother. + if int(snapshotIdx-first) < int(float64(keepN)*0.5) { + tr.LazyPrintf("Skipping snapshot because insufficient discard entries") + x.Printf("Skipping snapshot. Insufficient discard entries: %d. MinPendingStartTs: %d\n", + snapshotIdx-first, minPending) + return nil } - snapshotIdx := txnWatermark - skip - if tr, ok := trace.FromContext(n.ctx); ok { - tr.LazyPrintf("Taking snapshot for group: %d at watermark: %d\n", n.gid, snapshotIdx) + snap := &intern.Snapshot{ + Context: n.RaftContext, + Index: snapshotIdx, + MinPendingStartTs: minPending, } + proposal := &intern.Proposal{ + Snapshot: snap, + } + tr.LazyPrintf("Proposing snapshot: %+v\n", snap) - rc, err := n.RaftContext.Marshal() + data, err := proposal.Marshal() x.Check(err) - - err = n.Store.CreateSnapshot(snapshotIdx, n.ConfState(), rc) - x.Checkf(err, "While creating snapshot") - x.Printf("Writing snapshot at index: %d, applied mark: %d\n", snapshotIdx, - n.Applied.DoneUntil()) + if err := n.Raft().Propose(context.Background(), data); err != nil { + tr.LazyPrintf("Error: %v", err) + tr.SetError() + return err + } + tr.LazyPrintf("Done best-effort proposing.") + return nil } func (n *node) joinPeers() error { @@ -937,7 +1008,6 @@ func (n *node) InitAndStartNode() { idx, restart, err := n.PastLife() x.Check(err) n.Applied.SetDoneUntil(idx) - posting.TxnMarks().SetDoneUntil(idx) if _, hasPeer := groups().MyPeer(); !restart && hasPeer { // The node has other peers, it might have crashed after joining the cluster and before diff --git a/worker/groups.go b/worker/groups.go index 25b3faa62a5..26d93eebbfe 100644 --- a/worker/groups.go +++ b/worker/groups.go @@ -756,8 +756,8 @@ func (g *groupi) processOracleDeltaStream() { delta.Commits[start] = commit } delta.Aborts = append(delta.Aborts, more.Aborts...) - if delta.MaxPending < more.MaxPending { - delta.MaxPending = more.MaxPending + if delta.MaxAssigned < more.MaxAssigned { + delta.MaxAssigned = more.MaxAssigned } default: break SLURP diff --git a/worker/predicate_test.go b/worker/predicate_test.go index 4354d335c07..b0dd5f3d812 100644 --- a/worker/predicate_test.go +++ b/worker/predicate_test.go @@ -50,7 +50,7 @@ func commitTs(startTs uint64) uint64 { Commits: map[uint64]uint64{ startTs: commit, }, - MaxPending: atomic.LoadUint64(&ts), + MaxAssigned: atomic.LoadUint64(&ts), } posting.Oracle().ProcessOracleDelta(od) return commit diff --git a/worker/worker.go b/worker/worker.go index 16fd9ffd6ff..71ceb58ef4f 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -108,5 +108,4 @@ func BlockingStop() { workerServer.GracefulStop() // blocking stop server groups().Node.applyAllMarks(ctx) posting.StopLRUEviction() - groups().Node.snapshot(0) }